From 4b2011ec9da1245923b5cbd883240fef0dbf3ef0 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Wed, 19 Oct 2016 19:36:21 -0700 Subject: [PATCH 01/48] [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. --- .../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 c0200299376ca..f56bb39d10791 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 @@ -124,7 +124,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 f313117bc93b0bf560528b316d3e6947caa96296 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 19 Oct 2016 22:22:35 -0700 Subject: [PATCH 02/48] [SPARK-18012][SQL] Simplify WriterContainer ## What changes were proposed in this pull request? This patch refactors WriterContainer to simplify the logic and make control flow more obvious.The previous code setup made it pretty difficult to track the actual dependencies on variables and setups because the driver side and the executor side were using the same set of variables. ## How was this patch tested? N/A - this should be covered by existing tests. Author: Reynold Xin Closes #15551 from rxin/writercontainer-refactor. --- .../InsertIntoHadoopFsRelationCommand.scala | 79 +-- .../execution/datasources/WriteOutput.scala | 480 ++++++++++++++++++ .../datasources/WriterContainer.scala | 445 ---------------- .../apache/spark/sql/internal/SQLConf.scala | 9 - 4 files changed, 492 insertions(+), 521 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala 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 99ca3df673568..22dbe7149531c 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 @@ -20,18 +20,12 @@ package org.apache.spark.sql.execution.datasources import java.io.IOException import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapreduce._ -import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat -import org.apache.spark._ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.RunnableCommand -import org.apache.spark.sql.internal.SQLConf /** * A command for writing data to a [[HadoopFsRelation]]. Supports both overwriting and appending. @@ -40,20 +34,6 @@ import org.apache.spark.sql.internal.SQLConf * 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. - * - * Basic work flow of this command is: - * - * 1. Driver side setup, including output committer initialization and data source specific - * preparation work for the write job to be issued. - * 2. Issues a write job consists of one or more executor side tasks, each of which writes all - * rows within an RDD partition. - * 3. If no exception is thrown in a task, commits that task, otherwise aborts that task; If any - * exception is thrown during task commitment, also aborts that task. - * 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. */ case class InsertIntoHadoopFsRelationCommand( outputPath: Path, @@ -103,52 +83,17 @@ case class InsertIntoHadoopFsRelationCommand( val isAppend = pathExists && (mode == SaveMode.Append) if (doInsertion) { - val job = Job.getInstance(hadoopConf) - job.setOutputKeyClass(classOf[Void]) - job.setOutputValueClass(classOf[InternalRow]) - FileOutputFormat.setOutputPath(job, qualifiedOutputPath) - - val partitionSet = AttributeSet(partitionColumns) - val dataColumns = query.output.filterNot(partitionSet.contains) - - val queryExecution = Dataset.ofRows(sparkSession, query).queryExecution - SQLExecution.withNewExecutionId(sparkSession, queryExecution) { - val relation = - WriteRelation( - sparkSession, - dataColumns.toStructType, - qualifiedOutputPath.toString, - fileFormat.prepareWrite(sparkSession, _, options, dataColumns.toStructType), - bucketSpec) - - val writerContainer = if (partitionColumns.isEmpty && bucketSpec.isEmpty) { - new DefaultWriterContainer(relation, job, isAppend) - } else { - new DynamicPartitionWriterContainer( - relation, - job, - partitionColumns = partitionColumns, - dataColumns = dataColumns, - inputSchema = query.output, - PartitioningUtils.DEFAULT_PARTITION_NAME, - sparkSession.sessionState.conf.partitionMaxFiles, - isAppend) - } - - // This call shouldn't be put into the `try` block below because it only initializes and - // prepares the job, any exception thrown from here shouldn't cause abortJob() to be called. - writerContainer.driverSideSetup() - - try { - sparkSession.sparkContext.runJob(queryExecution.toRdd, writerContainer.writeRows _) - writerContainer.commitJob() - refreshFunction() - } catch { case cause: Throwable => - logError("Aborting job.", cause) - writerContainer.abortJob() - throw new SparkException("Job aborted.", cause) - } - } + WriteOutput.write( + sparkSession, + query, + fileFormat, + qualifiedOutputPath, + hadoopConf, + partitionColumns, + bucketSpec, + refreshFunction, + options, + isAppend) } else { logInfo("Skipping insertion into a relation that already exists.") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala new file mode 100644 index 0000000000000..54d0f3bd6291a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala @@ -0,0 +1,480 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 java.util.{Date, UUID} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.lib.output.{FileOutputCommitter, FileOutputFormat} +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl + +import org.apache.spark._ +import org.apache.spark.internal.Logging +import org.apache.spark.mapred.SparkHadoopMapRedUtil +import org.apache.spark.sql.{Dataset, SparkSession} +import org.apache.spark.sql.catalyst.catalog.BucketSpec +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.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.{SQLExecution, UnsafeKVExternalSorter} +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 helper object for writing data out to a location. */ +object WriteOutput extends Logging { + + /** A shared job description for all the write tasks. */ + private class WriteJobDescription( + val serializableHadoopConf: SerializableConfiguration, + val outputWriterFactory: OutputWriterFactory, + val allColumns: Seq[Attribute], + val partitionColumns: Seq[Attribute], + val nonPartitionColumns: Seq[Attribute], + val bucketSpec: Option[BucketSpec], + val isAppend: Boolean, + val path: String, + val outputFormatClass: Class[_ <: OutputFormat[_, _]]) + extends Serializable { + + assert(AttributeSet(allColumns) == AttributeSet(partitionColumns ++ nonPartitionColumns), + s""" + |All columns: ${allColumns.mkString(", ")} + |Partition columns: ${partitionColumns.mkString(", ")} + |Non-partition columns: ${nonPartitionColumns.mkString(", ")} + """.stripMargin) + } + + /** + * Basic work flow of this command is: + * 1. Driver side setup, including output committer initialization and data source specific + * preparation work for the write job to be issued. + * 2. Issues a write job consists of one or more executor side tasks, each of which writes all + * rows within an RDD partition. + * 3. If no exception is thrown in a task, commits that task, otherwise aborts that task; If any + * exception is thrown during task commitment, also aborts that task. + * 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. + */ + def write( + sparkSession: SparkSession, + plan: LogicalPlan, + fileFormat: FileFormat, + outputPath: Path, + hadoopConf: Configuration, + partitionColumns: Seq[Attribute], + bucketSpec: Option[BucketSpec], + refreshFunction: () => Unit, + options: Map[String, String], + isAppend: Boolean): Unit = { + + val job = Job.getInstance(hadoopConf) + job.setOutputKeyClass(classOf[Void]) + job.setOutputValueClass(classOf[InternalRow]) + FileOutputFormat.setOutputPath(job, outputPath) + + val partitionSet = AttributeSet(partitionColumns) + val dataColumns = plan.output.filterNot(partitionSet.contains) + val queryExecution = Dataset.ofRows(sparkSession, plan).queryExecution + + // Note: prepareWrite has side effect. It sets "job". + val outputWriterFactory = + fileFormat.prepareWrite(sparkSession, job, options, dataColumns.toStructType) + + val description = new WriteJobDescription( + serializableHadoopConf = new SerializableConfiguration(job.getConfiguration), + outputWriterFactory = outputWriterFactory, + allColumns = plan.output, + partitionColumns = partitionColumns, + nonPartitionColumns = dataColumns, + bucketSpec = bucketSpec, + isAppend = isAppend, + path = outputPath.toString, + outputFormatClass = job.getOutputFormatClass) + + SQLExecution.withNewExecutionId(sparkSession, queryExecution) { + // This call shouldn't be put into the `try` block below because it only initializes and + // prepares the job, any exception thrown from here shouldn't cause abortJob() to be called. + val committer = setupDriverCommitter(job, outputPath.toString, isAppend) + + try { + sparkSession.sparkContext.runJob(queryExecution.toRdd, + (taskContext: TaskContext, iter: Iterator[InternalRow]) => { + executeTask( + description = description, + sparkStageId = taskContext.stageId(), + sparkPartitionId = taskContext.partitionId(), + sparkAttemptNumber = taskContext.attemptNumber(), + iterator = iter) + }) + + committer.commitJob(job) + logInfo(s"Job ${job.getJobID} committed.") + refreshFunction() + } catch { case cause: Throwable => + logError(s"Aborting job ${job.getJobID}.", cause) + committer.abortJob(job, JobStatus.State.FAILED) + throw new SparkException("Job aborted.", cause) + } + } + } + + /** Writes data out in a single Spark task. */ + private def executeTask( + description: WriteJobDescription, + sparkStageId: Int, + sparkPartitionId: Int, + sparkAttemptNumber: Int, + iterator: Iterator[InternalRow]): Unit = { + + val jobId = SparkHadoopWriter.createJobID(new Date, sparkStageId) + val taskId = new TaskID(jobId, TaskType.MAP, sparkPartitionId) + val taskAttemptId = new TaskAttemptID(taskId, sparkAttemptNumber) + + // Set up the attempt context required to use in the output committer. + val taskAttemptContext: TaskAttemptContext = { + // Set up the configuration object + val hadoopConf = description.serializableHadoopConf.value + hadoopConf.set("mapred.job.id", jobId.toString) + hadoopConf.set("mapred.tip.id", taskAttemptId.getTaskID.toString) + hadoopConf.set("mapred.task.id", taskAttemptId.toString) + hadoopConf.setBoolean("mapred.task.is.map", true) + hadoopConf.setInt("mapred.task.partition", 0) + + new TaskAttemptContextImpl(hadoopConf, taskAttemptId) + } + + val committer = newOutputCommitter( + description.outputFormatClass, taskAttemptContext, description.path, description.isAppend) + committer.setupTask(taskAttemptContext) + + // Figure out where we need to write data to for staging. + // For FileOutputCommitter it has its own staging path called "work path". + val stagingPath = committer match { + case f: FileOutputCommitter => f.getWorkPath.toString + case _ => description.path + } + + val writeTask = + if (description.partitionColumns.isEmpty && description.bucketSpec.isEmpty) { + new SingleDirectoryWriteTask(description, taskAttemptContext, stagingPath) + } else { + new DynamicPartitionWriteTask(description, taskAttemptContext, stagingPath) + } + + try { + Utils.tryWithSafeFinallyAndFailureCallbacks(block = { + // Execute the task to write rows out + writeTask.execute(iterator) + writeTask.releaseResources() + + // Commit the task + SparkHadoopMapRedUtil.commitTask(committer, taskAttemptContext, jobId.getId, taskId.getId) + })(catchBlock = { + // If there is an error, release resource and then abort the task + try { + writeTask.releaseResources() + } finally { + committer.abortTask(taskAttemptContext) + logError(s"Job $jobId aborted.") + } + }) + } catch { + case t: Throwable => + throw new SparkException("Task failed while writing rows", t) + } + } + + /** + * A simple trait for writing out data in a single Spark task, without any concerns about how + * to commit or abort tasks. Exceptions thrown by the implementation of this trait will + * automatically trigger task aborts. + */ + private trait ExecuteWriteTask { + def execute(iterator: Iterator[InternalRow]): Unit + def releaseResources(): Unit + } + + /** Writes data to a single directory (used for non-dynamic-partition writes). */ + private class SingleDirectoryWriteTask( + description: WriteJobDescription, + taskAttemptContext: TaskAttemptContext, + stagingPath: String) extends ExecuteWriteTask { + + private[this] var outputWriter: OutputWriter = { + val outputWriter = description.outputWriterFactory.newInstance( + path = stagingPath, + bucketId = None, + dataSchema = description.nonPartitionColumns.toStructType, + context = taskAttemptContext) + outputWriter.initConverter(dataSchema = description.nonPartitionColumns.toStructType) + outputWriter + } + + override def execute(iter: Iterator[InternalRow]): Unit = { + while (iter.hasNext) { + val internalRow = iter.next() + outputWriter.writeInternal(internalRow) + } + } + + override def releaseResources(): Unit = { + if (outputWriter != null) { + outputWriter.close() + outputWriter = null + } + } + } + + /** + * Writes data to using dynamic partition writes, meaning this single function can write to + * multiple directories (partitions) or files (bucketing). + */ + private class DynamicPartitionWriteTask( + description: WriteJobDescription, + taskAttemptContext: TaskAttemptContext, + stagingPath: String) extends ExecuteWriteTask { + + // currentWriter is initialized whenever we see a new key + private var currentWriter: OutputWriter = _ + + private val bucketColumns: Seq[Attribute] = description.bucketSpec.toSeq.flatMap { + spec => spec.bucketColumnNames.map(c => description.allColumns.find(_.name == c).get) + } + + private val sortColumns: Seq[Attribute] = description.bucketSpec.toSeq.flatMap { + spec => spec.sortColumnNames.map(c => description.allColumns.find(_.name == c).get) + } + + private def bucketIdExpression: Option[Expression] = description.bucketSpec.map { spec => + // Use `HashPartitioning.partitionIdExpression` as our bucket id expression, so that we can + // guarantee the data distribution is same between shuffle and bucketed data source, which + // enables us to only shuffle one side when join a bucketed table and a normal one. + HashPartitioning(bucketColumns, spec.numBuckets).partitionIdExpression + } + + /** Expressions that given a partition key build a string like: col1=val/col2=val/... */ + private def partitionStringExpression: Seq[Expression] = { + description.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 + } + } + + private def getBucketIdFromKey(key: InternalRow): Option[Int] = + description.bucketSpec.map { _ => key.getInt(description.partitionColumns.length) } + + /** + * Open and returns a new OutputWriter given a partition key and optional bucket id. + * If bucket id is specified, we will append it to the end of the file name, but before the + * file extension, e.g. part-r-00009-ea518ad4-455a-4431-b471-d24e03814677-00002.gz.parquet + */ + private def newOutputWriter( + key: InternalRow, + getPartitionString: UnsafeProjection): OutputWriter = { + val path = + if (description.partitionColumns.nonEmpty) { + val partitionPath = getPartitionString(key).getString(0) + new Path(stagingPath, partitionPath).toString + } else { + stagingPath + } + val bucketId = getBucketIdFromKey(key) + + val newWriter = description.outputWriterFactory.newInstance( + path = path, + bucketId = bucketId, + dataSchema = description.nonPartitionColumns.toStructType, + context = taskAttemptContext) + newWriter.initConverter(description.nonPartitionColumns.toStructType) + newWriter + } + + override def execute(iter: Iterator[InternalRow]): Unit = { + // We should first sort by partition columns, then bucket id, and finally sorting columns. + val sortingExpressions: Seq[Expression] = + description.partitionColumns ++ bucketIdExpression ++ sortColumns + val getSortingKey = UnsafeProjection.create(sortingExpressions, description.allColumns) + + 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( + description.nonPartitionColumns, description.allColumns) + + // Returns the partition path given a partition key. + val getPartitionString = + UnsafeProjection.create(Seq(Concat(partitionStringExpression)), description.partitionColumns) + + // Sorts the data before write, so that we only need one writer at the same time. + val sorter = new UnsafeKVExternalSorter( + sortingKeySchema, + StructType.fromAttributes(description.nonPartitionColumns), + SparkEnv.get.blockManager, + SparkEnv.get.serializerManager, + TaskContext.get().taskMemoryManager().pageSizeBytes, + SparkEnv.get.conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", + UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD)) + + while (iter.hasNext) { + val currentRow = iter.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 sortedIterator = sorter.sortedIterator() + + // If anything below fails, we should abort the task. + 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 + } + } + + override def releaseResources(): Unit = { + if (currentWriter != null) { + currentWriter.close() + currentWriter = null + } + } + } + + private def setupDriverCommitter(job: Job, path: String, isAppend: Boolean): OutputCommitter = { + // Setup IDs + val jobId = SparkHadoopWriter.createJobID(new Date, 0) + val taskId = new TaskID(jobId, TaskType.MAP, 0) + val taskAttemptId = new TaskAttemptID(taskId, 0) + + // Set up the configuration object + job.getConfiguration.set("mapred.job.id", jobId.toString) + job.getConfiguration.set("mapred.tip.id", taskAttemptId.getTaskID.toString) + job.getConfiguration.set("mapred.task.id", taskAttemptId.toString) + job.getConfiguration.setBoolean("mapred.task.is.map", true) + job.getConfiguration.setInt("mapred.task.partition", 0) + + // 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. + // This UUID is used to avoid output file name collision between different appending write jobs. + // These jobs may belong to different SparkContext instances. Concrete data source + // implementations may use this UUID to generate unique file names (e.g., + // `part-r--.parquet`). The reason why this ID is used to identify a job + // rather than a single task output file is that, speculative tasks must generate the same + // output file name as the original task. + job.getConfiguration.set(WriterContainer.DATASOURCE_WRITEJOBUUID, UUID.randomUUID().toString) + + val taskAttemptContext = new TaskAttemptContextImpl(job.getConfiguration, taskAttemptId) + val outputCommitter = newOutputCommitter( + job.getOutputFormatClass, taskAttemptContext, path, isAppend) + outputCommitter.setupJob(job) + outputCommitter + } + + private def newOutputCommitter( + outputFormatClass: Class[_ <: OutputFormat[_, _]], + context: TaskAttemptContext, + path: String, + isAppend: Boolean): OutputCommitter = { + val defaultOutputCommitter = outputFormatClass.newInstance().getOutputCommitter(context) + + if (isAppend) { + // If we are appending data to an existing dir, we will only use the output committer + // associated with the file output format since it is not safe to use a custom + // committer for appending. For example, in S3, direct parquet output committer may + // leave partial data in the destination dir when the appending job fails. + // See SPARK-8578 for more details + logInfo( + s"Using default output committer ${defaultOutputCommitter.getClass.getCanonicalName} " + + "for appending.") + defaultOutputCommitter + } else { + val configuration = context.getConfiguration + val clazz = + configuration.getClass(SQLConf.OUTPUT_COMMITTER_CLASS.key, null, classOf[OutputCommitter]) + + if (clazz != null) { + logInfo(s"Using user defined output committer class ${clazz.getCanonicalName}") + + // Every output format based on org.apache.hadoop.mapreduce.lib.output.OutputFormat + // has an associated output committer. To override this output committer, + // we will first try to use the output committer set in SQLConf.OUTPUT_COMMITTER_CLASS. + // If a data source needs to override the output committer, it needs to set the + // output committer in prepareForWrite method. + if (classOf[FileOutputCommitter].isAssignableFrom(clazz)) { + // The specified output committer is a FileOutputCommitter. + // So, we will use the FileOutputCommitter-specified constructor. + val ctor = clazz.getDeclaredConstructor(classOf[Path], classOf[TaskAttemptContext]) + ctor.newInstance(new Path(path), context) + } else { + // The specified output committer is just an OutputCommitter. + // So, we will use the no-argument constructor. + val ctor = clazz.getDeclaredConstructor() + ctor.newInstance() + } + } else { + // If output committer class is not set, we will use the one associated with the + // file output format. + logInfo( + s"Using output committer class ${defaultOutputCommitter.getClass.getCanonicalName}") + defaultOutputCommitter + } + } + } +} + +object WriterContainer { + val DATASOURCE_WRITEJOBUUID = "spark.sql.sources.writeJobUUID" +} 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 deleted file mode 100644 index 253aa4405defa..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala +++ /dev/null @@ -1,445 +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 - -import java.util.{Date, UUID} - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapreduce._ -import org.apache.hadoop.mapreduce.lib.output.{FileOutputCommitter => MapReduceFileOutputCommitter} -import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl - -import org.apache.spark._ -import org.apache.spark.internal.Logging -import org.apache.spark.mapred.SparkHadoopMapRedUtil -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.catalog.BucketSpec -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.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. */ -private[datasources] case class WriteRelation( - sparkSession: SparkSession, - dataSchema: StructType, - path: String, - prepareJobForWrite: Job => OutputWriterFactory, - bucketSpec: Option[BucketSpec]) - -object WriterContainer { - val DATASOURCE_WRITEJOBUUID = "spark.sql.sources.writeJobUUID" -} - -private[datasources] abstract class BaseWriterContainer( - @transient val relation: WriteRelation, - @transient private val job: Job, - isAppend: Boolean) - extends Logging with Serializable { - - protected val dataSchema = relation.dataSchema - - protected val serializableConf = - new SerializableConfiguration(job.getConfiguration) - - // This UUID is used to avoid output file name collision between different appending write jobs. - // These jobs may belong to different SparkContext instances. Concrete data source implementations - // may use this UUID to generate unique file names (e.g., `part-r--.parquet`). - // The reason why this ID is used to identify a job rather than a single task output file is - // that, speculative tasks must generate the same output file name as the original task. - private val uniqueWriteJobId = UUID.randomUUID() - - // This is only used on driver side. - @transient private val jobContext: JobContext = job - - // The following fields are initialized and used on both driver and executor side. - @transient protected var outputCommitter: OutputCommitter = _ - @transient private var jobId: JobID = _ - @transient private var taskId: TaskID = _ - @transient private var taskAttemptId: TaskAttemptID = _ - @transient protected var taskAttemptContext: TaskAttemptContext = _ - - protected val outputPath: String = relation.path - - protected var outputWriterFactory: OutputWriterFactory = _ - - private var outputFormatClass: Class[_ <: OutputFormat[_, _]] = _ - - def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit - - def driverSideSetup(): Unit = { - setupIDs(0, 0, 0) - setupConf() - - // 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(WriterContainer.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, - // configurations made in prepareJobForWrite(job) are not populated into the TaskAttemptContext. - // - // Also, the `prepareJobForWrite` call must happen before initializing output format and output - // committer, since their initialization involve the job configuration, which can be potentially - // decorated in `prepareJobForWrite`. - outputWriterFactory = relation.prepareJobForWrite(job) - taskAttemptContext = new TaskAttemptContextImpl(serializableConf.value, taskAttemptId) - - outputFormatClass = job.getOutputFormatClass - outputCommitter = newOutputCommitter(taskAttemptContext) - outputCommitter.setupJob(jobContext) - } - - def executorSideSetup(taskContext: TaskContext): Unit = { - setupIDs(taskContext.stageId(), taskContext.partitionId(), taskContext.attemptNumber()) - setupConf() - taskAttemptContext = new TaskAttemptContextImpl(serializableConf.value, taskAttemptId) - outputCommitter = newOutputCommitter(taskAttemptContext) - outputCommitter.setupTask(taskAttemptContext) - } - - protected def getWorkPath: String = { - outputCommitter match { - // FileOutputCommitter writes to a temporary location returned by `getWorkPath`. - case f: MapReduceFileOutputCommitter => f.getWorkPath.toString - case _ => outputPath - } - } - - protected def newOutputWriter(path: String, bucketId: Option[Int] = None): OutputWriter = { - try { - outputWriterFactory.newInstance(path, bucketId, dataSchema, taskAttemptContext) - } catch { - case e: org.apache.hadoop.fs.FileAlreadyExistsException => - if (outputCommitter.getClass.getName.contains("Direct")) { - // SPARK-11382: DirectParquetOutputCommitter is not idempotent, meaning on retry - // attempts, the task will fail because the output file is created from a prior attempt. - // This often means the most visible error to the user is misleading. Augment the error - // to tell the user to look for the actual error. - throw new SparkException("The output file already exists but this could be due to a " + - "failure from an earlier attempt. Look through the earlier logs or stage page for " + - "the first error.\n File exists error: " + e, e) - } else { - throw e - } - } - } - - private def newOutputCommitter(context: TaskAttemptContext): OutputCommitter = { - val defaultOutputCommitter = outputFormatClass.newInstance().getOutputCommitter(context) - - if (isAppend) { - // If we are appending data to an existing dir, we will only use the output committer - // associated with the file output format since it is not safe to use a custom - // committer for appending. For example, in S3, direct parquet output committer may - // leave partial data in the destination dir when the appending job fails. - // - // See SPARK-8578 for more details - logInfo( - s"Using default output committer ${defaultOutputCommitter.getClass.getCanonicalName} " + - "for appending.") - defaultOutputCommitter - } else { - val configuration = context.getConfiguration - val committerClass = configuration.getClass( - SQLConf.OUTPUT_COMMITTER_CLASS.key, null, classOf[OutputCommitter]) - - Option(committerClass).map { clazz => - logInfo(s"Using user defined output committer class ${clazz.getCanonicalName}") - - // Every output format based on org.apache.hadoop.mapreduce.lib.output.OutputFormat - // has an associated output committer. To override this output committer, - // we will first try to use the output committer set in SQLConf.OUTPUT_COMMITTER_CLASS. - // If a data source needs to override the output committer, it needs to set the - // output committer in prepareForWrite method. - if (classOf[MapReduceFileOutputCommitter].isAssignableFrom(clazz)) { - // The specified output committer is a FileOutputCommitter. - // So, we will use the FileOutputCommitter-specified constructor. - val ctor = clazz.getDeclaredConstructor(classOf[Path], classOf[TaskAttemptContext]) - ctor.newInstance(new Path(outputPath), context) - } else { - // The specified output committer is just an OutputCommitter. - // So, we will use the no-argument constructor. - val ctor = clazz.getDeclaredConstructor() - ctor.newInstance() - } - }.getOrElse { - // If output committer class is not set, we will use the one associated with the - // file output format. - logInfo( - s"Using output committer class ${defaultOutputCommitter.getClass.getCanonicalName}") - defaultOutputCommitter - } - } - } - - private def setupIDs(jobId: Int, splitId: Int, attemptId: Int): Unit = { - this.jobId = SparkHadoopWriter.createJobID(new Date, jobId) - this.taskId = new TaskID(this.jobId, TaskType.MAP, splitId) - this.taskAttemptId = new TaskAttemptID(taskId, attemptId) - } - - private def setupConf(): Unit = { - serializableConf.value.set("mapred.job.id", jobId.toString) - serializableConf.value.set("mapred.tip.id", taskAttemptId.getTaskID.toString) - serializableConf.value.set("mapred.task.id", taskAttemptId.toString) - serializableConf.value.setBoolean("mapred.task.is.map", true) - serializableConf.value.setInt("mapred.task.partition", 0) - } - - def commitTask(): Unit = { - SparkHadoopMapRedUtil.commitTask(outputCommitter, taskAttemptContext, jobId.getId, taskId.getId) - } - - def abortTask(): Unit = { - if (outputCommitter != null) { - outputCommitter.abortTask(taskAttemptContext) - } - logError(s"Task attempt $taskAttemptId aborted.") - } - - def commitJob(): Unit = { - outputCommitter.commitJob(jobContext) - logInfo(s"Job $jobId committed.") - } - - def abortJob(): Unit = { - if (outputCommitter != null) { - outputCommitter.abortJob(jobContext, JobStatus.State.FAILED) - } - logError(s"Job $jobId aborted.") - } -} - -/** - * A writer that writes all of the rows in a partition to a single file. - */ -private[datasources] class DefaultWriterContainer( - relation: WriteRelation, - job: Job, - isAppend: Boolean) - extends BaseWriterContainer(relation, job, isAppend) { - - def writeRows(taskContext: TaskContext, iterator: Iterator[InternalRow]): Unit = { - executorSideSetup(taskContext) - 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) - } - - 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() - } - } - } -} - -/** - * A writer that dynamically opens files based on the given partition columns. Internally this is - * 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[datasources] class DynamicPartitionWriterContainer( - relation: WriteRelation, - job: Job, - partitionColumns: Seq[Attribute], - dataColumns: Seq[Attribute], - inputSchema: Seq[Attribute], - defaultPartitionName: String, - maxOpenFiles: Int, - isAppend: Boolean) - extends BaseWriterContainer(relation, job, isAppend) { - - private val bucketSpec = relation.bucketSpec - - private val bucketColumns: Seq[Attribute] = bucketSpec.toSeq.flatMap { - spec => spec.bucketColumnNames.map(c => inputSchema.find(_.name == c).get) - } - - private val sortColumns: Seq[Attribute] = bucketSpec.toSeq.flatMap { - spec => spec.sortColumnNames.map(c => inputSchema.find(_.name == c).get) - } - - private def bucketIdExpression: Option[Expression] = bucketSpec.map { spec => - // Use `HashPartitioning.partitionIdExpression` as our bucket id expression, so that we can - // guarantee the data distribution is same between shuffle and bucketed data source, which - // enables us to only shuffle one side when join a bucketed table and a normal one. - HashPartitioning(bucketColumns, spec.numBuckets).partitionIdExpression - } - - // 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(defaultPartitionName), escaped) - val partitionName = Literal(c.name + "=") :: str :: Nil - if (i == 0) partitionName else Literal(Path.SEPARATOR) :: partitionName - } - } - - private def getBucketIdFromKey(key: InternalRow): Option[Int] = bucketSpec.map { _ => - key.getInt(partitionColumns.length) - } - - /** - * Open and returns a new OutputWriter given a partition key and optional bucket id. - * If bucket id is specified, we will append it to the end of the file name, but before the - * file extension, e.g. part-r-00009-ea518ad4-455a-4431-b471-d24e03814677-00002.gz.parquet - */ - private def newOutputWriter( - key: InternalRow, - getPartitionString: UnsafeProjection): OutputWriter = { - val path = if (partitionColumns.nonEmpty) { - val partitionPath = getPartitionString(key).getString(0) - new Path(getWorkPath, partitionPath).toString - } else { - getWorkPath - } - val bucketId = getBucketIdFromKey(key) - val newWriter = super.newOutputWriter(path, bucketId) - newWriter.initConverter(dataSchema) - newWriter - } - - 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, - SparkEnv.get.conf.getLong("spark.shuffle.spill.numElementsForceSpillThreshold", - UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD)) - - 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 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) - } - 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/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 8afd39d657865..9061b1b9a2638 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 @@ -339,13 +339,6 @@ object SQLConf { .booleanConf .createWithDefault(true) - val PARTITION_MAX_FILES = - SQLConfigBuilder("spark.sql.sources.maxConcurrentWrites") - .doc("The maximum number of concurrent files to open before falling back on sorting when " + - "writing out files using dynamic partitioning.") - .intConf - .createWithDefault(1) - val BUCKETING_ENABLED = SQLConfigBuilder("spark.sql.sources.bucketing.enabled") .doc("When false, we will treat bucketed table as normal table") .booleanConf @@ -733,8 +726,6 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def partitionColumnTypeInferenceEnabled: Boolean = getConf(SQLConf.PARTITION_COLUMN_TYPE_INFERENCE) - def partitionMaxFiles: Int = getConf(PARTITION_MAX_FILES) - def parallelPartitionDiscoveryThreshold: Int = getConf(SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD) From 39755169fb5bb07332eef263b4c18ede1528812d Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Wed, 19 Oct 2016 23:41:38 -0700 Subject: [PATCH 03/48] [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. --- .../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 6dc334ceb52ea..be119578d2c35 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 b5738b9a95c36..b0e5ba0865c63 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 7fba901b85695..bfc609419ccdb 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1759,6 +1759,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 b427f7fb50158..4dda80f10a08a 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 4bd17c4606764242bc29888b8eedc8e4b5a00f46 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Wed, 19 Oct 2016 23:55:05 -0700 Subject: [PATCH 04/48] [SPARK-17991][SQL] Enable metastore partition pruning by default. ## What changes were proposed in this pull request? This should apply to non-converted metastore relations. WIP to see if this causes any test failures. ## How was this patch tested? Existing tests. Author: Eric Liang Closes #15475 from ericl/try-enabling-pruning. --- .../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 9061b1b9a2638..ebf4fad5cbcff 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 @@ -267,7 +267,7 @@ object SQLConf { .doc("When true, some predicates will be pushed down into the Hive metastore so that " + "unmatching partitions can be eliminated earlier.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val HIVE_FILESOURCE_PARTITION_PRUNING = SQLConfigBuilder("spark.sql.hive.filesourcePartitionPruning") From c2c107abad8b462218d33c70b946e840663228a1 Mon Sep 17 00:00:00 2001 From: Mike Ihbe Date: Thu, 20 Oct 2016 09:49:58 +0100 Subject: [PATCH 05/48] [SPARK-11653][DEPLOY] Allow spark-daemon.sh to run in the foreground ## What changes were proposed in this pull request? Add a SPARK_NO_DAEMONIZE environment variable flag to spark-daemon.sh that causes the process it would run to be run in the foreground. It looks like there has been some prior work in https://github.com/apache/spark/pull/3881, but there was some talk about these being refactored. I'm not sure if that happened or not, but that PR is almost 2 years old at this point so it was worth revisiting. ## How was this patch tested? ./dev/run-tests still seems to work. It doesn't look like these scripts have tests, but if I missed them just let me know. Author: Mike Ihbe Closes #15338 from mikejihbe/SPARK-11653. --- conf/spark-env.sh.template | 1 + sbin/spark-daemon.sh | 54 ++++++++++++++++++++++---------------- 2 files changed, 33 insertions(+), 22 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index c750c72d19880..5c1e876ef9afc 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -63,3 +63,4 @@ # - SPARK_PID_DIR Where the pid file is stored. (Default: /tmp) # - SPARK_IDENT_STRING A string representing this instance of spark. (Default: $USER) # - SPARK_NICENESS The scheduling priority for daemons. (Default: 0) +# - SPARK_NO_DAEMONIZE Run the proposed command in the foreground. It will not output a PID file. diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 59823571124f1..061019a55e997 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -27,6 +27,7 @@ # SPARK_PID_DIR The pid files are stored. /tmp by default. # SPARK_IDENT_STRING A string representing this instance of spark. $USER by default # SPARK_NICENESS The scheduling priority for daemons. Defaults to 0. +# SPARK_NO_DAEMONIZE If set, will run the proposed command in the foreground. It will not output a PID file. ## usage="Usage: spark-daemon.sh [--config ] (start|stop|submit|status) " @@ -122,6 +123,35 @@ if [ "$SPARK_NICENESS" = "" ]; then export SPARK_NICENESS=0 fi +execute_command() { + local command="$@" + if [ -z ${SPARK_NO_DAEMONIZE+set} ]; then + nohup -- $command >> $log 2>&1 < /dev/null & + newpid="$!" + + 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 + echo "failed to launch $command:" + tail -2 "$log" | sed 's/^/ /' + echo "full log in $log" + fi + else + $command + fi +} + run_command() { mode="$1" shift @@ -146,13 +176,11 @@ run_command() { case "$mode" in (class) - nohup nice -n "$SPARK_NICENESS" "${SPARK_HOME}"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & - newpid="$!" + execute_command nice -n "$SPARK_NICENESS" "${SPARK_HOME}"/bin/spark-class $command $@ ;; (submit) - nohup nice -n "$SPARK_NICENESS" "${SPARK_HOME}"/bin/spark-submit --class $command "$@" >> "$log" 2>&1 < /dev/null & - newpid="$!" + execute_command nice -n "$SPARK_NICENESS" bash "${SPARK_HOME}"/bin/spark-submit --class $command $@ ;; (*) @@ -161,24 +189,6 @@ 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 - echo "failed to launch $command:" - tail -2 "$log" | sed 's/^/ /' - echo "full log in $log" - fi } case $option in From 986a3b8b5bedb1d64e2cf7c95bfdf5505f3e8c69 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 20 Oct 2016 09:53:12 +0100 Subject: [PATCH 06/48] [SPARK-17796][SQL] Support wildcard character in filename for LOAD DATA LOCAL INPATH ## What changes were proposed in this pull request? Currently, Spark 2.0 raises an `input path does not exist` AnalysisException if the file name contains '*'. It is misleading since it occurs when there exist some matched files. Also, it was a supported feature in Spark 1.6.2. This PR aims to support wildcard characters in filename for `LOAD DATA LOCAL INPATH` SQL command like Spark 1.6.2. **Reported Error Scenario** ```scala scala> sql("CREATE TABLE t(a string)") res0: org.apache.spark.sql.DataFrame = [] scala> sql("LOAD DATA LOCAL INPATH '/tmp/x*' INTO TABLE t") org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: /tmp/x*; ``` ## How was this patch tested? Pass the Jenkins test with a new test case. Author: Dongjoon Hyun Closes #15376 from dongjoon-hyun/SPARK-17796. --- .../spark/sql/execution/command/tables.scala | 23 +++++++++++++- .../sql/hive/execution/SQLQuerySuite.scala | 30 +++++++++++++++++++ 2 files changed, 52 insertions(+), 1 deletion(-) 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 403b479a0e1bc..4c0675adb4973 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,6 +19,7 @@ package org.apache.spark.sql.execution.command import java.io.File import java.net.URI +import java.nio.file.FileSystems import java.util.Date import scala.collection.mutable.ArrayBuffer @@ -245,7 +246,27 @@ case class LoadDataCommand( val loadPath = if (isLocal) { val uri = Utils.resolveURI(path) - if (!new File(uri.getPath()).exists()) { + val filePath = uri.getPath() + val exists = if (filePath.contains("*")) { + val fileSystem = FileSystems.getDefault + val pathPattern = fileSystem.getPath(filePath) + val dir = pathPattern.getParent.toString + if (dir.contains("*")) { + throw new AnalysisException( + s"LOAD DATA input path allows only filename wildcard: $path") + } + + val files = new File(dir).listFiles() + if (files == null) { + false + } else { + val matcher = fileSystem.getPathMatcher("glob:" + pathPattern.toAbsolutePath) + files.exists(f => matcher.matches(fileSystem.getPath(f.getAbsolutePath))) + } + } else { + new File(filePath).exists() + } + if (!exists) { throw new AnalysisException(s"LOAD DATA input path does not exist: $path") } uri 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 e26b6b57ef569..495b4f874a1d6 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 @@ -17,11 +17,14 @@ package org.apache.spark.sql.hive.execution +import java.io.{File, PrintWriter} +import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} import scala.sys.process.{Process, ProcessLogger} import scala.util.Try +import com.google.common.io.Files import org.apache.hadoop.fs.Path import org.apache.spark.sql._ @@ -1917,6 +1920,33 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("SPARK-17796 Support wildcard character in filename for LOAD DATA LOCAL INPATH") { + withTempDir { dir => + for (i <- 1 to 3) { + Files.write(s"$i", new File(s"$dir/part-r-0000$i"), StandardCharsets.UTF_8) + } + for (i <- 5 to 7) { + Files.write(s"$i", new File(s"$dir/part-s-0000$i"), StandardCharsets.UTF_8) + } + + withTable("load_t") { + sql("CREATE TABLE load_t (a STRING)") + sql(s"LOAD DATA LOCAL INPATH '$dir/*part-r*' INTO TABLE load_t") + checkAnswer(sql("SELECT * FROM load_t"), Seq(Row("1"), Row("2"), Row("3"))) + + val m = intercept[AnalysisException] { + sql("LOAD DATA LOCAL INPATH '/non-exist-folder/*part*' INTO TABLE load_t") + }.getMessage + assert(m.contains("LOAD DATA input path does not exist")) + + val m2 = intercept[AnalysisException] { + sql(s"LOAD DATA LOCAL INPATH '$dir*/*part*' INTO TABLE load_t") + }.getMessage + assert(m2.contains("LOAD DATA input path allows only filename wildcard")) + } + } + } + def testCommandAvailable(command: String): Boolean = { val attempt = Try(Process(command).run(ProcessLogger(_ => ())).exitValue()) attempt.isSuccess && attempt.get == 0 From e895bc25481f73b433a3cc3ad46df066ec602862 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Thu, 20 Oct 2016 19:39:25 +0800 Subject: [PATCH 07/48] [SPARK-17860][SQL] SHOW COLUMN's database conflict check should respect case sensitivity configuration ## What changes were proposed in this pull request? SHOW COLUMNS command validates the user supplied database name with database name from qualified table name name to make sure both of them are consistent. This comparison should respect case sensitivity. ## How was this patch tested? Added tests in DDLSuite and existing tests were moved to use new sql based test infrastructure. Author: Dilip Biswal Closes #15423 from dilipbiswal/dkb_show_column_fix. --- .../spark/sql/execution/SparkSqlParser.scala | 12 +- .../spark/sql/execution/command/tables.scala | 14 +- .../sql-tests/inputs/show_columns.sql | 58 +++++ .../sql-tests/results/show_columns.sql.out | 217 ++++++++++++++++++ .../apache/spark/sql/SQLQueryTestSuite.scala | 1 + .../execution/command/DDLCommandSuite.scala | 18 +- .../sql/execution/command/DDLSuite.scala | 17 ++ .../sql/hive/execution/HiveCommandSuite.scala | 23 +- .../hive/execution/HiveComparisonTest.scala | 2 +- 9 files changed, 318 insertions(+), 44 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/show_columns.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/show_columns.sql.out 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 ea22b02d40b80..1cc166d5a7a9d 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,17 +168,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { * }}} */ override def visitShowColumns(ctx: ShowColumnsContext): LogicalPlan = withOrigin(ctx) { - val table = visitTableIdentifier(ctx.tableIdentifier) - - val lookupTable = Option(ctx.db) match { - case None => table - case Some(db) if table.database.exists(_ != db) => - operationNotAllowed( - s"SHOW COLUMNS with conflicting databases: '$db' != '${table.database.get}'", - ctx) - case Some(db) => TableIdentifier(table.identifier, Some(db.getText)) - } - ShowColumnsCommand(lookupTable) + ShowColumnsCommand(Option(ctx.db).map(_.getText), visitTableIdentifier(ctx.tableIdentifier)) } /** 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 4c0675adb4973..aec25430b719d 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 @@ -671,14 +671,24 @@ case class ShowTablePropertiesCommand(table: TableIdentifier, propertyKey: Optio * SHOW COLUMNS (FROM | IN) table_identifier [(FROM | IN) database]; * }}} */ -case class ShowColumnsCommand(tableName: TableIdentifier) extends RunnableCommand { +case class ShowColumnsCommand( + databaseName: Option[String], + tableName: TableIdentifier) extends RunnableCommand { override val output: Seq[Attribute] = { AttributeReference("col_name", StringType, nullable = false)() :: Nil } override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - val table = catalog.getTempViewOrPermanentTableMetadata(tableName) + val resolver = sparkSession.sessionState.conf.resolver + val lookupTable = databaseName match { + case None => tableName + case Some(db) if tableName.database.exists(!resolver(_, db)) => + throw new AnalysisException( + s"SHOW COLUMNS with conflicting databases: '$db' != '${tableName.database.get}'") + case Some(db) => TableIdentifier(tableName.identifier, Some(db)) + } + val table = catalog.getTempViewOrPermanentTableMetadata(lookupTable) table.schema.map { c => Row(c.name) } diff --git a/sql/core/src/test/resources/sql-tests/inputs/show_columns.sql b/sql/core/src/test/resources/sql-tests/inputs/show_columns.sql new file mode 100644 index 0000000000000..3894082255088 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/show_columns.sql @@ -0,0 +1,58 @@ +CREATE DATABASE showdb; + +USE showdb; + +CREATE TABLE showcolumn1 (col1 int, `col 2` int); +CREATE TABLE showcolumn2 (price int, qty int) partitioned by (year int, month int); +CREATE TEMPORARY VIEW showColumn3 (col3 int, `col 4` int) USING parquet; +CREATE GLOBAL TEMP VIEW showColumn4 AS SELECT 1 as col1, 'abc' as `col 5`; + + +-- only table name +SHOW COLUMNS IN showcolumn1; + +-- qualified table name +SHOW COLUMNS IN showdb.showcolumn1; + +-- table name and database name +SHOW COLUMNS IN showcolumn1 FROM showdb; + +-- partitioned table +SHOW COLUMNS IN showcolumn2 IN showdb; + +-- Non-existent table. Raise an error in this case +SHOW COLUMNS IN badtable FROM showdb; + +-- database in table identifier and database name in different case +SHOW COLUMNS IN showdb.showcolumn1 from SHOWDB; + +-- different database name in table identifier and database name. +-- Raise an error in this case. +SHOW COLUMNS IN showdb.showcolumn1 FROM baddb; + +-- show column on temporary view +SHOW COLUMNS IN showcolumn3; + +-- error temp view can't be qualified with a database +SHOW COLUMNS IN showdb.showcolumn3; + +-- error temp view can't be qualified with a database +SHOW COLUMNS IN showcolumn3 FROM showdb; + +-- error global temp view needs to be qualified +SHOW COLUMNS IN showcolumn4; + +-- global temp view qualified with database +SHOW COLUMNS IN global_temp.showcolumn4; + +-- global temp view qualified with database +SHOW COLUMNS IN showcolumn4 FROM global_temp; + +DROP TABLE showcolumn1; +DROP TABLE showColumn2; +DROP VIEW showcolumn3; +DROP VIEW global_temp.showcolumn4; + +use default; + +DROP DATABASE showdb; diff --git a/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out new file mode 100644 index 0000000000000..832e6e25bb2bd --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/show_columns.sql.out @@ -0,0 +1,217 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 25 + + +-- !query 0 +CREATE DATABASE showdb +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +USE showdb +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TABLE showcolumn1 (col1 int, `col 2` int) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +CREATE TABLE showcolumn2 (price int, qty int) partitioned by (year int, month int) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +CREATE TEMPORARY VIEW showColumn3 (col3 int, `col 4` int) USING parquet +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +CREATE GLOBAL TEMP VIEW showColumn4 AS SELECT 1 as col1, 'abc' as `col 5` +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +SHOW COLUMNS IN showcolumn1 +-- !query 6 schema +struct +-- !query 6 output +col 2 +col1 + + +-- !query 7 +SHOW COLUMNS IN showdb.showcolumn1 +-- !query 7 schema +struct +-- !query 7 output +col 2 +col1 + + +-- !query 8 +SHOW COLUMNS IN showcolumn1 FROM showdb +-- !query 8 schema +struct +-- !query 8 output +col 2 +col1 + + +-- !query 9 +SHOW COLUMNS IN showcolumn2 IN showdb +-- !query 9 schema +struct +-- !query 9 output +month +price +qty +year + + +-- !query 10 +SHOW COLUMNS IN badtable FROM showdb +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +Table or view 'badtable' not found in database 'showdb'; + + +-- !query 11 +SHOW COLUMNS IN showdb.showcolumn1 from SHOWDB +-- !query 11 schema +struct +-- !query 11 output +col 2 +col1 + + +-- !query 12 +SHOW COLUMNS IN showdb.showcolumn1 FROM baddb +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.AnalysisException +SHOW COLUMNS with conflicting databases: 'baddb' != 'showdb'; + + +-- !query 13 +SHOW COLUMNS IN showcolumn3 +-- !query 13 schema +struct +-- !query 13 output +col 4 +col3 + + +-- !query 14 +SHOW COLUMNS IN showdb.showcolumn3 +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +Table or view 'showcolumn3' not found in database 'showdb'; + + +-- !query 15 +SHOW COLUMNS IN showcolumn3 FROM showdb +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +Table or view 'showcolumn3' not found in database 'showdb'; + + +-- !query 16 +SHOW COLUMNS IN showcolumn4 +-- !query 16 schema +struct<> +-- !query 16 output +org.apache.spark.sql.catalyst.analysis.NoSuchTableException +Table or view 'showcolumn4' not found in database 'showdb'; + + +-- !query 17 +SHOW COLUMNS IN global_temp.showcolumn4 +-- !query 17 schema +struct +-- !query 17 output +col 5 +col1 + + +-- !query 18 +SHOW COLUMNS IN showcolumn4 FROM global_temp +-- !query 18 schema +struct +-- !query 18 output +col 5 +col1 + + +-- !query 19 +DROP TABLE showcolumn1 +-- !query 19 schema +struct<> +-- !query 19 output + + + +-- !query 20 +DROP TABLE showColumn2 +-- !query 20 schema +struct<> +-- !query 20 output + + + +-- !query 21 +DROP VIEW showcolumn3 +-- !query 21 schema +struct<> +-- !query 21 output + + + +-- !query 22 +DROP VIEW global_temp.showcolumn4 +-- !query 22 schema +struct<> +-- !query 22 output + + + +-- !query 23 +use default +-- !query 23 schema +struct<> +-- !query 23 output + + + +-- !query 24 +DROP DATABASE showdb +-- !query 24 schema +struct<> +-- !query 24 output + 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 02841d7bb03ff..6857dd37286dd 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 @@ -26,6 +26,7 @@ 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.execution.command.ShowColumnsCommand import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.StructType 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 a3dbc9234f2f3..d31e7aeb3a78a 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 @@ -824,22 +824,24 @@ 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 db1" - val sql5 = "SHOW COLUMNS FROM db1.t1 IN db2" + val sql4 = "SHOW COLUMNS FROM db1.t1 IN db2" val parsed1 = parser.parsePlan(sql1) - val expected1 = ShowColumnsCommand(TableIdentifier("t1", None)) + val expected1 = ShowColumnsCommand(None, TableIdentifier("t1", None)) val parsed2 = parser.parsePlan(sql2) - val expected2 = ShowColumnsCommand(TableIdentifier("t1", Some("db1"))) + val expected2 = ShowColumnsCommand(None, TableIdentifier("t1", Some("db1"))) val parsed3 = parser.parsePlan(sql3) - val parsed4 = parser.parsePlan(sql3) + val expected3 = ShowColumnsCommand(Some("db1"), TableIdentifier("t1", None)) + val parsed4 = parser.parsePlan(sql4) + val expected4 = ShowColumnsCommand(Some("db2"), TableIdentifier("t1", Some("db1"))) + comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) - comparePlans(parsed3, expected2) - comparePlans(parsed4, expected2) - assertUnsupported(sql5) + comparePlans(parsed3, expected3) + comparePlans(parsed4, expected4) } + test("show partitions") { val sql1 = "SHOW PARTITIONS t1" val sql2 = "SHOW PARTITIONS db1.t1" 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 c8b8e9ebabc75..a6da8a86c1623 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 @@ -1749,4 +1749,21 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { assert(sql("show user functions").count() === 1L) } } + + test("show columns - negative test") { + // When case sensitivity is true, the user supplied database name in table identifier + // should match the supplied database name in case sensitive way. + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + withTempDatabase { db => + val tabName = s"$db.showcolumn" + withTable(tabName) { + sql(s"CREATE TABLE $tabName(col1 int, col2 string) USING parquet ") + val message = intercept[AnalysisException] { + sql(s"SHOW COLUMNS IN $db.showcolumn FROM ${db.toUpperCase}") + }.getMessage + assert(message.contains("SHOW COLUMNS with conflicting databases")) + } + } + } + } } 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 2c772ce2155ef..ad1e9b17a9f71 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 @@ -22,6 +22,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types.StructType @@ -336,28 +337,6 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } } - test("show columns") { - checkAnswer( - sql("SHOW COLUMNS IN parquet_tab3"), - Row("col1") :: Row("col 2") :: Nil) - - checkAnswer( - sql("SHOW COLUMNS IN default.parquet_tab3"), - Row("col1") :: Row("col 2") :: Nil) - - checkAnswer( - sql("SHOW COLUMNS IN parquet_tab3 FROM default"), - Row("col1") :: Row("col 2") :: Nil) - - checkAnswer( - sql("SHOW COLUMNS IN parquet_tab4 IN default"), - Row("price") :: Row("qty") :: Row("year") :: Row("month") :: Nil) - - val message = intercept[NoSuchTableException] { - sql("SHOW COLUMNS IN badtable FROM default") - }.getMessage - assert(message.contains("'badtable' not found in database")) - } test("show partitions - show everything") { checkAnswer( 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 80e75aa898c38..13ceed7c79e35 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 @@ -167,7 +167,7 @@ abstract class HiveComparisonTest // and does not return it as a query answer. case _: SetCommand => Seq("0") case _: ExplainCommand => answer - case _: DescribeTableCommand | ShowColumnsCommand(_) => + case _: DescribeTableCommand | ShowColumnsCommand(_, _) => // Filter out non-deterministic lines and lines which do not have actual results but // can introduce problems because of the way Hive formats these lines. // Then, remove empty lines. Do not sort the results. From fb0894b3a87331a731129ad3fc7ebe598d90a6ee Mon Sep 17 00:00:00 2001 From: Tejas Patil Date: Thu, 20 Oct 2016 09:50:55 -0700 Subject: [PATCH 08/48] [SPARK-17698][SQL] Join predicates should not contain filter clauses ## What changes were proposed in this pull request? 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 #15272 from tejasapatil/SPARK-17698_join_predicate_filter_clause. --- .../sql/catalyst/expressions/predicates.scala | 5 +- .../spark/sql/catalyst/optimizer/joins.scala | 4 +- .../sql/catalyst/planning/patterns.scala | 2 + .../spark/sql/sources/BucketedReadSuite.scala | 124 ++++++++++++++---- 4 files changed, 109 insertions(+), 26 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 799858a6865e5..9394e39aadd9d 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 2626057e492ef..180ad2e0ad1fa 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 @@ -65,7 +65,9 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { val conditionalJoin = rest.find { planJoinPair => val plan = planJoinPair._1 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 bdae56881bf46..c5f92c59c88f4 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 3ff85176de10e..9ed454e578d69 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 @@ -235,7 +235,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, sortLeft: Boolean = true, @@ -268,12 +269,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] @@ -297,56 +298,102 @@ 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 + ) } } test("avoid shuffle and sort when bucket and sort columns are join keys") { val bucketSpec = Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))) testBucketing( - bucketSpec, bucketSpec, Seq("i", "j"), - shuffleLeft = false, shuffleRight = false, - sortLeft = false, sortRight = false + bucketSpecLeft = bucketSpec, + bucketSpecRight = bucketSpec, + joinCondition = joinCondition(Seq("i", "j")), + shuffleLeft = false, + shuffleRight = false, + sortLeft = false, + sortRight = false ) } @@ -354,9 +401,13 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet val bucketSpec1 = Some(BucketSpec(8, Seq("i"), Seq("i", "j"))) val bucketSpec2 = Some(BucketSpec(8, Seq("i"), Seq("i", "k"))) testBucketing( - bucketSpec1, bucketSpec2, Seq("i"), - shuffleLeft = false, shuffleRight = false, - sortLeft = false, sortRight = false + bucketSpecLeft = bucketSpec1, + bucketSpecRight = bucketSpec2, + joinCondition = joinCondition(Seq("i")), + shuffleLeft = false, + shuffleRight = false, + sortLeft = false, + sortRight = false ) } @@ -364,9 +415,13 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet val bucketSpec1 = Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))) val bucketSpec2 = Some(BucketSpec(8, Seq("i", "j"), Seq("k"))) testBucketing( - bucketSpec1, bucketSpec2, Seq("i", "j"), - shuffleLeft = false, shuffleRight = false, - sortLeft = false, sortRight = true + bucketSpecLeft = bucketSpec1, + bucketSpecRight = bucketSpec2, + joinCondition = joinCondition(Seq("i", "j")), + shuffleLeft = false, + shuffleRight = false, + sortLeft = false, + sortRight = true ) } @@ -374,9 +429,13 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet val bucketSpec1 = Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))) val bucketSpec2 = Some(BucketSpec(8, Seq("i", "j"), Seq("j", "i"))) testBucketing( - bucketSpec1, bucketSpec2, Seq("i", "j"), - shuffleLeft = false, shuffleRight = false, - sortLeft = false, sortRight = true + bucketSpecLeft = bucketSpec1, + bucketSpecRight = bucketSpec2, + joinCondition = joinCondition(Seq("i", "j")), + shuffleLeft = false, + shuffleRight = false, + sortLeft = false, + sortRight = true ) } @@ -408,6 +467,25 @@ 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 = Seq("i").map(col => left(col) === right(col)).reduce(_ && _) + val filterLeft = left("i") === Literal("1") + val filterRight = right("i") === Literal("1") + joinPredicates && filterLeft && filterRight + }, + shuffleLeft = false, + shuffleRight = false, + sortLeft = false, + sortRight = false + ) + } + test("error if there exists any malformed bucket files") { withTable("bucketed_table") { df1.write.format("parquet").bucketBy(8, "i").saveAsTable("bucketed_table") From 84b245f2dd31c1cebbf12458bf11f67e287e93f4 Mon Sep 17 00:00:00 2001 From: Koert Kuipers Date: Thu, 20 Oct 2016 10:08:12 -0700 Subject: [PATCH 09/48] [SPARK-15780][SQL] Support mapValues on KeyValueGroupedDataset ## What changes were proposed in this pull request? Add mapValues to KeyValueGroupedDataset ## How was this patch tested? New test in DatasetSuite for groupBy function, mapValues, flatMap Author: Koert Kuipers Closes #13526 from koertkuipers/feat-keyvaluegroupeddataset-mapvalues. --- .../sql/catalyst/plans/logical/object.scala | 13 ++++++ .../spark/sql/KeyValueGroupedDataset.scala | 42 +++++++++++++++++++ .../org/apache/spark/sql/DatasetSuite.scala | 11 +++++ 3 files changed, 66 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 fefe5a3953a6e..0ab4c9016623e 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 @@ -230,6 +230,19 @@ object AppendColumns { encoderFor[U].namedExpressions, child) } + + def apply[T : Encoder, U : Encoder]( + func: T => U, + inputAttributes: Seq[Attribute], + child: LogicalPlan): AppendColumns = { + new AppendColumns( + func.asInstanceOf[Any => Any], + implicitly[Encoder[T]].clsTag.runtimeClass, + implicitly[Encoder[T]].schema, + UnresolvedDeserializer(encoderFor[T].deserializer, inputAttributes), + encoderFor[U].namedExpressions, + child) + } } /** 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 828eb94efe598..4cb0313aa9037 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 @@ -66,6 +66,48 @@ class KeyValueGroupedDataset[K, V] private[sql]( dataAttributes, groupingAttributes) + /** + * Returns a new [[KeyValueGroupedDataset]] where the given function `func` has been applied + * to the data. The grouping key is unchanged by this. + * + * {{{ + * // Create values grouped by key from a Dataset[(K, V)] + * ds.groupByKey(_._1).mapValues(_._2) // Scala + * }}} + * + * @since 2.1.0 + */ + def mapValues[W : Encoder](func: V => W): KeyValueGroupedDataset[K, W] = { + val withNewData = AppendColumns(func, dataAttributes, logicalPlan) + val projected = Project(withNewData.newColumns ++ groupingAttributes, withNewData) + val executed = sparkSession.sessionState.executePlan(projected) + + new KeyValueGroupedDataset( + encoderFor[K], + encoderFor[W], + executed, + withNewData.newColumns, + groupingAttributes) + } + + /** + * Returns a new [[KeyValueGroupedDataset]] where the given function `func` has been applied + * to the data. The grouping key is unchanged by this. + * + * {{{ + * // Create Integer values grouped by String key from a Dataset> + * Dataset> ds = ...; + * KeyValueGroupedDataset grouped = + * ds.groupByKey(t -> t._1, Encoders.STRING()).mapValues(t -> t._2, Encoders.INT()); // Java 8 + * }}} + * + * @since 2.1.0 + */ + def mapValues[W](func: MapFunction[V, W], encoder: Encoder[W]): KeyValueGroupedDataset[K, W] = { + implicit val uEnc = encoder + mapValues { (v: V) => func.call(v) } + } + /** * 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. 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 5fce9b4fe97ea..cc367acae2ba4 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 @@ -336,6 +336,17 @@ class DatasetSuite extends QueryTest with SharedSQLContext { "a", "30", "b", "3", "c", "1") } + test("groupBy function, mapValues, flatMap") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + val keyValue = ds.groupByKey(_._1).mapValues(_._2) + val agged = keyValue.mapGroups { case (g, iter) => (g, iter.sum) } + checkDataset(agged, ("a", 30), ("b", 3), ("c", 1)) + + val keyValue1 = ds.groupByKey(t => (t._1, "key")).mapValues(t => (t._2, "value")) + val agged1 = keyValue1.mapGroups { case (g, iter) => (g._1, iter.map(_._1).sum) } + checkDataset(agged, ("a", 30), ("b", 3), ("c", 1)) + } + test("groupBy function, reduce") { val ds = Seq("abc", "xyz", "hello").toDS() val agged = ds.groupByKey(_.length).reduceGroups(_ + _) From 947f4f25273161dc4719419a35613a71c2e2a150 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 20 Oct 2016 10:50:34 -0700 Subject: [PATCH 10/48] [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. --- .../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 7f9ec19eae60abe589ffd22259a9065e7e353a57 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 20 Oct 2016 12:18:56 -0700 Subject: [PATCH 11/48] [SPARK-18021][SQL] Refactor file name specification for data sources ## What changes were proposed in this pull request? Currently each data source OutputWriter is responsible for specifying the entire file name for each file output. This, however, does not make any sense because we rely on file naming schemes for certain behaviors in Spark SQL, e.g. bucket id. The current approach allows individual data sources to break the implementation of bucketing. On the flip side, we also don't want to move file naming entirely out of data sources, because different data sources do want to specify different extensions. This patch divides file name specification into two parts: the first part is a prefix specified by the caller of OutputWriter (in WriteOutput), and the second part is the suffix that can be specified by the OutputWriter itself. Note that a side effect of this change is that now all file based data sources also support bucketing automatically. There are also some other minor cleanups: - Removed the UUID passed through generic Configuration string - Some minor rewrites for better clarity - Renamed "path" in multiple places to "stagingDir", to more accurately reflect its meaning ## How was this patch tested? This should be covered by existing data source tests. Author: Reynold Xin Closes #15562 from rxin/SPARK-18021. --- .../ml/source/libsvm/LibSVMRelation.scala | 16 ++---- .../execution/datasources/OutputWriter.scala | 17 ++++-- .../execution/datasources/WriteOutput.scala | 56 +++++++++---------- .../datasources/csv/CSVRelation.scala | 18 +++--- .../datasources/json/JsonFileFormat.scala | 17 ++---- .../parquet/ParquetFileFormat.scala | 7 +-- .../parquet/ParquetOutputWriter.scala | 32 +++-------- .../datasources/text/TextFileFormat.scala | 21 +++---- .../spark/sql/hive/orc/OrcFileFormat.scala | 21 +++---- .../sql/sources/BucketedWriteSuite.scala | 5 -- .../sql/sources/CommitFailureTestSource.scala | 6 +- .../sql/sources/SimpleTextRelation.scala | 26 ++++----- 12 files changed, 99 insertions(+), 143 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 8577803743c8e..fff86686b550c 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 @@ -40,7 +40,8 @@ import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration private[libsvm] class LibSVMOutputWriter( - path: String, + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext) extends OutputWriter { @@ -50,11 +51,7 @@ private[libsvm] class LibSVMOutputWriter( private val recordWriter: RecordWriter[NullWritable, Text] = { new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val configuration = context.getConfiguration - val uniqueWriteJobId = configuration.get(WriterContainer.DATASOURCE_WRITEJOBUUID) - val taskAttemptId = context.getTaskAttemptID - val split = taskAttemptId.getTaskID.getId - new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$extension") + new Path(stagingDir, fileNamePrefix + extension) } }.getRecordWriter(context) } @@ -132,12 +129,11 @@ private[libsvm] class LibSVMFileFormat extends TextBasedFileFormat with DataSour dataSchema: StructType): OutputWriterFactory = { new OutputWriterFactory { override def newInstance( - path: String, - bucketId: Option[Int], + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - if (bucketId.isDefined) { sys.error("LibSVM doesn't support bucketing") } - new LibSVMOutputWriter(path, dataSchema, context) + new LibSVMOutputWriter(stagingDir, fileNamePrefix, dataSchema, context) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala index d2eec7b1413f8..f4cefdab077e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala @@ -34,18 +34,23 @@ abstract class OutputWriterFactory extends Serializable { * When writing to a [[HadoopFsRelation]], this method gets called by each task on executor side * to instantiate new [[OutputWriter]]s. * - * @param path Path of the file to which this [[OutputWriter]] is supposed to write. Note that - * this may not point to the final output file. For example, `FileOutputFormat` writes to - * temporary directories and then merge written files back to the final destination. In - * this case, `path` points to a temporary output file under the temporary directory. + * @param stagingDir Base path (directory) of the file to which this [[OutputWriter]] is supposed + * to write. Note that this may not point to the final output file. For + * example, `FileOutputFormat` writes to temporary directories and then merge + * written files back to the final destination. In this case, `path` points to + * a temporary output file under the temporary directory. + * @param fileNamePrefix Prefix of the file name. The returned OutputWriter must make sure this + * prefix is used in the actual file name. For example, if the prefix is + * "part-1-2-3", then the file name must start with "part_1_2_3" but can + * end in arbitrary extension. * @param dataSchema Schema of the rows to be written. Partition columns are not included in the * schema if the relation being written is partitioned. * @param context The Hadoop MapReduce task context. * @since 1.4.0 */ def newInstance( - path: String, - bucketId: Option[Int], // TODO: This doesn't belong here... + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala index 54d0f3bd6291a..bd56e511d0ccf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriteOutput.scala @@ -46,6 +46,7 @@ object WriteOutput extends Logging { /** A shared job description for all the write tasks. */ private class WriteJobDescription( + val uuid: String, // prevent collision between different (appending) write jobs val serializableHadoopConf: SerializableConfiguration, val outputWriterFactory: OutputWriterFactory, val allColumns: Seq[Attribute], @@ -102,6 +103,7 @@ object WriteOutput extends Logging { fileFormat.prepareWrite(sparkSession, job, options, dataColumns.toStructType) val description = new WriteJobDescription( + uuid = UUID.randomUUID().toString, serializableHadoopConf = new SerializableConfiguration(job.getConfiguration), outputWriterFactory = outputWriterFactory, allColumns = plan.output, @@ -213,6 +215,11 @@ object WriteOutput extends Logging { private trait ExecuteWriteTask { def execute(iterator: Iterator[InternalRow]): Unit def releaseResources(): Unit + + final def filePrefix(split: Int, uuid: String, bucketId: Option[Int]): String = { + val bucketString = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") + f"part-r-$split%05d-$uuid$bucketString" + } } /** Writes data to a single directory (used for non-dynamic-partition writes). */ @@ -222,9 +229,11 @@ object WriteOutput extends Logging { stagingPath: String) extends ExecuteWriteTask { private[this] var outputWriter: OutputWriter = { + val split = taskAttemptContext.getTaskAttemptID.getTaskID.getId + val outputWriter = description.outputWriterFactory.newInstance( - path = stagingPath, - bucketId = None, + stagingDir = stagingPath, + fileNamePrefix = filePrefix(split, description.uuid, None), dataSchema = description.nonPartitionColumns.toStructType, context = taskAttemptContext) outputWriter.initConverter(dataSchema = description.nonPartitionColumns.toStructType) @@ -287,29 +296,31 @@ object WriteOutput extends Logging { } } - private def getBucketIdFromKey(key: InternalRow): Option[Int] = - description.bucketSpec.map { _ => key.getInt(description.partitionColumns.length) } - /** * Open and returns a new OutputWriter given a partition key and optional bucket id. * If bucket id is specified, we will append it to the end of the file name, but before the * file extension, e.g. part-r-00009-ea518ad4-455a-4431-b471-d24e03814677-00002.gz.parquet */ - private def newOutputWriter( - key: InternalRow, - getPartitionString: UnsafeProjection): OutputWriter = { + private def newOutputWriter(key: InternalRow, partString: UnsafeProjection): OutputWriter = { val path = if (description.partitionColumns.nonEmpty) { - val partitionPath = getPartitionString(key).getString(0) + val partitionPath = partString(key).getString(0) new Path(stagingPath, partitionPath).toString } else { stagingPath } - val bucketId = getBucketIdFromKey(key) + // If the bucket spec is defined, the bucket column is right after the partition columns + val bucketId = if (description.bucketSpec.isDefined) { + Some(key.getInt(description.partitionColumns.length)) + } else { + None + } + + val split = taskAttemptContext.getTaskAttemptID.getTaskID.getId val newWriter = description.outputWriterFactory.newInstance( - path = path, - bucketId = bucketId, + stagingDir = path, + fileNamePrefix = filePrefix(split, description.uuid, bucketId), dataSchema = description.nonPartitionColumns.toStructType, context = taskAttemptContext) newWriter.initConverter(description.nonPartitionColumns.toStructType) @@ -319,7 +330,7 @@ object WriteOutput extends Logging { override def execute(iter: Iterator[InternalRow]): Unit = { // We should first sort by partition columns, then bucket id, and finally sorting columns. val sortingExpressions: Seq[Expression] = - description.partitionColumns ++ bucketIdExpression ++ sortColumns + description.partitionColumns ++ bucketIdExpression ++ sortColumns val getSortingKey = UnsafeProjection.create(sortingExpressions, description.allColumns) val sortingKeySchema = StructType(sortingExpressions.map { @@ -333,8 +344,8 @@ object WriteOutput extends Logging { description.nonPartitionColumns, description.allColumns) // Returns the partition path given a partition key. - val getPartitionString = - UnsafeProjection.create(Seq(Concat(partitionStringExpression)), description.partitionColumns) + val getPartitionString = UnsafeProjection.create( + Seq(Concat(partitionStringExpression)), description.partitionColumns) // Sorts the data before write, so that we only need one writer at the same time. val sorter = new UnsafeKVExternalSorter( @@ -405,17 +416,6 @@ object WriteOutput extends Logging { job.getConfiguration.setBoolean("mapred.task.is.map", true) job.getConfiguration.setInt("mapred.task.partition", 0) - // 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. - // This UUID is used to avoid output file name collision between different appending write jobs. - // These jobs may belong to different SparkContext instances. Concrete data source - // implementations may use this UUID to generate unique file names (e.g., - // `part-r--.parquet`). The reason why this ID is used to identify a job - // rather than a single task output file is that, speculative tasks must generate the same - // output file name as the original task. - job.getConfiguration.set(WriterContainer.DATASOURCE_WRITEJOBUUID, UUID.randomUUID().toString) - val taskAttemptContext = new TaskAttemptContextImpl(job.getConfiguration, taskAttemptId) val outputCommitter = newOutputCommitter( job.getOutputFormatClass, taskAttemptContext, path, isAppend) @@ -474,7 +474,3 @@ object WriteOutput extends Logging { } } } - -object WriterContainer { - val DATASOURCE_WRITEJOBUUID = "spark.sql.sources.writeJobUUID" -} 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 55cb26d6513af..eefacbf05ba0d 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 @@ -31,7 +31,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory, PartitionedFile, WriterContainer} +import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory, PartitionedFile} import org.apache.spark.sql.types._ object CSVRelation extends Logging { @@ -170,17 +170,17 @@ object CSVRelation extends Logging { private[csv] class CSVOutputWriterFactory(params: CSVOptions) extends OutputWriterFactory { override def newInstance( - path: String, - bucketId: Option[Int], + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - if (bucketId.isDefined) sys.error("csv doesn't support bucketing") - new CsvOutputWriter(path, dataSchema, context, params) + new CsvOutputWriter(stagingDir, fileNamePrefix, dataSchema, context, params) } } private[csv] class CsvOutputWriter( - path: String, + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext, params: CSVOptions) extends OutputWriter with Logging { @@ -199,11 +199,7 @@ private[csv] class CsvOutputWriter( private val recordWriter: RecordWriter[NullWritable, Text] = { new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val configuration = context.getConfiguration - val uniqueWriteJobId = configuration.get(WriterContainer.DATASOURCE_WRITEJOBUUID) - val taskAttemptId = context.getTaskAttemptID - val split = taskAttemptId.getTaskID.getId - new Path(path, f"part-r-$split%05d-$uniqueWriteJobId.csv$extension") + new Path(stagingDir, s"$fileNamePrefix.csv$extension") } }.getRecordWriter(context) } 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 9fe38ccc9fdc6..cdbb2f7292613 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 @@ -82,11 +82,11 @@ class JsonFileFormat extends TextBasedFileFormat with DataSourceRegister { new OutputWriterFactory { override def newInstance( - path: String, - bucketId: Option[Int], + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new JsonOutputWriter(path, parsedOptions, bucketId, dataSchema, context) + new JsonOutputWriter(stagingDir, parsedOptions, fileNamePrefix, dataSchema, context) } } } @@ -153,9 +153,9 @@ class JsonFileFormat extends TextBasedFileFormat with DataSourceRegister { } private[json] class JsonOutputWriter( - path: String, + stagingDir: String, options: JSONOptions, - bucketId: Option[Int], + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext) extends OutputWriter with Logging { @@ -168,12 +168,7 @@ private[json] class JsonOutputWriter( private val recordWriter: RecordWriter[NullWritable, Text] = { new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val configuration = context.getConfiguration - val uniqueWriteJobId = configuration.get(WriterContainer.DATASOURCE_WRITEJOBUUID) - val taskAttemptId = context.getTaskAttemptID - val split = taskAttemptId.getTaskID.getId - val bucketString = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") - new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$bucketString.json$extension") + new Path(stagingDir, s"$fileNamePrefix.json$extension") } }.getRecordWriter(context) } 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 6faafed1e6290..87b944ba523ca 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 @@ -27,7 +27,7 @@ import scala.util.{Failure, Try} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce._ -import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit} +import org.apache.hadoop.mapreduce.lib.input.FileSplit import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.parquet.{Log => ApacheParquetLog} import org.apache.parquet.filter2.compat.FilterCompat @@ -45,7 +45,6 @@ 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.execution.metric.SQLMetric import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ @@ -134,10 +133,10 @@ class ParquetFileFormat new OutputWriterFactory { override def newInstance( path: String, - bucketId: Option[Int], + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new ParquetOutputWriter(path, bucketId, context) + new ParquetOutputWriter(path, fileNamePrefix, context) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala index f89ce05d82d90..39c199784cd6d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala @@ -26,7 +26,7 @@ import org.apache.parquet.hadoop.util.ContextUtil import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.datasources.{BucketingUtils, OutputWriter, OutputWriterFactory, WriterContainer} +import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -122,13 +122,12 @@ private[parquet] class ParquetOutputWriterFactory( } /** Disable the use of the older API. */ - def newInstance( + override def newInstance( path: String, - bucketId: Option[Int], + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - throw new UnsupportedOperationException( - "this version of newInstance not supported for " + + throw new UnsupportedOperationException("this version of newInstance not supported for " + "ParquetOutputWriterFactory") } } @@ -136,33 +135,16 @@ private[parquet] class ParquetOutputWriterFactory( // NOTE: This class is instantiated and used on executor side only, no need to be serializable. private[parquet] class ParquetOutputWriter( - path: String, - bucketId: Option[Int], + stagingDir: String, + fileNamePrefix: String, context: TaskAttemptContext) extends OutputWriter { private val recordWriter: RecordWriter[Void, InternalRow] = { val outputFormat = { new ParquetOutputFormat[InternalRow]() { - // Here we override `getDefaultWorkFile` for two reasons: - // - // 1. To allow appending. We need to generate unique output file names to avoid - // overwriting existing files (either exist before the write job, or are just written - // by other tasks within the same write job). - // - // 2. To allow dynamic partitioning. Default `getDefaultWorkFile` uses - // `FileOutputCommitter.getWorkPath()`, which points to the base directory of all - // partitions in the case of dynamic partitioning. override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val configuration = context.getConfiguration - val uniqueWriteJobId = configuration.get(WriterContainer.DATASOURCE_WRITEJOBUUID) - val taskAttemptId = context.getTaskAttemptID - val split = taskAttemptId.getTaskID.getId - val bucketString = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") - // It has the `.parquet` extension at the end because (de)compression tools - // such as gunzip would not be able to decompress this as the compression - // is not applied on this whole file but on each "page" in Parquet format. - new Path(path, f"part-r-$split%05d-$uniqueWriteJobId$bucketString$extension") + new Path(stagingDir, fileNamePrefix + extension) } } } 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 9f96667311015..6cd2351c5749a 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 @@ -73,14 +73,11 @@ class TextFileFormat extends TextBasedFileFormat with DataSourceRegister { new OutputWriterFactory { override def newInstance( - path: String, - bucketId: Option[Int], + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - if (bucketId.isDefined) { - throw new AnalysisException("Text doesn't support bucketing") - } - new TextOutputWriter(path, dataSchema, context) + new TextOutputWriter(stagingDir, fileNamePrefix, dataSchema, context) } } } @@ -124,7 +121,11 @@ class TextFileFormat extends TextBasedFileFormat with DataSourceRegister { } } -class TextOutputWriter(path: String, dataSchema: StructType, context: TaskAttemptContext) +class TextOutputWriter( + stagingDir: String, + fileNamePrefix: String, + dataSchema: StructType, + context: TaskAttemptContext) extends OutputWriter { private[this] val buffer = new Text() @@ -132,11 +133,7 @@ class TextOutputWriter(path: String, dataSchema: StructType, context: TaskAttemp private val recordWriter: RecordWriter[NullWritable, Text] = { new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val configuration = context.getConfiguration - val uniqueWriteJobId = configuration.get(WriterContainer.DATASOURCE_WRITEJOBUUID) - val taskAttemptId = context.getTaskAttemptID - val split = taskAttemptId.getTaskID.getId - new Path(path, f"part-r-$split%05d-$uniqueWriteJobId.txt$extension") + new Path(stagingDir, s"$fileNamePrefix.txt$extension") } }.getRecordWriter(context) } 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 1af3280e18a89..1ceacb458ae6e 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 @@ -83,11 +83,11 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable new OutputWriterFactory { override def newInstance( - path: String, - bucketId: Option[Int], + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new OrcOutputWriter(path, bucketId, dataSchema, context) + new OrcOutputWriter(stagingDir, fileNamePrefix, dataSchema, context) } } } @@ -210,8 +210,8 @@ private[orc] class OrcSerializer(dataSchema: StructType, conf: Configuration) } private[orc] class OrcOutputWriter( - path: String, - bucketId: Option[Int], + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext) extends OutputWriter { @@ -226,10 +226,7 @@ private[orc] class OrcOutputWriter( private lazy val recordWriter: RecordWriter[NullWritable, Writable] = { recordWriterInstantiated = true - val uniqueWriteJobId = conf.get(WriterContainer.DATASOURCE_WRITEJOBUUID) - val taskAttemptId = context.getTaskAttemptID - val partition = taskAttemptId.getTaskID.getId - val bucketString = bucketId.map(BucketingUtils.bucketIdToString).getOrElse("") + val compressionExtension = { val name = conf.get(OrcRelation.ORC_COMPRESSION) OrcRelation.extensionsForCompressionCodecNames.getOrElse(name, "") @@ -237,12 +234,12 @@ private[orc] class OrcOutputWriter( // It has the `.orc` extension at the end because (de)compression tools // such as gunzip would not be able to decompress this as the compression // is not applied on this whole file but on each "stream" in ORC format. - val filename = f"part-r-$partition%05d-$uniqueWriteJobId$bucketString$compressionExtension.orc" + val filename = s"$fileNamePrefix$compressionExtension.orc" new OrcOutputFormat().getRecordWriter( - new Path(path, filename).getFileSystem(conf), + new Path(stagingDir, filename).getFileSystem(conf), conf.asInstanceOf[JobConf], - new Path(path, filename).toString, + new Path(stagingDir, filename).toString, Reporter.NULL ).asInstanceOf[RecordWriter[NullWritable, Writable]] } 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 997445114ba58..2eafe18b85844 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 @@ -54,11 +54,6 @@ class BucketedWriteSuite extends QueryTest with SQLTestUtils with TestHiveSingle intercept[AnalysisException](df.write.bucketBy(2, "i").sortBy("j").saveAsTable("tt")) } - test("write bucketed data to unsupported data source") { - val df = Seq(Tuple1("a"), Tuple1("b")).toDF("i") - intercept[SparkException](df.write.bucketBy(3, "i").format("text").saveAsTable("tt")) - } - test("write bucketed data using save()") { val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala index 5a8a7f0ab5d7b..d5044684020e2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala @@ -39,11 +39,11 @@ class CommitFailureTestSource extends SimpleTextSource { dataSchema: StructType): OutputWriterFactory = new OutputWriterFactory { override def newInstance( - path: String, - bucketId: Option[Int], + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new SimpleTextOutputWriter(path, context) { + new SimpleTextOutputWriter(stagingDir, fileNamePrefix, context) { var failed = false TaskContext.get().addTaskFailureListener { (t: TaskContext, e: Throwable) => failed = true 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 906de6bbcbee5..9e13b217ec305 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 @@ -23,7 +23,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.io.{NullWritable, Text} import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext} -import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat, TextOutputFormat} +import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat import org.apache.spark.sql.{sources, Row, SparkSession} import org.apache.spark.sql.catalyst.{expressions, InternalRow} @@ -51,11 +51,11 @@ class SimpleTextSource extends TextBasedFileFormat with DataSourceRegister { SimpleTextRelation.lastHadoopConf = Option(job.getConfiguration) new OutputWriterFactory { override def newInstance( - path: String, - bucketId: Option[Int], + stagingDir: String, + fileNamePrefix: String, dataSchema: StructType, context: TaskAttemptContext): OutputWriter = { - new SimpleTextOutputWriter(path, context) + new SimpleTextOutputWriter(stagingDir, fileNamePrefix, context) } } } @@ -120,9 +120,11 @@ class SimpleTextSource extends TextBasedFileFormat with DataSourceRegister { } } -class SimpleTextOutputWriter(path: String, context: TaskAttemptContext) extends OutputWriter { +class SimpleTextOutputWriter( + stagingDir: String, fileNamePrefix: String, context: TaskAttemptContext) + extends OutputWriter { private val recordWriter: RecordWriter[NullWritable, Text] = - new AppendingTextOutputFormat(new Path(path)).getRecordWriter(context) + new AppendingTextOutputFormat(new Path(stagingDir), fileNamePrefix).getRecordWriter(context) override def write(row: Row): Unit = { val serialized = row.toSeq.map { v => @@ -136,19 +138,15 @@ class SimpleTextOutputWriter(path: String, context: TaskAttemptContext) extends } } -class AppendingTextOutputFormat(outputFile: Path) extends TextOutputFormat[NullWritable, Text] { - val numberFormat = NumberFormat.getInstance() +class AppendingTextOutputFormat(stagingDir: Path, fileNamePrefix: String) + extends TextOutputFormat[NullWritable, Text] { + val numberFormat = NumberFormat.getInstance() numberFormat.setMinimumIntegerDigits(5) numberFormat.setGroupingUsed(false) override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val configuration = context.getConfiguration - val uniqueWriteJobId = configuration.get(WriterContainer.DATASOURCE_WRITEJOBUUID) - val taskAttemptId = context.getTaskAttemptID - val split = taskAttemptId.getTaskID.getId - val name = FileOutputFormat.getOutputName(context) - new Path(outputFile, s"$name-${numberFormat.format(split)}-$uniqueWriteJobId") + new Path(stagingDir, fileNamePrefix) } } From 2d14ab7e644b64ff911772e71f42653ba949cb07 Mon Sep 17 00:00:00 2001 From: Mark Grover Date: Thu, 20 Oct 2016 15:30:01 -0700 Subject: [PATCH 12/48] [DOCS] Update docs to not suggest to package Spark before running tests. ## What changes were proposed in this pull request? Update docs to not suggest to package Spark before running tests. ## How was this patch tested? Not creating a JIRA since this pretty small. We haven't had the need to run mvn package before mvn test since 1.6 at least, or so I am told. So, updating the docs to not be misguiding. Author: Mark Grover Closes #15572 from markgrover/doc_update. --- docs/building-spark.md | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index f5acee6b90059..ebe46a42a15c6 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -217,9 +217,8 @@ For help in setting up IntelliJ IDEA or Eclipse for Spark development, and troub Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). Note that tests should not be run as root or an admin user. -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: +The following is an example of a command to run the tests: - ./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: @@ -233,9 +232,8 @@ or a Java test: ## 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: +The following is an example of a command to run the tests: - ./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: From 1bb99c4887e97ae5f55c8c2b392ba5ca72d6168b Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 20 Oct 2016 20:44:32 -0700 Subject: [PATCH 13/48] [SPARK-18030][TESTS] Adds more checks to collect more info about FileStreamSourceSuite failure ## What changes were proposed in this pull request? My hunch is `mkdirs` fails. Just add more checks to collect more info. ## How was this patch tested? Jenkins Author: Shixiong Zhu Closes #15577 from zsxwing/SPARK-18030-debug. --- .../apache/spark/sql/streaming/FileStreamSourceSuite.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 aabdccaaf319d..b9e9da9a1ec53 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 @@ -664,7 +664,9 @@ class FileStreamSourceSuite extends FileStreamSourceTest { 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(!src.exists(), s"$src exists, dir: ${src.isDirectory}, file: ${src.isFile}") + require(src.mkdirs(), s"Cannot create $src") + require(src.isDirectory(), s"$src is not a directory") require(stringToFile(tempFile, content).renameTo(finalFile)) } From 3180272d2d49e440516085c0e4aebd5bad18bcad Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Thu, 20 Oct 2016 21:12:55 -0700 Subject: [PATCH 14/48] [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. --- 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 267a38c21530b..5960c6206a6f1 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 af81d0586e0a6..1c806869e9fbe 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 57e97fcbd6fe62af4acd60896feeacfa21efc222 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 21 Oct 2016 12:27:53 +0800 Subject: [PATCH 15/48] [SPARK-18029][SQL] PruneFileSourcePartitions should not change the output of LogicalRelation ## What changes were proposed in this pull request? In `PruneFileSourcePartitions`, we will replace the `LogicalRelation` with a pruned one. However, this replacement may change the output of the `LogicalRelation` if it doesn't have `expectedOutputAttributes`. This PR fixes it. ## How was this patch tested? the new `PruneFileSourcePartitionsSuite` Author: Wenchen Fan Closes #15569 from cloud-fan/partition-bug. --- .../sql/catalyst/catalog/interface.scala | 4 +- .../PruneFileSourcePartitions.scala | 4 +- .../spark/sql/hive/HiveDataFrameSuite.scala | 7 +- .../sql/hive/HiveMetadataCacheSuite.scala | 3 +- .../PruneFileSourcePartitionsSuite.scala | 74 +++++++++++++++++++ 5 files changed, 85 insertions(+), 7 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 1a57a7707caa1..a97ed701c4207 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 @@ -102,8 +102,8 @@ case class CatalogTablePartition( * Given the partition schema, returns a row with that schema holding the partition values. */ def toRow(partitionSchema: StructType): InternalRow = { - InternalRow.fromSeq(partitionSchema.map { case StructField(name, dataType, _, _) => - Cast(Literal(spec(name)), dataType).eval() + InternalRow.fromSeq(partitionSchema.map { field => + Cast(Literal(spec(field.name)), field.dataType).eval() }) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala index 29121a47d92d1..8689017c3ed75 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala @@ -59,7 +59,9 @@ private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { val prunedFileCatalog = tableFileCatalog.filterPartitions(partitionKeyFilters.toSeq) val prunedFsRelation = fsRelation.copy(location = prunedFileCatalog)(sparkSession) - val prunedLogicalRelation = logicalRelation.copy(relation = prunedFsRelation) + val prunedLogicalRelation = logicalRelation.copy( + relation = prunedFsRelation, + expectedOutputAttributes = Some(logicalRelation.output)) // Keep partition-pruning predicates so that they are visible in physical planning val filterExpression = filters.reduceLeft(And) 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 f65e74de87a57..15523437a3404 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 @@ -20,9 +20,10 @@ package org.apache.spark.sql.hive import java.io.File import org.apache.spark.metrics.source.HiveCatalogMetrics +import org.apache.spark.sql.QueryTest 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.QueryTest class HiveDataFrameSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { test("table name with schema") { @@ -78,7 +79,7 @@ class HiveDataFrameSuite extends QueryTest with TestHiveSingleton with SQLTestUt } test("lazy partition pruning reads only necessary partition data") { - withSQLConf("spark.sql.hive.filesourcePartitionPruning" -> "true") { + withSQLConf(SQLConf.HIVE_FILESOURCE_PARTITION_PRUNING.key -> "true") { withTable("test") { withTempDir { dir => setupPartitionedTable("test", dir) @@ -114,7 +115,7 @@ class HiveDataFrameSuite extends QueryTest with TestHiveSingleton with SQLTestUt } test("all partitions read and cached when filesource partition pruning is off") { - withSQLConf("spark.sql.hive.filesourcePartitionPruning" -> "false") { + withSQLConf(SQLConf.HIVE_FILESOURCE_PARTITION_PRUNING.key -> "false") { withTable("test") { withTempDir { dir => setupPartitionedTable("test", dir) 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 2ca1cd4c07fdb..d290fe9962db2 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 @@ -22,6 +22,7 @@ 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.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils /** @@ -62,7 +63,7 @@ class HiveMetadataCacheSuite extends QueryTest with SQLTestUtils with TestHiveSi def testCaching(pruningEnabled: Boolean): Unit = { test(s"partitioned table is cached when partition pruning is $pruningEnabled") { - withSQLConf("spark.sql.hive.filesourcePartitionPruning" -> pruningEnabled.toString) { + withSQLConf(SQLConf.HIVE_FILESOURCE_PARTITION_PRUNING.key -> pruningEnabled.toString) { withTable("test") { withTempDir { dir => spark.range(5).selectExpr("id", "id as f1", "id as f2").write diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala new file mode 100644 index 0000000000000..346ea0ca4367e --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, PruneFileSourcePartitions, TableFileCatalog} +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types.StructType + +class PruneFileSourcePartitionsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Batch("PruneFileSourcePartitions", Once, PruneFileSourcePartitions) :: Nil + } + + test("PruneFileSourcePartitions should not change the output of LogicalRelation") { + withTable("test") { + withTempDir { dir => + sql( + s""" + |CREATE EXTERNAL TABLE test(i int) + |PARTITIONED BY (p int) + |STORED AS parquet + |LOCATION '${dir.getAbsolutePath}'""".stripMargin) + + val tableMeta = spark.sharedState.externalCatalog.getTable("default", "test") + val tableFileCatalog = new TableFileCatalog( + spark, + tableMeta.database, + tableMeta.identifier.table, + Some(tableMeta.partitionSchema), + 0) + + val dataSchema = StructType(tableMeta.schema.filterNot { f => + tableMeta.partitionColumnNames.contains(f.name) + }) + val relation = HadoopFsRelation( + location = tableFileCatalog, + partitionSchema = tableMeta.partitionSchema, + dataSchema = dataSchema, + bucketSpec = None, + fileFormat = new ParquetFileFormat(), + options = Map.empty)(sparkSession = spark) + + val logicalRelation = LogicalRelation(relation, catalogTable = Some(tableMeta)) + val query = Project(Seq('i, 'p), Filter('p === 1, logicalRelation)).analyze + + val optimized = Optimize.execute(query) + assert(optimized.missingInput.isEmpty) + } + } + } +} From 595893d33a26c838c8c5c0c599fbee7fa61cbdff Mon Sep 17 00:00:00 2001 From: Jagadeesan Date: Fri, 21 Oct 2016 09:48:24 +0100 Subject: [PATCH 16/48] [SPARK-17960][PYSPARK][UPGRADE TO PY4J 0.10.4] ## What changes were proposed in this pull request? 1) Upgrade the Py4J version on the Java side 2) Update the py4j src zip file we bundle with Spark ## How was this patch tested? Existing doctests & unit tests pass Author: Jagadeesan Closes #15514 from jagadeesanas2/SPARK-17960. --- 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.3-src.zip | Bin 91275 -> 0 bytes python/lib/py4j-0.10.4-src.zip | Bin 0 -> 74096 bytes sbin/spark-config.sh | 2 +- .../org/apache/spark/deploy/yarn/Client.scala | 2 +- .../spark/deploy/yarn/YarnClusterSuite.scala | 2 +- 16 files changed, 14 insertions(+), 14 deletions(-) delete mode 100644 python/lib/py4j-0.10.3-src.zip create mode 100644 python/lib/py4j-0.10.4-src.zip diff --git a/LICENSE b/LICENSE index d68609cc28733..7950dd6ceb6db 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.3 - http://py4j.sourceforge.net/) + (The New BSD License) Py4J (net.sf.py4j:py4j:0.10.4 - 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 7590309b442ed..d6b3ab0a44321 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -57,7 +57,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.3-src.zip:$PYTHONPATH" +export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.4-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 1217a4f2f97a2..f211c0873ad2f 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.3-src.zip;%PYTHONPATH% +set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.4-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 205bbc588be09..eac99ab82a2e4 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -331,7 +331,7 @@ net.sf.py4j py4j - 0.10.3 + 0.10.4 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 701097ace8974..c4e55b5e89027 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.3-src.zip").mkString(File.separator) + pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.10.4-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 525dcef5b7d99..99279a4ca8be9 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -140,7 +140,7 @@ parquet-jackson-1.8.1.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar -py4j-0.10.3.jar +py4j-0.10.4.jar pyrolite-4.13.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 562fe6461e753..f094b4a7e167a 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -147,7 +147,7 @@ parquet-jackson-1.8.1.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar -py4j-0.10.3.jar +py4j-0.10.4.jar pyrolite-4.13.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 747521aa2a566..7f0ef98680a15 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -147,7 +147,7 @@ parquet-jackson-1.8.1.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar -py4j-0.10.3.jar +py4j-0.10.4.jar pyrolite-4.13.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 afd4502c59d33..4a27bf3deecb6 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -155,7 +155,7 @@ parquet-jackson-1.8.1.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar -py4j-0.10.3.jar +py4j-0.10.4.jar pyrolite-4.13.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 687b855b649d8..151670a8e23e4 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -156,7 +156,7 @@ parquet-jackson-1.8.1.jar pmml-model-1.2.15.jar pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar -py4j-0.10.3.jar +py4j-0.10.4.jar pyrolite-4.13.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar diff --git a/python/docs/Makefile b/python/docs/Makefile index de86e97d862f0..5e4cfb8ab6fe3 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.3-src.zip) +export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.10.4-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.3-src.zip b/python/lib/py4j-0.10.3-src.zip deleted file mode 100644 index bc54f33af1515c0676bd831bc5a02f112b28e0a3..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 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;vSJ@$f#yuDzFf7e&r7MV+=%bWhUq_X^r zH#f`P&42VV$1|9dGMLYINZV=CIs65z8arM2I~)(ioa!ao@ltTF#5lo^S4-PblK zy-)^8!w`v0o|QoxZ99dFDMc75zp?k3fo2|d1Z5ZRo8&x)3LBju_h)PcAkPz;!weMI z{)}eE=T9UMfkS~x(jv?~6%`-Wl>*`w0Zcx<@i_P*VX4?uGh!yOxFp444%{297-k?m z73R}T0KhRE(Gd<_T1K7%!Q$a?xK0?PJ!+t3Os~N@G}+=JO!)L>@SVh~N)E#5XjaVA zRRDFHHyv^zAeZ2SfNR`pgV!L@{#q3~20qP;IFg(Lug1yL$V^9<3?>ECLu##>*(a?(5%JZX?j+I~RO3kc(>$>z<^^d3qMZcn#>Yyk>3K3x@ z41k6#Wfu>{KQP9Ch!`waWhjP4OnHeDZz?j5zfmZ*-&VpVG=IBJk}bN_0U->h(1&=Y zp~`^s;&XijeW46KegJUco0?Fqo%t~6*a47k(LUcZ?bpCjm|U>zb{nnFC_C=*zfTp~ z{PT<6x^kEuR+R|~$-&ES&H*Z5d-0<0iUrH`F1V~G9oquImm??3f(8A$X4u&kMt+q_ zM3ZcJW_!RqWM%)WpjL8jM{N$}ufkEtuO{oyk}uXA%2x4V&5?AxV8#2I!e8miKmH@m z?=@RFwUHG-LGB1rUTKaoJ^I&ya=SIlLYGdM^Y4Ah{#Dxc8KFh3uJhJMq&_YnLD@VdMa{Ec{;W{F%n!HV&j7gCSgph=&y;u>)}hhC|R z7tx;^DWaz6-v*>G65LUz(D0J_hWhps8Wk@_6) z9jV8Ps*QfoVZ)cr>*8}*4hem=l%?0QDO~1rkQ+kLzWVKlhV1*h0 z-SccbgI_x%J&EHeXW-iq0Xa{m%l3p)( zg(;{&Kw2i{#Ud&wy`v|b8$9xax-=MFPvZNXggoBAovLZJkK-%pv>f=jQgf3?&T7=+S9>@qnQ zRmjEhp3~G66(z%1Q*ip6#&qJ5z$ur6fMC1>*D0cEnph38HZ}}sekyF0RZ0eSp+3lr zG=`V;jB&z;OBL;T8F0{4cO@D?D$c@8Q8$N9gkA(YLMb-g6MbZo4^$dsg#gubMz7Sj z?9mGRIwrZpb>e@WkswsdjMJbsttq8VbF0)T7+oz&Y-4JmJ2oyP0{Bod)5?A!W zN6+KN@>p+TiPB1m6j&4lm=f0{@_bgsMv3m_$n*1lpBP~P0J*y|_V%v*tE=qHRu$5I z>Arrx=zhM=-`|K|TCIHCS$@2*=K4BvwY`Cvn5E{6v$gayrR+{+Ly}nR-QlJw_H0Is z3k50*n`oXXqn;O{WWQ`h)jR8nKVGl>X^RD8(E)yX{X-MckeW(0{ceMy2Dm!;QV-8K zRcu>*utsrGu)q+hp>c{@(enlj0>;Rp?gdq5a4B?UQ-mn=jOS6}Y(p51I4NZ1ax{bP zOU#)HX$G6S=<{Zvx(eKG_bsMq^YktLX99?OzMG)vatO}` z=LmeO)`zKY3PMpisuG*WtjWn)^FkJRWF+>akWin*E`W=!ZEus_05F=5NBubT*qXjp z!i3)XwB5RTy@5E?(hQ_3OV`)K*oe0XV=Av|hn;VFLx}sndTN?YW<)!NqgNlefECuK z&e)1IyPH9zZXZr!HFV@&JSs@sV2ngllwiY!Mi)JJcwYT(_@`htK0DaRm>^P-UIe5# zDwN7f=lYn7N8DxK5h}jo8!)7j zqA?>t6;gG}u^s5J(K_Az8DZc3pPwO)K>_hFA`hmhwY)0i!&|OTd6yj7DSA=l#Ho1} zOJOvsywZXd@ViS1YO3eM48DK32G8KPwYHJreW*YyT1BPF6veW09BO0;p0&(5Ln(1KwEj|rg_3tHSjY=5Pw#6?6U0-UxuaSKE+$Dzny%=wb1>#>hla@`@$lPdb z;l6+++M)?hkm_5S7igBrBVjS$>|T^^A)2KJ+yBIRh9^-)pqjN|B4I#c#gx(V(iN23GQ&d{yID$&I*SXv+uZHy%@ zZwj#hjuzu!opRqD2?frtnQuc)D2JL|n+(wobld!^V@h11N4tzz2qNuO7#ImZx^M;r zJE;$IC8f=}SJVCUhN)m5?5U6P#05(jy+jORX*PZ)rap;)NaIx;DqS48a`5;su(33qP z(C-Ndau(t96nS4zW16r5$8(~+B^KwUGgM1`HL6E^(ikk)X%C9B4b1c@)2)Pv%j~3{ zQxe=XMuJwFhdTVXpd&-&Vf2uvrf}4bb0aJJRgEO?je(Zzs$rGiq>g^9>TZ|*%2coV zMS@6NLHXCMMtvSgM~)l%;?ov13z1@oJXE(bk3d~ zjVD5q%4SS4-yY#!$P1)m#lizf!L2AwMEsg)t+?rIZmof9}g+jCpRqSTQI27ks>hF21Y$50fb&D0p*5q$>)lMrlVy5nXva^{R= zA2R7>%cNS;#AtA=y^D+YD^SCH_>c_2n<*0Th9{j33SHzkda@@GaAS|iD`0O0BV-%3 z>nu|T+EgH;nn8cv9I4NAwUNBZOIl95l^5!PAB!u$+yt$?nj<_ZY#-cQt0}hwCcU!i za&YB9GUxJ>_^oUJu@Tu(e4AZ#b^qR)!2C1%?)u2R$TQUfw?LH|XB4|{OoH}{!wGDC z%j|FuNwFD`;uf;(gdlxz=2>8{#e>Dw)PmZD$L&KSxDoLaE` z)rzSEffj;y3Y&y5y_yg5D;(J<=h>S^&fvDdj>M_EiT!cCg%Ely+^VdJ$|eTvbX#Bh zx;On6DE%SC>4C=*pv7v_z}1JUvz+W+286fuVR6$%pbiG%zx%^#FG#(xVYbF#FbDSF znxIQhZXLx_?_F5VQBb3x22$8+!8I>Q-?Z!N64B=i34EpYAT+>lzbK%d14{6~Up#N~ zMkOKbCfzjlO1uPaXyJf;75(vezt#SnzwCXS8a5n&g8&pUxzB&j-3|V-vt?KWXRdZz z_vz|j!wZOg*n(v}i#|}QJ70??w-7$96SeZ4{~N{dPUZ4yvq9XrfjW)dK<*b-plz_kN75`RWl39lW6qTS~CUa7W0 z55eDBgJ3Ryb@m80E0mb^sgH`-TasMw);Itf*FN4$-a>Dv`+er}dAfx>9dl?$skzHL z!G#Sl%1C7?CMVaPV9um{_!!2xrbp<1%_~W|dHC#8t#d5PW*s}lZwA3;sC%@ve{Wa2 zq<-IF3NwpE$<5&N@>L@=NaLo3q8tMnleoKHk+#Z?hxBF;Jc^_ zaZ*~q!NP?%`jWFK;%mDap(_iQlH^*JtUKXvxL#TE+=FBD`AP{ZTujVq(YA9mKTN`P zs?U`$dFRV_#BPn3tBhJrA*tETuhg&hjw=rj&85vfHxwf$t~2A&5umj^;7W_v{0c>m zm{OCOWDa9E_CecHo1TuuP)3q;r!(6W=Dk`f+~|8VcU-UH!_iTA&pNU(Z8}~}EPKS< z*Q?be=XWaR!i!Hv)Fvpb=yZA44k_6WFV8d1`0VT#s7)KK_r9gFb534`VG)|cj3c6WB*=cB)OJLr66UAbhYgf zLEEs>`vc~@*MrNn>n~uX7j((JMFvc?Wv#NoPHQRl!W2U zFE(O3P8S_B9ca#wI?F2K* zbe^lV6!uWP4AS6ago29*mlp>%+~JX{i73lIEN!+?*|*X;P7a|(QD@ThVM}I606O^X1xFLIGx>3 zrz2D8F*14rJF+mqvU)tHA{uY0?%GYizW}7WV3Tje^B29#m6R7op{Kohny#Lc^3W5MAWr=z<0Lq`!puV%Z)ZYxW;Noa ze68c`cnd(E>q|G2bNL$Q?V8Enc}UV$j4h zqq3p4NxTto^`}@!o|U7gqZHgmkeRamwHRdW(;JgngzRA0b+r80oH2I;t!4KjI8*~T zF|u)g9+s<#r;+k;cfW%r!x^=;r30p8QKVYwyebFbeR?%Fe#ag@-j~P6s;>%??Pq5C z#)LQb5YbDGw`<~1csP5<379~S`H(|&b{}#)gm)DR-o{2VRq2uvO;b6O!M(?(7zeAE z8VD(cwRiu7A9w<*|82aNhRqZj?Kk@2zQ50fisE{E5s|<;Uw?|WFNJ_unB(dbhRAn! z;0kwvSjO!eH<)TqWx(U4fvU!rRqv!Pg>fq6 zR5fSr;FLq1wlLJ_Ir4`@w%j&$Y3oXp4-t;Ze02B?GXPpvTODb8#>RtWzWBVqyQ{rB zQ=q?dZ-Eua*h`u3uMKQ=NXIk&MA7H^@ zthG>VS0%h}YqI@}-BC?)Jha(&cBjkMGaZE6ie16Y+M~=sK*)$Rpi*mxHG)7axd9aw zHk)W5<5@$xxs2*b3mwD!WBiWt_$0CpkHypeA zyhjDze{7R$bL?jgLbIr2tJ2B0A1wEyuy1kILaq8%Y3qP!q%P-iDKV=>UfG*N&o(W# z#eXSPp`UyL#k_qf zW^`@;QI0sP(HgU%kC*&XjOCezb8jowfgs)6rl3~Lz*%>*_<^4>SXd(u-{Sr0?P>S* z4tD;Y<5bz*ya6ppbcc-q^YL5Sil4c&LNgHgr4b=Rx|&9xypEpE5b zm?CtdzM4yj*m_w3G-ZzQX&ku+>7t%0_x9Giqo^6aG=m+`LI*@67QA z`~Ro$?muHazCnP1jG+HhW&S@J&)(L-@PG7}Zq@17MFymvQ!24-=>}@#3*r!JFAbu- zfDnN{jslfqk`&eM%G}o6n_3WQih2@yD3W8+5ubxAy$P(p#V_nW?Jgt zNOLg3!6Tux^o$q9)s0gY?a-;4a@qH7V8OcpsYNjqIwxovTqHyysq8_dw|1SaJ}dBj zBZS&QQ=lhD^ofsU+w6v`nG*ZTF9Ek#c=$eWD9>3CS{4MOmm{@w{sF*>pC_pO{se`_ zCPy$gevK{ev^bHhTp!m<4JDS-vS&}`5EM`P9YZmp(WPZZR3!Yu6~c@~=WCD9!kD@p zHn&oVBJOZ?r#A<2^Qm0S2-k!nu8CB=mtf3IKH@g@KsO_Fk=KfJF9DMi+>7(#Ka!;+ z!T2OF^Yk%>C1+ercu;AI-%93tLM&(_tcB}?+hr~*{Y67DC&K|39J=MqJ!2+Ux2WtF z9;3H&o*8TOujT=mzk7c*-dSJTl73N_v>j{*|bt~-C5@f!)lqjYzLnz0w zD(k{aCKkmj3U^7$Ju7#k6U<%qbzb7lWz4f@3;Y)fN%)i#N}0as154zWN`g-V2K1FH z?Ie`P0k}6T{SuR}sDF16o38X`v`aGb%`&yaW37Wnh$Blv%^S;9y+?IJn7VU;{G zP3Os@N6SCz+WoCX6o1_>5(g_-%loSgw1RuvzFjdtim7BUZ{E~i^&A}QbBA3O$GUIm z9*{#x;bbh23 zES1x}kki`J=PQtlb>Wlv12xB?5R&Im=Jp_>N^cBaeN+f@)IN@vWQ zZ<504AzqZS{OCRzDIgaxXdtCAN%SLCy-g>TXJv{eqlBi<2!n)eb3eVwAsyE+w=QP( z7-*_iG;oHVUBe+e94S??TowSR=UlU#aNd=uZU|9urUbO_p`}AN0<|znxnMjRBzVYE zhk|q}d5^Nwt2iRlZ%%&ZE()>!ZTx2KXC;?RR5<6p{Z3VFJ2og$%dM|_qy_$<92~tn z4_K$zi|uGupdlx=&mjV-+IRrRHNiv(lcQduggt14WMDX?Dp5&e$IY;5t3Gcs5USE} zWRyn@7i?D=LW=m*P^BSXD;-jk_{}$S%cD6{ta`>|7#*sZ7->&q zDKi36Fw=QKTy2R!qPa+GGDRk5WP85t(fsZ^oEz59T(Ggxv(Ed-d+X0X*>gmxB7$Vo z9`JN@I|7gPyW=AjITdS_t1L=j<`seD6jFY5o3Dmc%au9Jlg}B;fKhndGHT;oH!KZ!92oZ*{B?XYe@a^xnuJ!> z;k6>Pixz&*@~VB*S^2TREj&pLN|2eUytb${LFk*D-m-EdLwSV83#|qz*?-WLkHbsr zfJ*jeowK{$Bb}sUrzN2co8cuL`;N&&Z{Fn`f&VXFp9fC*i0$)TX8HoIodaXJlH~p| z`^%#|Phq~bmUlZzGvgT+C0~cKbw#7L|nursAQUdyP zK+G;$Z0lLQ9HuONI;c+3s3oFSJK$LU|2qBuh+MbzX_VjEaky{ti&!pd|U z^_Wzz{R9^zp)$<~Pd@1ELZFM_=jlP)%CdR$ zdc~4QGen70Y5GtCoWut(0v%C^n^9zd)ce8C*At$PS1O1X>3H^^0xV(dST9pMZ^W%# z(nu4w8i>OD7ccjK)AchS>#q{>$d8EJkOdXF(*^xN)23B(EXsMQ#Xk~@n4wB_{Cg{V z|Jr4)JIuL6ya>WC((cs!?7zctKPfR^0{2KIl02vYcHN?A5ibCg*&~PKFfj~+$$6+C zH7dUf{(hVa9-Y#;WmM`((@JH`vL*CaHk}gT56APV^IJX%#6xXp%t%Ah^AWs=9j-F5 z>~<=0N$P|}CVj8xF8*j^du}RgLg*WN@V|sE(BJ0TFBVxn*S*eprWC)#H)*xtFi!7p zYb$lj1!NNadC0XhjHfQ~wJw^rrxkr1TfB+KdGDB(!Cd)c8&aZusgfNTp$m~#LPfOd zsu*9ukR_ZuwEp=lxT~hVrvc`qz27y3q>YQg6*GUPhqH)MD=CU#bQdnaPG4aj$F`a@1p%(V%)Z>=P<^2*q6SbRc)!ZPEDC!F4>FdQltnIC z@@D5>GXg#7`x?sMoOEL61MMj=7t2V_q-XRI8kC}n5Iu}1yZ8}j3?a@}If;k3w(eA9 z=mwOLba2I(aTMM0A(lzH5dkHO6DUMK7^PX~WVllNgb(O;76ByLt!_uGv50Vczbg3- zT<6m%k+o+^;qI|q$$3=UkA)fN9<)J?7?2naxDux>JmZH^WjGoIQT9g;D*o!YNJqdJ z?ir0lCyuq7CR`XkFDuR`tnd=m@Q9XCL!%UwI*OEI?%X-O9jbo~=tpekuIV+fLb*j9 zQ`k|4V>b3!rp{y;k_6CH2r5zfL0Y`KF{e50ztKZcV!ZY43pXcE^Lqovo*U4!y?=gU zmu~Sn#Y^>0d7`IlBB$X$fh}5N1suZ~GL3AC{Uk4?ThAo}ziLk~efvi-@YoCcVRE1u z(;SO)$?ld7O>7%TyT%nsune&c7AlD)yy|X$Ju-$CA9&!h_>vtU6b{gUnj3Uj1ZU#< z)71w|H2ZMl(|x$$gsW0UVDfCLo-*NPbcVo)Lvkg8v~ozvZ{*?N@=&@5w2a<1Ei5}l z59b6bb!+y6byiMlQ_k}`hiWda@cU^ZQVQ1Vr8rtzNlBCEwa(5e_UzA-HOQH&E~=?z zj%a%lZ0^+rmcd7Xgk3wa`;t`_mC`;0i`X1A#XhohR(R8PjKjjgJ>^)y*|ybA58+`! z^uz*RM!WqHhDN{rH*~K^%>6EqS|5S$h6(L-S|w()rV*i#6vpj>o2Xljh)w)Mp8TuM z*cvK7`4EW1*S8+TfDm+91?RU|IOk2x$$U4B)TrCdl)UtQpN!tZ+0PIym%*wZPpUV~ zLi8%=S{Pvx8E;GOv$|UkxbK_e^Aj@{Y^||}uHlK!jvUrzhxos#b;l2RxG zU)(}5Q0g9aC8v2=z5Zifna*m65d?|c*$l$Pzg4%=!K|(*tlygoa7yj$4B$$(RI~u< z)r#qhU4HAu|AITMfbAKAzf)(AIvR6y1t0v%Hj(z%9=sXk5N}b1wp9ASk5@W|9)_w6 zm3`BSi0)@h?>Z+zZaQwPg8#ZD?_I}R4S@8mgEYh0V`nq+4&p+Y_e+C@bFK!)qs4L)#T;VuK1^H^O$=VXOa z`AxQfK34HAhP7m_bIL^<*NzKYTV$@ha3P+toYn-lXivT+(9IN(Q0XDeh|cPdcSqCJ2wv9=?`H{Lyr4XFS&}HW+Tyh*I>hpgN~O&8S}nnGcs)) zU3e$SenoagtkZ3q3qw$yQ*?$hIY_UaN=j+rH+mRja1CBDfJ-6GV2~rPNjBh zHRUV9)Q2Jd!xV($lzNeilCQ5@R0e)5L2~=l6;cTv$;ru3^u7hLG96bdEbtz-j zvXPVCq&)%$9?ev>JO*8N^JkEktgW}T>wA*G-Cj9Sn~+WRXK!%uzgTTH5%5F-RZ+*q z24EW#8#`Sr3YM`+1x0c&b0Zcl0M7V7aw@U$Sh>l!g3%B!8P~Qq{A>t%0mWa-Pbxgl zRdYBFp|+?iHBP`qcd~5fMLCS4VHEM2lh91FOz|NGQ{CVRPFGd14yp(nQ2etd-?=;J z0Um`7wQcBd6ZY4Vf{pa9_-1PM(<}jgJb=&vje03bpqej`S+e&=tHuYnby^T_?2^Lq zpp>Kk>VCec*4h&yAA&WFe(Krmcki;brVT#NbZT`z$gby9T6v8L+2#IJ-=BXu^so;K z=4{6zpW~aM4`?UI4kuhIW)0nLRHZK7I_iuuE(9`5o2;Lq2SO;GfU5*1BFyH_@_lN; zrA1kNe35Eva>)@z2fWRk!qDc1xeY5XtLHlQwG279;(YJea$)p?cd zf%jhIj|L48^(U#Z*>!>I<%faqKY{M!&b0^EOTi>sxY@A=A!prwMh$y{>F*D z?8bE+jHU^_8-w6nS=|oBA6u2&12Y0{Ks6)?NpqdIPvx*RY9;1q9B9n)adQ8K;Oe2g zVy$4Qtj@hJebNrgH!i4Jm1qU4oTlXLOv-mFycN!A;eMC?JdIkD|6EMOCd{nbF=R?7 zZfe(SS~qS=PgYuQXG&k3DnuIoS3BNL72eLz*w2v%!&V*(Ig__Rue0UXd)LpS=eIu_ zRb9>RXl(5g%I5Wr`c369EQw1a{7WzhwCyZ=L^GuP*up1(nOKb`fL-p_uT{B@7p5hh?NtgK!@!Xqg=|I!i8s7$?IS*xsl0$#Gaw`p31q$<)&l zGjf_{Hx`S$sJ*92<4|Ivaht=Mma3#cTuPk-n;kcnj4~p9)cNv!M-wmQTc+COSpfgv zP4no|LGc=WyTxX_3aV8SGql;xUcFoickT+I);~-|tt0=UcORFZkE{7pX56-GA3vcn z)%uKyP0nVOo*ey~i;>a6&H>cbCx$MczBL@wlZIdKU#B|d;*au?j+65TdQpBQ57yKh zR9_ujsSf;QcF49?5t3kn#$)*%rAk06IAg3|?AyVbEsz(+JlOLIUMtV?E%-Gt#czwB zR!ra~W%dP9GBY}Sv7xEVT>Z^k#q_}2$C5;sGvbi$i-+IN=1)i*;Ih>k2Y_DNi!*o| zXme)-r)R}Ev6JO$^ypS4s>MH{SIIh_oS=e+QAltamy79Wnv(8alL4u+v4I)yiq#R# z0@d03c7d~fJyL0gkCr06*II7XO0pLK`;S%2GbNWDJ*?&8B%9Km;^x`?=``)&Oe^+1 z;VG;&eA%@N>bVMU9t{1;)7ePMK2TfpgwvDanSTsw164Q)wknp=e~~B7h`UqLz7rb@ zO<#!JWlLR%m$<4s8=$By`P@`rTg%Vm{k9D;RKV!Z6y(iJxg^B&5l)IMViPZ#H?bI@ z2ekb+8mBNaC!Hyw${k|{VH44vZ(0^EZ~VFBlW%lZBZ#QR13$Qq;C3M%udtW;0G`Vc z*I43Rdsv*)vU2(96e1f%qQj1vNM<{xE(SGZ^q>E9pXIvl==pvq``2N?ZolY*4aZTUE{MvR3;>RYU_a8#TrP0BkESL5<`lC`_Hy%i;wo(WYIMLJbcT)9<=oldSvD(e1p`}BqPbRt}L&bF-SD}&O{jWg|o?7(lu#u{e@*=gS zN@Wop`SP`GeLBd)wm@doCKyXioe(hn>2ar#A3qo#;w7;@`XX2!A3gc_tN8dUfbZL~ zwl%FbU;SC{fhwSz>SXxZ_kpLE4#RVOuk5+(QIk!(cEvTpRMNAv>H@uw&M-OJM(;pl zCQC6c0cmvPM*LF#I(!dEXgpYijdXPt2DcnvcJmGSgwC`%2l! zGC}s&^_P!y^Ly2vadwbH%zT@xDkbOF+-56G>#@kpN6p1cf-oB7Z@`0>;bX886UzN0 z5yi&er7afGdTAcqT^TR2tNiay(Zc*PYGtYg=>vPY^mQ-`kI3yGmMeaCDBCBpKei~sXG8h#UiMU-VUZdv1B;H`x~XdOgWbsK&q^N& zF4zT(PJC0xgK00Hz1pZVk4$)Te{tfoF`oq@=u&nEc?UeTrV@Z>Gx@PE=l~p4Kw9SOVVKjE0Ky~JI-?h*-cjKJZ>L8 zf9h60pPtQg@pH^ENBwz%wD>(f773pLguZTfAIGL{cVF-C^TXggH~gPFaf_vQz2?53 zFIP`rD0;rP;k&;-uh090bpF0xi8(pBKR<|ek%W|UgM)>QlUEOK%k$^fWp@vIcW0}m z^tpsN#@tF$+MGNd?+=T?gS%#!;-~*DLN4b|AbOkozxCTUMDG&hnueFs>l5JX5#$1p z7OLq#bbcPbSYKMH)PmW+KMT&|5`TDq8Vmd{T<86{L)Pu0aL5TGt@?c^tkssCpsVcl z&&(J^Mn-mlcRLyH;B!mG9D#Od(vmOZga$w$lUpd1W@C1DcIgn!=JqFTVaKj8=0#<7_&*&NY4~Z> z^83iS5^sL=O*x;(RSBXl1Rt<%b44(R9d>WzSZpxf%D98(LeYO` zda1Ei#%;2OfS|p)x>Z1aa!ONBel&~`59qn+s$ZJE+#ulCLo^Hv7Z-{))`-TGVvzTt zl|(KLm|?FZCx?oHi&}lA8kFz`xq5<{2I?uq+oOI=P_EDu1+&2ajQ4YYK!UVgikT7`vTVl49^`)kWolJwaf}DVxkzqM3 zg$cv7jtM!MHn>eOjtiQN`2|5)*j4TS@Nw{D>gMS6wplotz_(Bnn`kaQc6i8 zG2}3s0i(Np)JHG;R6APkz|%f=X<4 zx|mOODVnip3QFx83y|;$sf07da`m-Pd{KJR@dD-PA3cMVSOh$qR7jQ^**e~xzP7G^ zHnibJexwH))xaL71Mb5b!L-1&ML-6KXi}yd(vure#Evfbu&tjNpYW`akOFf5e1Uat z0>ub%Z0w*=@iAWClGP2b>jMFsE{wx75EibUY+bp7$0g@mG=d2#;G&(oauvjuT&i-) z>jy3HiYzr#1nA$Ls?t&(PBpB=-PX zI((lm_KrN=obE0t-(!2GV)gUsGIhItSSSlair`a~%Qk-(3lIqdeU0{H%BhMNiuy?c z?6f$!In32SaL4g4js@kF(F9}qjr($i%4)hnFG9)VfratFfyBopn@jML4GgpGosnD{>Nb+9pUbvcAoB^t7FABwcSEQ&hC>aKhIh|%E1eAe zWhQYj({4&nz@gZkajIhe2sa9^{abkI{PGtYV3;}^3D#1WM9Pg+J49HzFNC0;(QFq3znCATg8MJ&S`!l!A zB?!DhB+-YAUC{91WcI1%1N~Kvz2`n^X1yRqoW&N_-6`B$ATVk3390@8$v-2{a4wa-N6OSw_xVJXCyrZ^C1N8z(-O>?fU#;wIOw4K!mLg+ zo@)+GC&$yhIvrhXAPz|i;_zM<{wErE1uu3~IA}^bR!piq=CxvmGeMZyiung?^ zo<*sIpOEzeH9IIwTs@^^p{2JZRA=K-;=8 zm@s{bLt&f>l(5`hHciQu=MKt`J=13`Xa3*I5)@q*oy(jD#b&7}rD;jdfdI1L7D4t?2}J9m9pOfH@qqWkd*KM}pJ<&7tmb@OgRO63M#-O!m1Cz+ zx(1&2Xw#H-)6_ZGqnI(6^iAa9#0@)vacT?N#edw@S(A|NI0imBIpkrZ{@XdRuX5z9 z8~mP{{V*D!oyckS#!b4~%By9nVF?xg0^1RDMa z=|-x|0Av0ajod|w;{iw!^a>f)VC&IaV-&p(RuJfCwgeK&Fu=X3R6lT)P-j7+zoUel zGxjv5n}lQZF`I&8z#}A#QZLv-WQdqlqpdMdX&gXA^+NY(QGk7?-OKs*_2={Xyt43C zFD3fc2t1p@w@QgnvnBwoM1UrBU?OF}Xna&OPZvf=)SApA<0aM z_76&l(>X2~JPFfM<)`Vs1Z?#4wW{F|X9}mbY^-$5_SllmcO1Z2qH!R=a*Zb-F>vBa zt&yc1{e2$(#f%VJv))60Wf{cxA-IQ3gi)S}wAv`2l2mX&+1JK%&L?2*^DZzBtPJa< zBT&-vW{lmiIK^4K0hrmfpCj>8=FO*E8kr1Q4Qbk0W69aIRCz)W(Pf*V$w--PiI85A3}8Ec&^b} zF1qshS-^!}V^i#iXns53E=(J+zMq<-(E^^hPz>3jyf&SwajRDsdH4nz1-ji~C~*5R zBBL{tS8!hg50YN6oNqdhyDh8g;Y>JU;oy*}3wr+vl=$~<`Uv5L^glXc>dr2i{>Gk> ziEg19o&yICTXfxyJ(d_VQjC#yjx!si&YQjfgs4|~b&?8=@akwXS?rDsfg*?z%$PLF z7tiEFk?yDy3-_&XxZ-Y{f=(#)4p1244~tuW1uwBZQ__&#}h<41(bV+@$Z-845xB6dYrE)6A@W!6vC!K}qYL|3QBEXlj8B&*- zOA3!RhSQ8cl{TEp)sx{$Cm<#=Q63WopM@skE-nX|^Dv?A@1GdO2lBI{&73_>auxvj z=pq8lzRv_Mp7cRs2!W|ZtP1y{CvZaCAFd?jQ0aJklgyarxAY-%40yZ<D8xrSqx!$D^d09x%re6VYHK4l^y28_z<`rCQZDl>TndyYBvZQ!^ z;uEP)yD{RG(MOGkg=w?KkdZxJjv_w^a<36DMAK_USDaF4i{c-mT@Y?giDLzIe`M6< zw+Yx#J&*sJ@&);$y*v*31ld^-h|G*y_oj(T;lV0U)`>$Z#fY8#)5qZ07K6y*!P zM;rHs(YfY}K(4G@7Xk}jU!52kFJVXzUtij(nyinQSW0K{Ry=?e38&Q=TGdoXEZ`K?Y1SA{R1v&~dR_j4T-f1Xv16XrDu{Ri z?>3d_wm%KS`(upeK1xu<32t!iv;|GcuNFwwN?E)&Q_h^v zV zF_KXz2ZKuI$8)Bj;sE!}*kmd8p?~&dM@g#oe<=TvxY864usFle8uhUZRsPu4(!<>z z{0^_y^ueOQ{*$py;SYRC*~xB^6LVy;Mx^Gn9uP<`cG;2|zC-$1(PVT0fO2{=67$#< z)C@_KH|!U$5F4zZ2ud+3P8d!ith4 zqNXIvLa3{6AG*;G+P-HE6G)k^cX@PSyW>8Z@0=7y?-$%8<|sYUBprA?$8(uXR)C0d z_s)axI8vyCt_#EhM#dF-koF{%G6S`&jkZDTaxJ#y2wN$ON5IFe*mpt2s!m%r+K5j^ zE!Z8%=K1eKibgi!SiIn>!Q9v}L!bY>PEqX2?Ok;nsYN5BXoz|IJgkbMlhST~L_GUi zcER1qph{>gV50-Ai-G*?uzsB{2;73^!ws+A;x2L$FKE?1jf4090bW3%zlnQ=|6(bPzya*SMt%SA!A6Fq@FiQOqLI5^Ct0*pH z=uU=>=pS%2vE}yhv|!1GV)miKRQf?P62RX}E))bzi@~FFW&!|T(*y-bq`~l|7r%y6 zaOcmzIRCJzL$K%=^oKj1NmcxiLlhv+w#L$9IQ+}wlNU$lB({-1>nVOkK^nlHUt9vC z!+J7CAkyw)T zqUjZVJNF|R)i|pE5G$}FQde-I;JPcQsc&>Qs zLK|9DO-T+A=LXx3WZo1Af&ivhw7xlsQ_PWEv zHQXms4-S)Yc&tLRWD(7}IZpv+YM(xR#!H{K&!#YBqI|@EtAJqm-w(d$F%Een@O@UN z=TD5}@a2-?EWY6#N~RxlI(#necVX`ig4*L0)BWVC-w#ALWo?5#4y2M}>!kG!@fvLd z1IPxUKKnjLQ#Nw(dciS{$Uf>7={Q}njnlGO&hvYEtu+6fpCWByQ!nSEbH8*6g>=JR2B1ks41X3)ZH7p==GbVfF*LWho zVC~q@3;502B(>DI0RqxOAH>#AKC7xea!kkm@ai5Oj_{lF`6*7s)moxP@|vE?I-t=o zC}8kPzM9+sB5+;}cSk}4h6uE6tC-^sTya6N5E2|$6Si9YhyykF#|gX~eA4%aLJ2}+ zGzciK*%Er}DcBlx#;u|hj10*zCw^ciz)%xR0792I-4$CJ1aZ{^-FCX*c-WiAL3oK$ zXxW$|<~X6vOHztu01Hw_%XwZKYs?ryjE&h%O{qaIOd~x<*ukl5ta=qnUp70%fEH&z z15#jp2u%uoS#WLk$6p`4Jd2NM=$~8c^wZxQ9d=kF^$##e{q_gm@eTcpt*t_^!>t9C zzHHeY{2z}y8&b^U0Y(-4V-4%!%Y%!HokkTJ{jVEIvCBd1#t_$g_!<#&2O0oK*J6Kc zr-4{%Xbdb>V%-LsVc|%(01fA{ZSB(p7&AJsyPh*K^49hw&{O=~)WdYs*6Qu5`gPQ6 z+g9cC;WixZK1NPkLqp>3kcqcNCU$^qd^u3j^#>i9f*BnM`kti<^(EWD28W(4BeDya zO6x25nQ{6e!-pa6R2RVPc4M2c**%+Ota1O@m`1*ev zS%)ng^Ca*Xou2<8V6fZEKR3W^=-BW(6T!n|Et*t)efU zz~1oZTOK~ly73q^{N_7WqH4n%>{*A_q+xa7SwilpDV$YXk?@6$7z>AL$`rU|L^)%y zg<=xFPG-X;$!KVv;lG?J`o$V zQm(R$GJt8E{Rk!I0(}IZVE7w+d?JBxwr3dB1~~XAK|O#oOfA^3x~Wa&Qq!Ch=I9q- zEZ{YN>DjmK0h$t-sYu36W$3&Gfb=lK(AA92U~Gj90h|wpOm(P6sBkFUKA98y4}?6M z>Kr<~kZGiod3)_+o!aw#3c+H#8w7g>Fal(d+XV{sI|NX#7gSek5fE_Lt|XFXKf$j9 zhdgYIB&hX$-CW2dfsa9B41jDK>wzO<2#R(Qn$W@rtU@_0VZt-xLyd4CB0jycb+J-R zqxG(UCAC>?U$27wBNwa$+2V&o);5ZAM<8+2WBJlB>1ccluorkRdYe24h~W6MTX zddSj0Tjgq6JuBz1q3_u;$ELrH^U9atvg zJmVxm3}pZGs}EQ=9ules$eR@P`i0jJeG%gGiRKJ9FpruR4#VRP ziPzL~NeopsoGvtOOR0+w?h<{Bkz^(&r83FPxCqRVa~i2O+XbFn>#tC|qeIPA^=`f7 z=&**4BHtThtAGT*r39c}c|&9bR}DrV!c^O1#RAd*qdve?m00VZnkBEL{;ij|vamgP ze^a8nOsv-!+~G&8iHq``mWeO}$26Txp20-pY{a9gt1>zcty5*Q&_VB=xwWv`qM?{0k4 zz>Q&5!vnAkAIuO4#HZd&qo`yENcy;H?8voYUh5CPkn$@iq(8z! z%1Nl0{`3p#!h&M@!!M+iiNN$%0IP$puujLT0Dv>8SjqiqgvTHO>AzHl^qc-_#KVwST4=ti3tm@5@r2jU^)wsVsFZsWbr1`bVXm&#O93hCnOU8ymiX`Tg ziIr-IsscuC0=W7b)yzZ>xafK=N z2h_#G9I~Y?F^-qF!s0{rc!Z*R_O}~XJuw@;g=Q2IVpJA&cN(ImWl2AH&*Zfd(vfYG zN2_)3RBW)3qLGvdE~s$S#nY?FxkpGJGEgIe&yg_LL?;8_g)-H(`a{#DJ(i6~GxEaX zTu9IyZ3e6ST>tdbeYGW`7(bAl`#V2yM^Tx3~9niH{Z2+HK0=)i0(LV1P%=#5utG1knEe zv(5^VA+sH-wv9}*&a$n=zEra}#-f=+zwqNKsAfAIIr3htA=S{90T_uy0-v^CXzkSm z!2Bh6s}-Sy{@9_{5}AQd-GR)FH|3@TyW*2)QH`i!*Do@_TEiU&6WqK7a#JsIh~9=L z6XF5E({?2UdSHFmWgQ~7Dd~@tjeM`sq*8;C&T&raB3V6%Et8^yJX#?l(+EWd$Mvla z>|9TN<1jvH@)_G&6c&p+6MN0rlW9T8ZxovDa1=@OM2~>0Wsi(nHg~@(m~on6JkW9N zCke%u#+g(QMq)!D^PuX;>$1z{*ABAF3>Yp0TU#G>O3>{sbsGQ3 zdk%HzbKvgx;1~+G2Z zzNFT+0g*&uZeNFipwupoX}x%i;!>M`ua98~NHdt|HT11fDkAV1UWt>?am*PI>1W~+oa{(^V3YG$Q0V^y=s09K_K z!1Yi4F{ZR{LFir;;qu>UU8d1^4IB#e`YYF7}#4nL?vt;A=lIbKPo3_ocL3bsb3wq`{&$iZ_8MPq56 zh6?GZM!zW3wZ}@Oupba$JE$X&q+gUNQshC9~F45r56 zsBN$zPfhoL$3#?<; zuYD!g1XBm~k^#J8i9R%L-Bs*051MboCmK1}WX@Z&3DhQ-9zeIH zgwTm16I0BPhv|CF$)Q`6Mveh*Dtpe)qlz(4K9-Wn{|fk}bJ|%eY&gcH_!-)?kNvIQ zLLPNVHW`jgyj91Bm52|v&aDe*8>gmAwXGRQyQCo6x&=CBv=y`mTm$Tsf>~@z0sqK< zD{L#Z2Y-6k@WM8-}@Ir;4eH&}yY8 zyA|=rE|YK2q*+z5b*rcrbzlR=_8isu`RRGD*Fn^}?{?C4EWn1qV&iUmMQzNRUMbi< z$z)22n#4E6i;+OddbIELH-or=3pn(tEoxM+0@I^9Bv@LrogBhxcOjAp!KozijYtv3 zJ#7t>SY|;N99)4L9b@k3cK;O~xxZE!LG#C`R2w&WFrP=3PMjRLpp0+MQQ?+p@SXKN0*omFM=BcwBQ;{k%##o+0PDhzixO1GOf z+7ekEG0hz{*X@pq$gVAM4@Tbgws~`eDTkRDM#6z!jHFR&WKL7{v4ovmB>EJHaBO3GFW!VSWs zy~=HRq+l^=DE!}VkE<&j{ul^+C;1OLI4|K2eOyy~vV-@%ed;lCEohp(vTw^!2~ z5@LDNp{8`D3{`I=AIn@(3RixDt7Ghf zb5(+~q%)yCGl~ zbV8)A3K;XkRm< z_9){o7Kl%lv>JaO?aJ|eI2Hp_PpyACs<)&7#(ZyQdI*^#!iPbQHdEHDDDmX;(=Sdk zPOjmnu7H4dI-c8KuVznv(ic9uGv)2J|8{vk;*53Q&_WGgOUEn}@8qE%j9oHX7^b_) z+zrVjcFkw8gNgg=Fl2*Kw%w5JWD|0L{X5yOzjDK8w6tS#IrMwC*Kt)=yh`;*c3nzX zGb`^izEaJkJuOXoQa(bDNnKzwyT^!6JgV&jV=Fv`Ci(l`!0vh5*S}yLF6jpeFfvF; z{^B(9ZCH`O{{ zHcn8yg;`$VGT|($J}ha($TBL3x##+DPSVc))g>5n&|qg$eeN*vGNiPcHF6tpXB_*( zmK)cl#g9U@Je6M&-7@el$~`|iJUagTNW(6bEw{v6n$y|np`$X%piGO@+`BRK8;_C*`h`ux!W&p?*e*YfZ@W>< z<&h5tU4y3bE?4&WwdL3&?D`!rv_KF@ZOToniFa?|uS{z?dsrrpr~W4L7xPk5fU?-Z z!p1QR*xx<7e_p)*Zn2ucj%t(CDiqFOs)?P88X1wf*275rRLYUX<4*Vy)EoNRk@;`~ zA+Q!bGIxT!$Z& z>9}^o1!$=Rz-}QU1rRiBi-3c1k;yM?zopHv=g>v1?o=JMAv4?xt})S8=0RM=8RQ4n zxB|K19lhP|7i?<*zVG5vY^@g3+`yC?Y2RJ1+||0APhHCUgh?-2Njkr7o;BqA>J-v4 zLp(mx^Kc1vr1ij5t_5EED~>{ z0r%Q+da6UtvSm2hvC=_Dz1>@!*47U8ZT35VdxM6AthX(0!)C!}x1Sl5yWCzM?rT8i zM_0eh1iFO8o$7UF8Du}e=fx^!WW}$aG!LuC{ne0KDOy;F#*`|cp!eN^-tpMV0{|H{`ZIa9c zOaQ)?pgR&W2n$)ybH`G=qtb$Z?~40YYfsjZ_d4JNJJdv7*;ebAlU^uVZNG@MGh>n0 z4lTzn|8eX??(ic|Ci`tjjbKIW2R(^I(yk=tC?$5i8%!n$+qjQ6WHQHjanhK>Y?`Zw zl@|!l3N+Eqw%dCebkhOvUl7rLemJcDf2CP#d~B^5=L7TkdWIuUd7&p{c{CD9KQzXl z$&lm=o0|=xuv`H!4n6G>`CeDv@SBH29COws$&VXON==#lAO8 z4idb%k}a4_=hzinVFn+Vncp=hXi`Dc1=B|XZ#oS^qoHy}t zdAmn!&lrSTREs%bP&WK-cRfL_#e=(gGuf-?@av1~ijrkaP?s$ES`0{odo|}P&?yhJmt2xZI!N8&&%zt* z*V|;@zdShj!!o?cBc>`3;tPVk-ZK@HdNFgv9Acb$E{84SG!I45450k&g^X+UDA$WtsN1xjdoL4pnwo6#iLJQZK7U zWAOM*${iZil%MMU>4{F*Td~br;ZQr!^}tzBVFri*E2CEr?3iN zeR^bSEba@0%uez)BMetgog(v>L!t2&dLF8Xn@e$YywujP1)YKn^X4 z^ujeDHY?0kX)Y<198ZFpMl7TrSmOkxum(EMO7}>!362y%zCb!hzfR4WocQlGht7>$ za3%o4sp)uaN^+v|Wy}@dfGWUjl5|!VP4r=KXlwlE_qSs&ZE9|GZipdCuTyK*8O!4o zYNUF_K2NS*qts{BC!$dgQj76aHa@<*qdt z80iKcd`fv9hV5|K&^v24{BgpJS6|jE?a)jz8klQKBR+Q}W z4`)YRnM$^5=r|s7cdZ-J=*3~%|GN0N*WmX5H7Lsm;6FR8FwJ>ut{MC!Cw!9ALB9N2 zyr7gVk3i9HHbs#YuJ|%XaD31`$9>}SvbZy+I^V;0hW?9R{35B6eR`5gsNOL7rdXYz zrd3zut*uqVY*e>WRn1CkxaclDr*}S}@Gar$UQ1$;EK!?r1K}i3!MgCPq<95^&1ff=RZns)<9`(0kIO**}%BG|G}TB$C*L0 zyrmQ{^o+lo8t$`!gEYw3K-Q_u73+h~`a@Gq^SST4U6pc%#so(kBO!%}r6jUp(g`l@ zKhkv1yL8Z?UND`6;&A4FoMU z4RjLOzs`CPY5$Rx0NObGEt~*vqZf}0O82^zRPCQD)OH?0R}4sd2l{@P(k|uz*8uR@ z?_icLVa)d4wxjtF)pXkDizo-n){|#sqNGw8B2E9R?fV(x>p?t*r0{& z;%fV-?%I#wvfYJxsc_s~nEjD_#q=x5cQ#6p;q+~670l>w_P$P`b1-}qsk#kplj0} z)mZfr-LNp~yosF;aPK|hLV5{e!jBKX58e!!BTTl*s`{GlSl}!HkkN*(?8|9%03-MX za_8!xO_>ESgkCOyR!dJYR680{4TKuUE?1^J*j+$1N z(9~c!Cc4!q7Xw90I^djICQ^oOkFLQ-@uZr~43V>!Z&BiQiqQDJfNI5ePU73yx{+J$ zA8^;C&o4z~Qg*P<(bE2oMr#d&7r(sE{3|K{0ABqQ@V}q_{tqqpQ__Zz83riCNi9g5 z#=B*n>1ELJWXpYof_~6KP@uIprl>P9kIXfHu<))g9Y+@=j*YJDqTwf@Pu`8@B!El=NgkH<;GANR z=Uv%2bJ^hSN_~W;9)M^Zx+inb*ePn1NShUL0H1w<%oJUyPT2(O^1ZQ9Yvux8X-XO& zcoXH0S8x*SN8iJ#t6IMX3(x%#1FwSHf6Z(faN>;-w~X1EcQW+wrU2AH*ff{BlDs4LA9|ytlh}PJ zta}r+mVdy-ur5J7pc$(1pc`oIpzPQiTmB)vY2AUk+KVUDL!cEF?aI-V$61vv#vwH+ z;twH99SuERrrq{8`5mXxtD_Uy5!%x>^qZ6~A9{Grs{eKna+nS-PjEWkMRwe1jNeXloW~A(?*EM8g}f|jzj`C2QhS_3X~gSdJ7u+*qxJ2; zH9kCmUSp{3D{HU?q+@_>CnIBz?IOY8{{L;FX2*Di#;~B7ZY0gjg=of97up~P|XL(1u+dL8c-={DE2?v5X1PE0WRPJ&s{kQ}uaTB+-NRwbY zg$nWR%(XjvIdRGd5&{8@?cLdHc6N5ozMFl#^8I|6*HF?As@U$F*&nv?CsrW#WQsSB zi!YzdzIyLdkj{Y)JzSEgSzVNP3nukUVO;yJO@zB*o2%Q}6L^>?DN;oxkz4FcQ$wk! zdw8aB3J(>ApahOAe5~tqgHIF}mJqvi!9J=Uy+2ZG z=ok0SC{=a&c)kyZ54xMv=Y;I%U~?w>#pQ=nnB${c1T;J|Nj?`^d-LvIg;aK75ZBYs#RhT${GZGc0t!gd*cJe ze@BLSTwAGf*f9^7wws7HJEghgButWUM_^!RS?ZpZCx~$;s_XD-7YP;Wj_Q;vR0%%V zy17J%gpshG{f@~n$Sh#4O)(czIGpi5vK-^Up2z@?y@>C-G#}yzPY&{{E~gz~?7)8X zB+TexYFR=}4M2jd{KvJ>$T2J&Qi|v*UfVkgVwNa6+caIZDn(3lL&%?(pCm1!>)A z`d~9-T?(3tb=gwA25Jb)CkDwaFkD)oTDKriRZ)(KS}lcvze8ap?R=`BN=X#Pf~j-I zZDYJMWa_0 z6t6eUI3n>hai`f=Ted={M-dlWzO~3cFr!*msU%Ms$XS2IKq#E=12!8v*LU@L(GdW$ zJfj|~5d<{k!^lZzxTbNF6*3(ST&q-g^)i)!{yegb%5rLk)tIF){c4MB%Z-P5FNREpLf+`*PL+^&{)0{ zevx)O==@e`8xtQ{9eLr+D$ zN?4GxCH$qTwjtbv;a-}0%$%*Vhh{(}mFQ{>6tL){O`_gZAY=k_PhSkH`KYp-9E-C$ z%p%IRqjPoEkEI;R;htu-5|jiUh@*+y8O^AV26G)bUL!uPc5Qvnfe((Gy&~x7;TmqQ zxirqR-)cW$i}1FeJrn{G%9v>?pORS)HBqDhstwCN0KI!nZg|J;OZ12sumUk`sngFu zI_Y30uDr$HH(!*$RIOS%$|#|-dTNr|g6n!ZWl{r&w!8_>ubT?QZ@#CZ*HD`HyNzi`7R3OJPs;0$ z*0~c0}+7jA70PKr-Hl6=5lfV>jSY)gSSt~R^Q#%0-{ZJW$pG%MKr zsprDUQYxvaF_2IiApt40DW3rl+HC{DF5}i@oTnM;9bb|sYqES@ui%-cS9}NhB~~L4 zEd2DsN1M$EaO1&|Hhl2_FDLlR`TJhUaV;VeepNPHq9BwoJwjwIyTvW^tGp=@JE&JE zr)w%$_M3GH?djU{zT0pmGR(m8R(@WZtwFlY5kGthAM>AH7{#-s>rZSh;eY~dYn%*M zHm_RHNPI|M7cx8YLYB!O6)Y1r!lyk!qpMxzPudU@jG1q}b&vmbw+ z_~&w6%{FUb@oVF?VclnDZMOsp68l_uCE#);^lDiG?OsDrnVI>{bc?AVcLje22e@c4 zfr&<)A~JTfn`X1yrkKdjX(j>>FXq6R6f*=uca8WYp_op>i>h)Bp0wI%g!D7Wsu9cN zymwtcYL}K?Rv>S@h8~{Ib;S>o*O6kgd-B3UZ8*qVSrI@-CaV z4{fo9lfC|((hW1a0O*SkY#87-9jG(`-J?MhHrZk8BTdnsRR{suO?nD5u9~On_pT|8 z*A9Z)mM#8eSb3-|(&mNFx~iP@+-bNbTX3!ihqp6{v!9zz&5Lo+Uzc`-_a^R~Ah+yF zJ|wUfF?4QIgxRu)wlu;e_5(is0ySfN3rW=}0&b9fvtKkJDwXh$>Wga*(U=Q`@NNiF zBdY_4!fewP%j$j&$77+MPUHi*MR9-&19=&YE7vZXdP`BYN;pL;I>;NMOz>1>-{HKV z9)&P*Rj(_?_^$T#;j)`rlkhdVDu$H#sZR*N{BKQ2bSM$3D*r)_r#m2R66V2hGF~$Z zWr-&X6%nho>K@37n)Ui99K}f?oQ54}Z%XBw?snm{v{}xRJ&|SFd;Ug8_Bgw%Z*Gar zg^`F;!Rmppv9gN-Q)4Q-Ygu|h=O~zp-!Zpbqqv!Zwy-D_*Pl`$1xpc)0@Zuw)LNA& z&16O_N%>|>LInjUM2poD@HqfKgeE9NJy?z-hgtmEF1ji$e!hlx>BR$JP=0w7d z^fDvLwA%estLdQOLEP?G9H9EL;9@#NhYjB!RyMc2Vd3FP&?pH`9HEh2_^oWjDO397 zdp6Ss3J=5gyJq#$`!8Q!d<^!<=#%+;$6R)^+ja`)LT(fuIwR?BC&@doN8)i_0Et$> zwNLo6NSCv#N)d~=op$5zW*bQbCup;~xh+m&Q-ktR?3>$+I?9JG@^g`nrCEP*Guq3J zV-d@TJ50E8;1l=Q^d>W6yo2XS#CHeJ9<&rY9=~##bNg!`f8O_D9Qhucc`+jr&N#96 zD_v=5#>ajBAAlUsW|(3BJ&1^sU4H%H`xhTwyf0BI0z<|qPz7A`{UUO%wOcq3Y6(vY z&x?w@huJC^3p6h_IW^%84NP=)W6?peH)vk?2ReHse5JL^#QhjR`g_LbshAZ~! za(EpXJAf@W!>j5s5dkC{xh;PF!3R@_5ObvUc|e4zU&Kq(FK1C$=Fk9KSP~(U=Hw@~ zn6`AkknZngy@w$&C~5IH)A)bUjXyHs?-%eN=LYjHyN2)88TfNX+3krp6O`;#7CbGN z-J?9o%|4O8vdM_L1yhXg4%^#PF~1^%9$>3)N(M^Bc7^(kuKImQ^G@l30f&@yVHdxv zN|-fN3aHBa^9^8Z-CC5d+YK_A88DL+jtyf1LUj%(sNnc~5-WU-k9db=mK7Npiy@=h z1+=VDXh_0{JQrT|SH$f4nY$k3Hh^$|A7KnzR^^IKxC?)bD;&XDHpP*In%%mc8D$d` zAuc#BAQm1ow^ZQmqnhi;c_%v2JbjFc18D!*02&68kUJt$aFkGl#vRzk6fVsuuV-Y9 zdM4%$1xU>iok*|mqTxib88dmpzy4?ir5K8W<3G7HGcm6(zk2^O2owgEm0!5i+Rf-F zok^Q{40+sdArTiY;9{}NdxsH53Vi(~aNrg`q2AuVImur0-@NWsz$wbkyk>O{@fV7N zlkOsD&ReY!FLaM1G@jJ{(%5TL^wpnxjXqH39awDKwKq_4bS|bdY%bH-xZj&Zu{{MI95MFBC|P9clT*o0&2o$Zr^^@!cSIdm}LVHf%JV7W==$LIK>e7LHFUn189 zl%>(GK&PMGHrs9A^>G!;a$E-bkyEBnF<6H!;z9pFQV7(o?(q;(!EE>dGHdwtG#GJi z^85`8y_`(fvr;jsMSyGZ9P1Z zMAU{)dM?VXMA%dn_w?ETupSv1-W#7tJ5|kM=s*GmHvN|Jh0C`K3JFR{; zq@QU^Y2V0b;gb*8(k_cAfOvnSCJxY=$Ej>IQk?5VHdqVOpt`DEkc^Z|@lR|@s)f{B&@_3XNVtE*Rlkjll9xS*uo^(inZ zr{u9&&`tN4oakfS_CTqAuNFIO3zvq19ng;W%?Rr{YK4=N_JC+AjEq#mofSvQI{V+M zbYTW;V>T=(S}F9MF3Yrf*i>uG#R;M=vG&<1()E z&ErS|RsFm4!QY^f3r0-*3zTFvc|e(JilT8kc)fbXSDYAjoca{b*cPN9gZHf}Zlgnv z1#e!Kfw@nx)~sLThM98b0CLv zVf#8BsGc#`4f^YU;^D+skw+>LZQ|v-%{_(50WBRUD#~BbK1UZKbcADbh1J8;QAj>k zBbn^HgoL~m!iv4Us$>vt_0+;sABfW(57;mN2&uC}m-^+B{b#zOWyaCcx{cYQz(}+h z&C=1ia)eAJJUu5hpR06A==(TwhbQ4u3@1SWoYN@Q`tkH7qZQGAP`5v;{}zP9uC7y_ zgr)5-X0EZ83|S2?URCK`Qi3NnC(_^XIKj=(H_cs|Z5oQ9!fy+7a@gi$6Gvruws!z5 zt!r)~w89}H5kF4K=^n;mCMcjX3cwqV-7H;{yGly_5Ys0*RscfoLocc|E#PR^UBC3! zC^1jfDm9>w&!_J}k|h|;=a?gE@Ys@wk}SEbR=K6gKk|rw9zM*zn}3ijYWm@mZ>C?) zzJT8_oFTKJckuMJnIr3bv;JjGw=TutQQ_le09{Gwch&PvWsDQa2+;VtTYK886>1_D zH_XdYc(q;znga+Uz2&jPF^?#J)qqpsx_DX>`r&l?)pYQ5icCeCh{d8x?ue2dA?ts_ zu!syM@6D*2a-|rf7)EKx?`YyX&JT?Y@T`;QW{r!!a)Kci#Sz0H(aS$Q7*8nkBB;OT9=+AW8|3AaWk!a~amj!F!W7 z8C{f=eD!{h0Z1@ZE<#7AmNJJ=nx*tVY2)%G;L?JIPEYoV34Q6@$q2gLzZ@bXO0@t9Ti$_K%1OD#|z#d1$Cy@fjy_oNY(j zO2v&FJwWzLs-2WXk7FopQEs=mh3Sb4{EhB&Zv?cFIfMxnQfUo?BJ+&%C$t&Tgu2o+ zb!v#*6CvQ7syy)VDXWd$5D@6sO=%fS5K8vXxGom}B7||vm}tVJ3LI42FF8NnoaLaU zxYF|{17)sl&Dmy#J0$@m*#$gs`);$G{iDae>9O{^T56A5H`{m>&!Rutc6S8_q`-wM z|Dsb!r?%5XX@JViC)pgiX!AVA7d8V0%@4m)nHW42 z;QewUxg-xa5ykPsw?F>$U#~&k%vaJ^bi3mSVgB>$;Pc1M=T3=t=(Zw_xXU2XW zpiguXu=~ypkmC2^`|QQ=;;$=IPRy*_H zLWfO0n;p|vfeH0=0>cf#3mxHqc>UAa@6aWAK2mqKQL`mc-L8{0P z006i$000mG003}#G-@w!a&L5RV{dFOaCx;FYjYYm@caD=m41Pm-o$C9?a;b27+)Mx z3>0uuPm(#rAD`T}iE0%UWy&+a+-pdvU?x9-h6_Ol%8y`YD6|ILpMx z?E4i5JPijHJVU71Y<$lbQN}iL!fr(*5*~cN+b6p?6;>#+ZMi-|@uft3X)znRc1TJR9vWNCeM-GLF~;M4dkCvZqf@pIBCp?}1g)W9K{p zO#%@f*aZjoa6h#$Vm3(}GS}Vgvwb35m-*o?PBONSd@o)LH}ErbB#7dxG!FKea3zK% zDbW)@K2uaM9WVn=*6!-ZXCgud-O|`v;FuzX4(_Y9=1>7B;y|=wM-0>_&Xj=^a z1u?`AiIukN=BmQrArm}F_~B_6QSlp*y5Nk$pBf84b_cLd^n(H$vMh$c=&C=75PoDa zes?U(df9)~cdz*FIfea?&Tskv70_QLr7C5CX3ufcYobkX=CM~N!oAqz>Kn8TqLVs>So#EVk2QBNWKh>YSWsR># zMrzcurv1V1{YxWZ2hJJw=VsC#uy7VHCKq}n1TGWz%DFms$oFx}dNmmu_#K#O;mrDr ziLuiyNV)+o z;5WboxXib(SiOCg<^(QIPU7S&PFQim8j+@4LNi8C_IUos*o82g(XvBD-NEGQst>=X zau9Vtft3&y%#-7kSZgRMJ&3~)wyD$_^hcu{T}Pyc+@24e(M1ik0T8Yu2%GBxn7W4G zBf|W}_2OdkdhDtdQ^QIDi(;k*2>^m2IFbvs22T(U6Pe8FTtkGta7NP_WD5`(!!-~` z5Uj^nL+5o3_#T1Ip95#wsLT$*7sI+0)(8L_IsK~|Yz{E=K_53|mhM-ca)wgy5?0py z`n*PWYvwRNf|3?cM#F*Ah!hy4kuz>Y5e0L>92U)tJT;M-*aX7M{5UGkhMY)~xgP&M zp1f|BvCXl)jxX047~z%k5UWI~I)Dm##k~(K-9haLm3cY2 znKERj5}8{|T9ttlMtCxT=#2LWGKHBFQW}wJl(g41)$APT05rmF{RrYMP4dD|ivFP8 zl|pcSEU*cMiiKvJFvpSQT4NaJC@K#$;oQ+gC)bPVH7#k_8x~ZFe4;8^S2#V^4tTg) zbM~>z4m~!GBhitSwyM!8Xm>o=3q+NF*#Y!oNxQ?&u+U2OqNS*8y;GDRQMaX=wr$(0 zv~AnA?MmCWZQHhO+h%3w=^m%g`0wo)@emL3xc1z!=3X(s`U5UeNgKgTUM?t|WzSKH zhku3AMNrd)!tw^GkY&m|5pM~9$a3k@V5$4-Wap}zVd3p+c!tOw_XbhwLkh2v9heN&3 z`HMGC;x1=AW`^kY@G(RHp@eXOVC?>?m|Q>0fSxAW+mubcTI|XY3~(!s+zqrA*En>> z(6G^#G@#T|6d@&SjNT|ZcHoYJDV0luWIE3WG?N?@OC*7KMc`GABT$UFpx?3}O|yus z&k`q+^qG*G;aTR`pN({Zxp$rH8DsMxV7owE@3VfyK!de0Og7uKlFeY7RO0JRefhjK z){oq>dlUhood5ymxVLKdK}}Sf8s{lq#GqCv@hFAEXd^=@yYGDtz&}FxxkFs>AmCpx z9w`<^2mX*0q_Gt%&S?k!!^i&O;}-8hV`iwSC_JL}*Zbh!s^44**AMZMKY4E=NgXr+0 z&y?Xn=te1E!t%vqv2su-OIz20e`wU%yMV#AHUrZ8UGTC-^KO(l{V&T$8Xu|CuWO} z&*^caji59?+UP^=fwJi|DDL9Anz*nGEPu4RLuFB%GFnl0z&o1+=&WJ9`3}sB|I_qA~6-(&=Aq zplpM5jKE(gk+~c$ZL^!i88M^o|IH<~o)lv)H=)(4p{W;_fAMl?_%QIJ7Xc{eT1Llc z`tclvN)NPy#_S9F%qIMY4S3Xf`ZM}(IW;REZkZ2W+5{tPS>;FS0V59&LB2c8U&V=Q zG2^rA&AwPC`YqI~B_yLnzYW^CAJSsmiUXb%2aRFu@I*~5mDJ~B?qz}^Us`*`UCHb1 zo-1Odj)@@9*DPf)ylD_kU!73owJ2W;fAwjJRBW;48tv&U6p+p|lB0j;AD65}cVOYl z2k6W)koj>oB+4lXHh9IaXI)xtj@_r=tI&6n7_8o19ikRVjvdY>UM!!&(hUX`HoDAR zuSMP1HYv=dP0G_MuvFEW`!iH?Qt#0KuF}M$M46^jGg6} z=*X!|TrLUC?damvEo5jL!j3RI^l^emg4Topk}ioSm4#=DNE_Bjpx|Z10-Go~h-I2# zOUl6rXRZFBzzq1$P5kTml^&qHyrdBCHA?cUK|mIg@qVH#96Fw#zL#E`0n$r4MIe)) zM~=OT&ZWEiADR{LUetT)McW##`gF?j_kDC{0s&T3Wdrw@Dg+j>$^^A+N3J9ZB(h5b z=UpBTHS(`Eb*{<;G$?YDULYx)bql0l)90N?2&=?`m8EfZaThvTh&q}v9|^O6uzm8q{AfE~SwEfzPyZHp)pRUvpM9%v z^6Bcy-%%$~H+Seo3I8qBHj z&n;o@Ul6P8YrZel1{ za1^6SZy5QkCmn##L{QV0Cnee)zAK?*4@yR>m!|PWUB|tf;>U|VkZO(aK{zJSHS7WZ zs}(Lo8&FcpCM0uEz*_0NcisDoz|b*-<|v_1$w4bKUcY2}k+cvK5wj5>sP2=D8U^v$ zo9_H4@4F0V_JiKrfx^|E!%_4ypQAuvc@(5+pkkDn9}mh52$>kUTra|m<>;_*9mLvK zGoULYpljp@5iq8!Lg1e#wY-CS{RFVPaeK=17SayONIUk7eXgdfKsO$4@-qb|9W?4u zt!NpPOsC(&CQ=$0plWQH=;W?(bGc(~^#HGXu7X?h>5q-}AFwrVzMb(e`plh!!p_MZP7rMUk>OsHt&tDi>m=Wm7~1bf}6!1 zRRPqUBH{d>Z2yy2USjl5T&Nx2CKD6@0OFVE_S^ge*n6;8(mPq0{Zifj3+q&^Ha_!z z@P{;QBxN9QIP%4K{4|1w@#?Y=pls<)CE>W%p}9Rg%(@3flwq`?tl>Pn*1J>et?S!d zZW`5Cgz#D~``?Y7@aHnFmn+q36_{N5FJrXWO&?@f3{lL)ANjyvT;e;JE8*QIAKIqt zpGKZs_ErK?WH6s#AJT#ahZ!2d?tX$J0U$nd*RbYWh@c2yf6S^?d5{-iV&Xo(`gWJO1hQ!98)d~MkI-V|3V#HQLR+PV2tOkkU;PRYxJ6I%*X! zOfLU}?!Zm!#4=z2c%JX?XNy}2 zy2BPXG&xzp$1DjEz%;jq&!C(|heXq7Wj4_UPMlyeN*2YmU4y+-vhM|jZ6E_IOv{_9 zXS@&<1IZY8V{k4wQKgWeE{wvPFX2^x#&NRlY%~fW;Gyf5>(OUY3Cp$N5j@Ky;_xmp!0~LxGD9C`ZR?L$@4_;}W z=fh)12w*ymhg{cUXuaFij0hye)B9sKS1Go)4ewt%Q`fGzvc)#jlLBmD#H2c1oEYQG zMwmN~t167L&u~1}oFPP*R!k6ue9D8I_C&TZ{t+RVbcko?ffe?PB0fVI$1pk2Kd7wh zd0haapo5G~rd;;ZpAybe3NeJwRC5tEk=WvTQ&XTJlV1D~ug1623QMEGT8MKnk_l3z zUh_n%9XL*gvv)2LG9-qM*L#$fF%-x*!93xfGesyOM7`M_?NJqVgx*exzKD`9ru;4M z?j)9Suh(lXs25xBniX*Rz^78cSGK9@cKT0W0{HYh!by93lh6;##mM# zDi6HFy8~NFJ0KQk`1eON7TPL!UOAkqU&zAjVLDmC85SA`vXYK=0{O$GxPv0OoNl*l zWy`lCG}IJIO8!ot9|X)uP&s&*I5HWGAEOVEH8J6|;>di}t9O7j6ITEz=6Bfb^^ltG zBDzy$^?}V{bnmzcx)we`#sWOY_wd(YN>OhxCm>fJK7O>N;Fy)+m!A(_H2~7h&i8J= zKXeVq%R)u0vwUxd%MC2{#p=aDbZg(i^Vzsn-s9<2HDIIDHBuWkek^yb{Zo(@$t37H z@>D)%CZ&v8{}O&wn$KfGl&|8E@!07%PFkocAE|9e6IKK*YUcQ$cycB239 z7a;)t3+Pv4H}1O+0svt7tM>mF==XnKG&Hd_GPf~sv|{{K{$e#PyF~^R-`m={0Vyb* zjhM_f6pKE60?v|Vq&!9u+k=*!EcS+M8`OhdC-3oM63Xl`A^Oo%@6$)KTq83|_ka`+ zEXtX5VV3bWd5S?ng(=UQTPiAF%8a;!)~zw!*&(w`a6EjpQaK( z&rnRQy)#eIAy`B+^}QepB7mY-3tw+vmY=6*XEfh8Pdr}^-fwokcSPCPQ*yoCLP$>- zUc^Dg0ypk)0>}Qbbqq*CF{%2Pm2|8m0NcmgVPN=2&xYqXvqJ_O@*u;;pg z_5u9nvzJB|p$i z$|`df(mWUMiRprO6jj(N?qpyRt3z2TsIkJFV{)gGiQZPL2?&^8QPcp~axOc8k7HxP zn|Y9EFV!o`9y0Nh0KX=6%zA@{3V{3t3n4=ijs`8z;Vkha$6!M$Vj4r-lRNjC25pV| zDX4aLaCPJ(C8)23Fm>PEGR@hQy5-D?xrzD((Ucwr1-le8;XHb79YUkWDqkoffCVF%3N!+mMv%oO zG9>{jK+wSnn02eaA?DlxCkkxc^JD`g<(ROF^Zew`JLjn#A~m5Q3oP>t)SYKVmE9k< zObT=$OW!yk>53y1zfu37YHSkn%>nD8R0$x_09?x=ZdpLVYIaT&hD2Fs?33bHmle7o zFRelxTi7L6GXXRPv`wl#i`xPm#y@qtrq3%uj5p7hHh33;NA@S-g)BJz?GCkRKzUk=>lp3Bg2)0{q#bU|ypdlSK?xlbO^L0mOuNC(y}#jVC?HBUad$ zt(d1owb|rRMZBiprq8tg)pov8Oh8vzyM#S)5nbC#BaXP+dZCmY-w%1Auv}wTh|z^M;)M zws3#Kr};f#B;a|1e4XEl{<-`hnQuOu-fMyA+j zSVB)^c+%~FzGmk?^|0KTf@9eSBme33W#$ zPkG+{P=S)7a>E0xo6rnPba}S$ER3ziSoFYP#w1w5PO;VP)3_F1)8@}%p)GjrY=F;2 zK4RlbjLykaUxB2TDu-k^dqME@*?=mjx%@EVm=I3WTT$gk-hxZ@5RXuS7J*J)fzDvN zSJNUz1XE%7p3dZ3c6U%cTC2M`HDu#!hJD9bloO3#-0fxB910*euR+FA-Pg~`{gZHy z|8osV96lGnR07MLY~xox`#Z3w##J>#C@h*RMtUO}a|%*m)sb4D9=fL>Sbfat;$%X| zS?b|Zagq{S)W+X9iW+RrQ)kUCOV;(y1zkxE`wlrE5OnzIYCS?Esg8wWRBWnv2TW38d6GvAQ$Nw&vYaA=5P2s!Fzr*_sfv`1X zvIhE41{8#}{w?DyCx80fXCBs7sn4TiV;Et6vbVdrvWpUJIAq*m(^>0p>RUToXFHQ) z*8<(}>*Qq%o&SaTFnd1i5nG>6F{Sf6Wkl*dvS-v$XiU(w(n89;Ya8P(a>75q8Ka)g zGbQHBL+!0gsC5j-s8)_QY$VV#^&nSN?+K_sYYx&R)KeiSxwg;!qo)ExQodH}vB3K=6&5e74K9%UOed=R} z>TNzhCEPx_{7f`zxE3c4Df!QJj6XLF2BB{@uv%0bj2zMj4&A<7v@Tn47$bD~9#&lBaM;nyy(6X=p_r}n z-x|YmG$h|s1)`&i+=$0b9kzV#BP<7eizTDD1*_&FW;}bjr4Tuk&}pCj_(iW?_tq&>#OLqIra0Yr!Z4P>+jEhI^6|h z3m-2Lqj-Bu?pJrSe!)n81^fyyrA%&aK@ZQBU*FQWsVFoWl&rG77;Nbt7P~K&OV^2F zOk0DtH7TzoC7b*rnr^{cV9cFkbDGWs1gyrVq64CO=+G5DT(A1*G7+Z!I|N z&tGprX}n^ypR_Di2sgpDc+DH_=wA({k`3uWrSbZ8pRDt+_7*Y8i;F$Pfn2?3N?AHe zg}DP~1S2|i?CUDkvCm;b8DKJ`?*<(u{H^uL*IbwIhuI@dI7<;}5a$?S*USmaQm2dM zF%&14y&3frQAbt>2K`2sOuMSpQk7db!o()|$z6S|%F`&Y1~NHC z&Z08vmnIwMtGS*x59Nnaqk_G&v%l}P-I62opqPFGyPrgU!{w6z%R*W4u~P9?8{&6$ zNCZ?obFJdf7}gwjyf_UWq6RP?0ymJ!6-X=SyQPn0?tqVu{^Lej0nmuSdH4%8kLs7a zZV%~!X!8QoXH<|z(k5fYzYAqFXA_cAlK|Nl&rG}xnHwQ?^?JzH(JyTZIqM}X)*Dfp z=i}DCP-sf!#|U(=W=-n(Hraotvi-gER)bdQO|_~IR59_bJi)W!Q)goBpq$i5xdvopAP5MVn*KgX1*JEdIoOOd()UA= zY56B7EUp&inyXsZkD43opxQ565pqFGkjhcD78g!}Mem3m9I<}#f|Qn}8w|)?5a=JK z3Tw!M3t%3lH58&S!k^(pC>kaM01<{baWG<7VlRFqEJ9ing>;=zbvDgS8=iC|GY%h5 zAXkmuNI;(2MhbLLxH<7R|AHFqJd~i2bbwW!ZX@81QEg7{B(t*!V90G8>0x^6d5*b- zj5n<2p6GcVL$fw$F`?Q~wt-dbanaXMy45}oS%jR2RnBCdhyA(r`LG<1#zLs<_diQ;4Z7Og1Ryc7^?vRQ_MN6S3m8e8c;z}UminX{VT^9 zoWy|9X32zn!Yg8Wn*=ws^R8)Tfrcn115HNUEJ40xc;6W&+9Bf(}lF*%Km^ z9z0oqV#8ptIP}Nt9ol>@>_BK8AUeb(`MDSw*>}&d3#|eL!P6x3x<>R9HYinW0BNo$ zLb(J4Yf-CzFu45-*KeP&3d)^PuXs*|B3*!AT!oLqE)mlndMT9z=<`ym$l)Ej0sCj{ zOgAR9#j~vQMQ(FDaR+CX5f%fF+$Q=k^xO4Zp>Wk-p7&P8g2Fb*Mu@|&)ZCkz3g36m zF8y>1m&p!qE7$rtpCbG7@RK>PV5H6y!3o@;n**=cZ}6)fE^QMxf%rl5zDd&JxzZ~N zGzBOMbp@zyQPF_E_AlQCj7>2qJRFxy*LsXoaTmUwHU$DXPPPYeAbrV7o6WR|oHL!4 zV74K{n0f2$d3PM8JHgj=nX8h3rtp3aTp#B0^(@ZCJR1LJ67DN}9PDJ%C62fBL#=-Lt>c~%7yNapReYcFMbHH3B%4$`d@Z;_=mf7{NtQ7AUu^ySz zVMb$joKe4i@h0tY_lpW+4WI|yR+$U8TP#>lr}aMOmY>(WZM>#NcJVwrC!hQ zP^uLsuj+Xvqz9|8(tzH1qGm{Fh~UA1z=+%s*0d|CjMw#SItQ44^HHrGz<~j9(&hD` z{D8BA-g2mSqkP4Kho_mRnXSH(d^eu{{=H`sl`1kUddJD#xFU!l@tOLRCRf48vcMGC zbC5bu*jJEkCb$N}NRH9rvb6kVkpPuYW7A8eIy=5uTazJKSBKMsY zhXaj^C8y)+yXd{VYrH2A8BS7AyPbc#11k7G($%11TRIk&n!DY}fAy(X5BpHvY z3>N-6x4IJJlFvI!BE(&6k76aS5=mnRuo*bk#=_@{DIbWE*}gtEAUo((vt7L-y}N^5 z={u9>gZoGHvD~Q>gs#*CY%IW%Mo9;!t?=8En-loE$|$)ip()X}%8GOC@JN6rbV@k{ zXMzgB*Gw{Cdr_ypsJWMg0~CKi4;7Kbd%5J((a0HtaXB4FO;ZI&Y*`PSx{u(E4r6lZ zq^1>6^Np5xV8?^QrW8h8al^o9U)+JKVR1}6eB0NF4n-c2Ev&20nsasPZV9L9)Cda6 z%(vaHnDMyfg|-NV*U+uDEJ3LfUeTT74Cbo^Hu-r;G(f5wqzvi_d*KyRG7zs>tY%yL zSWQGOV1BP2tSA22#_h#VpRma^!R|FNas4De~-wWS+GYQNOdWL>!nnp0H-7P4m`fan& z7fXp|EheYj-SN_Ebd!Vr#k>!GKZbsT@q+S4=CD=tT`tdArEytvw06*OSa9WJcNERx zfyBB%W6(43^4`YA>}?~k7XcNDwhCNkI`u+RG5 zw3&_pPl>O)1i3fdpeiDSTOoTdf?H2@3@@`9%&%YAYApQjgtsZ(4iW^~)@iV)2;e6y zK{uxAsIF*Y2R?b3bJ*Ls+)15#1NhA}HfL|OsrTh!kS@(2NBmN%`-Hl=5K_7(AgY2I z)FRUPU|M!m1Qh_%T0WlHi4@N*LrS#7*u~qZgR;p)$j(EcbVa@g`(>8FgO4X8Qc^_8 zn?tobMwomk!^#Rfh!h{lfY(Z~PX-W`MbA`#Zpm#4I~D_w$(GrLX!C}FW(hC?^!e)$ zNRnbCL#aa5Mcwk++dqL-M~dZywY}5WLc?v*Tte>KCi^5zztZCH z9dzW4XB#+5Ah!mbbB4rAzVy(hU*=Jz{qkYIT>$gOc9@Py;k}nh!EI4@6_Y(i>edNY z_;kMeS-Sl{cihr#Z}JAN7P1}ieI&iBmTS?xBnp5SZhx>D)(PMs4`Qhjd*~s4{K6D`BEwuRUOx7ED*?)TdtAC-r03bQ*uQA)GT;0A6 z^aC`<<|ER!$GCWZ7QAv8rb$+r!^>=-birgOl3puSiwag7%FfAs%(M~&5^%nCbxXBb zNe~ces(u`H-8_?zsz04h<^I#6NfUelKmi5-Q2VuMaQ@SxF}3(T-nQ`kAA4q5ecLvZ z0j1}ZlExEQGYF`Hr5tIY!n!+cNoJ#?(xL%}y{TwKHdb?wAIp#kwB0nKD>q*MW|sC6nEEMO8Qpk1sRtNtz0l+dnUq=3Lh(!xu$W|e zMbSbrj3t3lKF3l%>XdmPrTr)vOw&SLwb}z)BRhH}?sDQ3Y^^(>$RVs%l~kQCr?!g( zHLCP4|DT6UE@|$6Ge~Wg!psYu(d-N&T8&!`RT$l)>X0dybre($0nD@tY|-1$k$cUm z<#TYnDRMlx@VDcgac^g22Q;(eCgjdXj-c^1K4}z5CFo26+@*fR;rs(?BS`&-3-WWJfktsV==o-n#za zJ?Dy+N4N$w4FD4C$KkciX_~|lrX8-=uY1B0wU`y7yX`fm|E*nckVL;h7zZT_7B9Dp z^8-Q@xFZs;JOG2tji*Nt$YFjKPeG_B0w+R7S_TZaBs$4CZB}hXjG;Xp1D`=A_@o^> zbhJkpj(twTnnK&|?RGgbV?zMnzcSp2A=ZA~eR2ryjooY3ctdV9AKf0ochcU7?we~H z{0awVZZCPyIlsAhB|Ww4K6W?k>NxxqeEW!bUBJXCBe#zN`^+lre;Z17PwQqF8^^IK zmC8^$J-P5x_Goh8AQ9*h(ahRd0#4YU_vFm$Tk2}s!w!4zB^EYIGJR_iDAys}O~}^} z8TtYG&(%a@UDJ2*yP5`20RXW6b2VAm{+i=1HovB)iSe&Deyk?t^j{+*pK9iNhRh@6 zvf1N+l74rR;1hs!Si?I-w%>w^SP!@HG~*4%Y_LP1rgzdx9?c&w-77T~It5L1P~slC zYB0wpN=h+IZ#}{My;pu)^6KCx-RpdvvRImrnpwg@`L4L)G|j1K9LiaM-Jq5Tg8<>D zUEA+lH(9GH#X}=-Jfe2vvlo6;Bb|2=os76sw zg86Q>=uNhM4?Z-2P2vKd^}S`619%NkZh|z`wK(7-8bqn=%TZ~99mO=gpfG&m>nM56 ztOpO4$RM%v=wK6{VXNk(zO@u_SCqt75WG`-G<-h8b}5r?x%1NX!3y^OEdG5t{%a8P zXnuS9`aZ}Mf)0RL2^%0Y!=aD`ty1QyQd^zZuxeE|7%68b_SN!~TDgO?z$$BycPj7h z5e*S2l_L75%+KPw)s0f5Z31$AhPjZj1Bfrtycoa`f|i_TD~0%HtZG!S6GgPitSA}d zRAq({4in`Pqgq0DN;f{hze$vr}r4 zJjgcf?e)l%Tn$*oN4cd(-11Vg8S`2wb^}>|`Yt^R*LK$))}eW2o5;f0P2MZ#^x)q)u;z4E zwGiIDdMMrf^W)&@E%snHnR}mOrH)`r*T^MK1kDS!DaNpv0KbST=pNEU#*KpnRRf?# z_{xKKH5^+tz#(xz(OV{G?OQsrISao;;~x!+TrYj2X9=kje0mYLPIWq7-Gk%E*2hPw zF86bAQ-YNje*+J8J0e!{KfvHqO@WY|0vkSCU&;+!zUXJHF_MMuT@3|@0+5G65B7UL zlLlYBImV+T%M7fb&j2Dm68Pv zc-BSmafeC%xGI(9Uiaev_^Tba^RfHCbe{D^>_@#zwcy5Y{wxepR&$zcC7u36YCMwv zr$XR%`2X&l003nW006lEsSuV1t_FGrj*bQ%|9u{ktSV!d$&b+eNUfU1FE}>h02fd5 zM`ocSc|&Sb$+(*AWo<+gt)p}3x%bxh&VV5;1)KS{JDm*%y64tQ4oF}>>XDbf9IoA+0AHXCm}zR}2=ir(FgeHIMmPsi(yf>ZR8exFYAl$DRF3T(jZz8~ zN}q}7j?77wst9i1168pY)00+;GKtf|B=T{ak#~S=uKVT&AG*k30xl;* zUIvQ0fWEv5G2$-)HHIPulxCQBpkheQsSSp>1YWS;queyig|DI0D+tmcICaA?ZhV<9 zs=sgxbN>ZXtAl{*p-I56D{)$H945jLnQEX`UMbxMDP`SYy#& zXHNrPykYq-I%Gq0qbFOkSKTD8s$*j;nSGH+u07(ltmp827?3`~e?HU7wAjFpFG>7; zNA-d@k?sN74fc8a{4`Xs`-{#WG%}3g$e$Wz2Vh4B02vlX>?3k`Ahu#hFYy_6-UL)i z_w-V=q1cj-7vRmML?CnOyRC-s+X(t~kk+1C0edz0;%m^5rzWxOoux~wF-xr4SEogX z$ZZCuCSeEFa<}&%6Yg%KI{9XTtAuF%3qTm=ozKp(DNFwp_bBWl+OWQ(j2z~8-ZKKPY7t*Z9`$$Oc&`QY&_|*f?+mbWY(zCrz!MPGO zi38KoO}t5a759sH{QeV+uOSBvz9v^9W~-IT5I9&R>jk*P{B(pe2agWoIx3|lZck#df z&jyDmeET;E5&)o);s1A{_rJ^6$iUj#(7?#*zbp7!(`@#4oBOw;X8?tdq)-4C0-{`y z)F37@iIg`cv(@S$aj-|iFYx6yL+l`$30L9=wZnS<{*d_aU%pfEI7 z40qw?wXJ#yCYy+AqBbuxm4JFLQ777Z2MJk^-ugB~S_s&1ZIV$GwZ{4o5Y_-voPR2x zy4I755c1s?ct=#?n}MLE*mNWv8Up#A^tK~7J#_=Z9%PbM1f|G#Pb7+qG^*>K5HEY2 zozfAt7+etFn1Bj#jCOW<5Q6o~M}@xXngk<02@aRx<)Eb%Oo(F_!m6gv(S(B~$}(2M zfRht9ctZKx_X2r#Py9RbA`0Dq${V#64~)yA6DTPJ@MFPKymXt#_~HX+BDw}XP?EOq z=eKbXlZKr*nb0WIuxU3aj+0esQnQnjC)iCK?4c~8J_6Q!lG5Zf@dC8vwW#{FvK5r0K#0Oh9k^d zMJUWNRS6W*k~ZvtS-W5J)zl7f4QHumPxMx?w`?qEJj~rf_Ap8j;XT`r5FYGPyffti{KPy3mE_?+kh>(6k)3v z$s3hBIeSSgbt7|P`5S<@<@JUzeJeZnbe?!`%qsM18#(!I%nE)nE0R+8GVaGFINELiz#LT~9t=~={UM!;L zHRNXC@XzZ!crV;ie!>xI0h=qk)b7i?c0KPX+Tr_~(+jil?c@Px`0hY(cl`JBMr)nu_VIicvCQviT5N~Q8em#Mk zPALf~tR7U`^7?aVk8RCQj_>;0|JoCys-nqin&SJMI{h)gXm7v?t%+`nuX4_uKz81be zieH;;{9S=z9Kac-h!ZnW+T)fh$O&&II&r;(OthXKFBo%vmPV~bvyz5)z!Q?GhlRTT zK{PdjkaL?DV$^xDwrhT9#==~x8Q{sxJuF#!Jz~eP=$bo z1@FC`kyegSb@?{Pe^iZ`l)s2lKFt&`)*j`l1R)c$HsxZJaon#8svl5kwXzBy9Qj1} z6z&S}LPi65Iuu07IDO&>je!0@hnKTHyS&;A8mlXaP$%?v#EaZk!>uc}8g#a%WIfNQ zzn=i6#F2|MYUVwR1+DkMZ;Q-bicRcDm@>{4e$L`+7!CmXx;N2VHm6W~evcDbrG3iu zi_x+(aFgLLxAW$fD?vyvzAHrA{W1PC?y_{S5X2PkSYpbYn_+p;UbpJg|$D@1J`_#3Z!VrzuWoP+m zm0ktcJz*K!|J79j`qWWFJIKzv!es4kfZ3PrE)B-Yi zTDP3Xjrq!s?ehJBm03-SgfyPhF{XnVP$1cJBgLd1srR%@hUCgNF|v;t+60q)Oja(s zo&O%{-7(mA39|eOO+_Q`WUZ|Nk`r>pRVPrapBW}JJxR^fFiO-+Tooq`wHs)M@Z|%+J%Q?RUj0WtU?ziw>sQuum`2%qf_H zEj89M&40ZzZ<1OO8HE*zKM|nAH`?WY%b4+1FvGq-L#^s|N!Io-`azxn|McTM^Xo!e zU&395Bk72JOZsgUoDYy6XG_&tUO1zrUi?HfWY@M1;i-Gi_L!*%e94JCh=Ig-+@yuf z-H79w8{SM*kO#-GLim*0YvdIafd~E}H}{UxNb^;FnwBT%Lrs4`D}wH}3PWm{$y^nY z{zDd%Qr%bnm%ZzOYSVd2!>iyV?o@jliA+Anqy_JDhnIYTuQTtEC(Ym$J=Fy~6|g}L zD8wi)Jz`nvHj-lgediDD4CUPclA5}`V`l-+Ac~t0JdzFs{lS`dj%gz{Q01qjZB2(f zuK%R<_imm6>2?*#RAJ3&luy7Q%ZEn&%8y9@3)f&()%XRK^05R<75D45`E%(V7;d=K z$K0lxS><5G5FdF}+Iz8_vVT&GUsX<662e%#qO$PK4{sf*P9%6lr((v++lX1a=`rDG zKrAZ>Fru&t+B($qbB$?lp_^B`v{(LX%Y`GSt=RixGI6e^aV=pqlpLMy3jPK^7>)`` zH9t^LA%41Xrk)<-|K`Bb1RsUP{YMPKitH2 z&f)|&?|3Y&0s=o>AbGAys^zUXA$!2j8htYN8Z&HY*y z%t!zL*#D1)#Mt70#ej2S~8*C+?j z3SXZxN860N9R9P8kc9w>y50it59WL{OMM2%Cy5{jkh(^482_r@B&W({Y zPgkKFtf*WvcP?gLs!J{4WIP$cm}`%BHhB;V(x~GU1D{cqYq^29T|Uka!yXUvAFAVT zkHo6)7G;3D>)Re8@4{4)Q!OiL{Gl$VCN ztjs}6s|PXtVr4u_{Ye9av4R{{Oj+M{;4v^%w5Nb7JEQ%jE7?M94jv(%;vD{Z%F-?| zSat=wEJOXP50TPYH)&V6X!~y83qd+udMcB;Zlm6n6?HSib&efM`6j3{Pge&G7VyB zUfFO{>ynm^2oC9zWAHBfV$PyQk8_eMmOvS(qr#_*p4my`J*PRuxkM6;5H?t`0gz;p z0c1{V9@@14CLd$V)uEFlry%$r+rX}EkUVv64@c>wHG@obE!+BlH#KXXBjeM1={`0=W?(pV zGbHIUeuL}2fsL(D{=+5T$}wXVM^jpEX>TApeXh6WtZQ#Z^p5u5mbTnilm6K5VueGkkVZ&Y^&1H&jzDu5>l?) z);VFf^cJ`mj(8y#Rd7prW+|*T)&Q%J&Xh-L~@6S2K- zfd3Q%t$xgq+25gF(J}x4{Qrj#m>D>mxc!eeagBTJcqn}JH+P_IN?*tlsg);B2vf2) zIIy#&>DcNldQ^h7oMRNOWl>k(oggO=U1;IP^V^2pV*iaoEORYEbwN_|`n9Fy@g{~9 zYE+kjXRfE6cl#Y7*ZRDp#|4Lx&A^Bjrm57r2=;?#`sS5yW8-5B?)-XQ*RlxG@uB)?ZYZIG}90=ezZhgGtR zA&m4Q?Ym3=!6vS5HT)KKSnA#Q1neeB1Oya<*zV*5>4@^DgS1Ch>|t?A#+GBmdY^oU z8%m5OMgS7&F@P`r&R{D4e=+t>?V*LymS$|*wryv}wr$&Xc5K@=cWm3XZ6}qgr|Wbd zT=doY0dszuYm9ds*gr;)BAuQtdbqkeZY|VI$M!hZ)X>Yt=sto(4Mw34c;u+|n?C$K zfo@OGl7HY(x@YAT59mqzYNLzeLVo<+$;@$p|$O>iV#BQ>> zI=@Tq7({ttq06I9HW^Li_E+y%s%%joJqPe_S!pr-RnBo(Uz1py9{p-ydO8r9?(X|G zJGYt{cbZ=oJ4a6^ndaa?n#^Vhyg*?1L5!$%4u^XB9?@B`cbH%= z?e{|nwnMtO=pkN&{Q*a%WnCc&2>sZ_m0|nKkpxPJLM&fFReG%%O*?#{+WuHtJb8)A zgp{%1Uit8TJhz!_?{%LQaGVoj65bH=t#6nTOKVF@jVxUcZ##*Md00kQPe(_Gi;tsA zwvVBm&S%|Av6F*`3~7+8#uILz&Ih$4$>lPL7q?$;$fK+tCU6 z%KB6L^HSrZc{cDT6b4Zqs(=Zmu?XjEOHbr-?&L(g93A@Qs_6+`1 zUS+g8>bB*`=fi1=AO}`)du)8`l_&SRSMWr`ZCXLW&=)U~;TSH+#&V7OYtM_QGqA zxr9XGH;S_rZJW-tsT8amGjo30eUOco^pA*u3o(+rQP|+T*BH%S`W_#_q=-ok{;jF1 zN)wnaKC7m1A~g9&kG;2UbZC{jmEXvY`zIQ!9YzZKy;y~X26HK|X(Zm|YaRMI?en6$ zW7eP->w>e3xfna~t&OQJBXR^>{;H{bnKc5~KpMab)?v7=ex7uEkNhEcjS=>PU-9{7`mu2Ed+DzCl6|r|^YW58Dn=Noc zQNGD`ezx#s2)dA8edyS&oW1dW*;0FXm8fH-RritX?EjSd(h64PF95(;f}8{F`$`ufj)l0iNCluThmt-tD+BMK_X0ga*q1O7HouO zl-3p^NGkO~W||mME7KpI1)OU@6*G#VATSBhwu5;~PEmH?&JcIVAv*afA?KdL46Go4 z9`?=z^%P}A5J#%$OOWI+&jaxcvisG$y*1G@x&R*c8N^eEK0y{34Tw_BVv?l>=#af< z2ZuEo82o?+ffkh?g{l0dZZ6=hL=l2jUxc><{HBOu7nn>C?;(X{l*RhECd1GTS%l^U zBosm}iH@Yz54ojGbXDA$J{vA(`+Z%(ff zal9QAihv;E<#=P*8Y{*}wWDYgA2CR&QtWcLA1s=7Dl^yJ7MuIoY40Qa6Zq!qp1ApE zCnO%;v$K7U8m#P|u;#WQA>1OvPpO=fbrkC#kT4BJ(6-s;R~-fx6p-?<7(i0_q!inW zj`#`5X?Uoj!ENgJxUo}m5-pBJ94wp|>phPt6tLGcv2X$b7rSWUu^79cBQnKqCct;V z{R8L`5BUOwR;L3dMo`W$Hmd?O;1b|$fLNS*zul?n%0qmu#%~+$fCRqE?DqW#=fPJE zWpz{Rl7%BM0HlJy2fol$+d4+2+tp!7wHp&3ZyoW7HA%`8;x3KJFse+LII_y44NG4_ zUf9tJBh2grkB4@PlX2spo1*i`&u64vq>NTd6iQl>*29tk8NeRvv&fT8=bD4ne&{e? zL3u!uF7i$L`IEOJ5RRyV%kr3`XmOT_f@U_ntPJX?IrL??U!;iHHj`cdGvV9GlkC)Q zfU3CUC{L}(m4@dIenhb?mIWaD^zM$D{{o7yN!Y6=>6GaYL5l4tbqNu7*J zG0;fa-HOS7W~Tqsv(k||D!d{jELb0EJ^6uQwtyZ37C+1lW`(*x4ig+sD7u1Icr3Ed z$^cO<=`0I%SnOPp|5N3*m#sV2{cTM#KmjZ4kHp|T|ByI%Vn}6yYYhcq09~9PSXe|-l(g?#+UWQ$U zZ>WVDBnr!yHDmfNPz+24=yE7=)w>MHK^CnSoGYK~kFd45sMP`UQ`b&yH8Fik zB`QozijJ;b3d(IN_dF#F>4inUprXnIr-fSo){Q^4DDFqP4U|88qI7}tWh;sjrf0J3 zLQp5(o)rE2gxc zn(i>zLiC-{l(3L%3xmSU_<{^ASQ#l$bWgb?xo+ru{7ySwsAnh?3c5rzrz5AX3~a;( zYDiS_hl?+!289Cphngq)Qu&fn3R8F(aP}seqH-Y|+P8&NFuDeMg zX?QdVE9s(PHbnfN<`*LjDnB>z0GxHkX5&msobc8R>CEXnY}w*RKP1z2D`G?mq2K++ zmsh$gl%M3&bC3t{!(d>=UuhUBU{w2RH#*c?*N27f4O0BOO#57I#-RH5NS7_@4uq!R zA{JdMD)UR9xHIfCym0ZIzPA~G1;2T;$>DAeb3$RGOA`TY3`w?NiX}yu?O--$xzELs zelv`%PHmde$XPcn>o?ZTypg1`!yWVbS{R~lKEs>iIns}Ky{T3MYww%;TnbJh5iZF` zilzuHknslWz-(x3;0PoMUj5&y4>d&Ml?$O_lGGaW?&=UTt zXuCY!l^gyeYgy!GHtbI?)(+pRw44~DQ|%a%32btbv;_p}C@fbj{$;$J=DEyX|HKyt z<$TBDQ2&YfHjOV7c5#M*ava%moQ|;|H~!SW(AeqB&pTlVfw+)a(MyuYL!P3A=DvzLkbCC4jA=|H@>#@%G?Q&CX4Q9G?fPW8+^P`)JgsOtTf2^+@kuH` zrFvzyqmmWy>g7s7amg3D7~2MK4>=F&p1%ffiztGw&+e;fZv80|p!o2O-A*;;8Ctp@ zSA!0WaOw?|aFV)X(H3a564b(F{$88S1x4Nj?+9kgKT!11NO=P&?s?H40=NE$2L1MB z%T0aTNQ|L4l$!1sl#Dq)v&r1$D+&<%81&UAuofqhR?PZm;I{c?Rxnl@ulH@;N1XMU z$w1)l3NIELN(ZscY#Nuw(&sx*`|>L3uf6#KLPFOO$zu0uLdcp)wGLTn{; zt|j(~f-}0=>=LsU-^YR5Qxe<-u(_?TI`YmxK6fD=h6e^$&?pvgf5F+;cG&H22l|kanel;|Q=V|Y%%E~I9)HZ6${~n!N>JMB3#3pR zO$#Ox!*w|RMKJ$*Bk{m+JOL;_G1HL8Hc!|GMl#`RFh@4BTOsI|C9uiZYANxWpcku#a5@6ga<#0!gCW_sGv5Gy zJ`j=B&o6r`yO6vsSHr_4^oi|}2w4jqg#?3{llhTX%2qe6aBj2V(3@wVY=$>% zhC#kq^)DIq;Bvnoy$qel=;!*F8bmu@UgBdYu5Ch6>P1wPPNiejvzF`6R8O(B<BGdF+N=FjAmB&p#cDW%;J|C^vn9Fj0NuF04$wkI;h0CS*jgj;I9jPQp*Kw`T8b zuCcq&Teu<&YK5+#BJ5X?_GiU^O}K$wS;Nnv<F5b$|31_TcV#f!0siJX;3LR$L-^`dX0?#*K6^HF-|aXS3z6J711L z+ju?eH%$(Pe@}6RKj{KccQeefWxFtXo2YZ+*3z4eQ+jG>dDFQ~EjMuQMU@6pt{J!H zQE7CQB8md7>0zHUL;+bi_4tavu)|TMoXS2Nz~tldWMdhsZm;nRmukFpFGj}MlV~RP z@zo|pKGA~G!H_;NVh1Q%w!qqMs~c~uqs;R)tJf6q=bFK@MXMYFy~PkZTZ|69X34fO zwfqrD=VbycQtsGmRXj-#ic?!n_8TKJoOfp~O;9y(V~Pw?0YL`mt&M5R5v@spJ(wtg zQ#?de0y8y@6`=*sA)d1@4U-aphGXxWg@jlQ{&9>8Eg{NG(Is;OjwIAHTnp&dRo}nj z#RK}bvpHjiRk6Uqnk*uEn6BlwS)X;Ca6?H^%ZX6qKs=141_XoU74;v#HaKWv0s&u( zw&p8&y2>Q^6oj>c%7eCI%cZ%?^z3{?0wCm<+uvB#q3ngxT}~j5eQ?MK<%5yLW~$d> z*mv?M#yaM1$|$wwl(xYy@J&RwJ+f2;t|Fe}zSIyWM(I_94#Sq6Xqby8m?}UNA9$Uv z`RYrA_sU9m+Fwh5xSD&glbr-t#IO01dIC?CS9eCkg+CDy9FpOkG0#wp(9|NQz8utQ zOjxf3gM&R~kX$%&uI=|A5BcTcG`!d4q~)MNP3x$i*vG|4sdF+oMluHPt+z!Y*e#6e zVXGNGY@qLWUUrY-WD!QkMsD^QfiGTe5=QW8^_w9^E_!?+xn*$S)<>(v7PB_)6kM3)Q}9aT@O7@c91Rq)X}$N|)5U7=oX^ds|z2 znp<-@#-UgL;`wPkuA;zx3S(El2Hgg`7ocwq_op}C;O`-Ts;xn3Er~L)UUW-sGVST> zVg!^fH~QczGr^C2Y_rezEP!h)xsXU;W^nEKVubT6O)z|)K)~b9iubbFEcWAGDR}lpmAfhA5?vt?-nZnB&`VzW=(Xve8huYj z@_cVKG&#vqM48tG0S!s@ep$w&Fjt}uctK+VI(ixh^#||P<2V1wQt6WSJ32q@9R~4& zy-GHW4&X_;-lFOV=gE_6B7efbGFG;#eMOZ*7>Ads_!K#5&V^-R*T^X!5q3PqeAMSv z^vDuhSlxW@HqDnx4gyK7v%J5^ZkE1yA(3zI+DEakBJ0i$Nl_!yC}T#~&3q&3o*X+x z%f3h`aDGM9wz-#6kEvFTPGh^nZsz2*1I+5v9ehft<)E=xUSF|%4DbxcK7cz$9>mz@ z&eKGzx(JY9t8MLW$(SC6Mpj?FvvZk`q5;!)G2uNkY}*(*-d*~U_W*hf$7<;ZUFP=5 z9`E~Rc`$Yy3FUYLJaOKeln3UD22?bc==j1OVe01eFDMb*gG^RWocMJa$!4_S`bQM^@)cfm2{;CUi27U!Y$yS=tBK- z7<%FZDkrLZ=$A?W#?g(7{w>c2?}pm{tETGz&3ISK5`A4x)h?q|5|N5un20dNDjsund-KJLsn5|mw9r%{o=jf)g8yELU(x4X%7t-fXe;qwynp}X>gNVIEpf>9=e2}yntgJn z6<79Ol4~UXMR+*<09m_(O1t~VjP*{E>_#xcf7KD@+WRG^-~}alWm#JTQ)i}t z@=^KdX%#iEpYC3F;_4zW;M*g6QFh z34%T@jR!NEjDlu18k@xYjHs)a(dEyf!$)yO>=| zCGKf{j?iDXlugV7pw0ci)`}CP7u3Y0(ak*R^|0s^D@tGwHbi!5PXO4%65g>1-c@A1 zG-$oLfc}n}U>dOPV6a5lPg4>#P~K@H$D8Oh4JAngS4uE7tzXm4B9dnQU!w$gh z&hn*cPS1N-tKDmLqv>cQg6X)+=y&oy=OT>ld^yqvmoFbK6KPV_j+V|aK#64)2y7`2 zN^5adoyU3|;A};hD%JY1=2KKz!i3h2$1!UGDKdY>>6{uv2H;OS+l8O|>xSsno+Z48 zuCP8}YS2Pnc$KL!7zP*9_)5RILmtMGo}9F+>eFtZ?~zaDP&xb^S5=u|{&JjDQaiUg z8TuzMvCB)8at{FimtPx8MNkBaYG(EV<^5F?IDax3C{LqLC?hrL#Yvqdc&VxU0He8u zca%?udQ5_dyf8`FTfYq4Laa)=X-e)fXp+|^ID}5&fRnKcvR2Lza*{v}p3wr>K0soKW2DhA!`VeFmFf-Tz2;_-mFlUqCa18E1CU4 zUzj@7%vZ1-`vj^WOF5n+Z>N2*Wwkf7fqZlI5CIo88I(yx?bkATN`>cn<{5;9WmjN@ z9asWHJwdeUZE*yXGCXN)oeKMtCX1#RN`m!J5M$WpeHxz`5Z%!eLegf0@BCx%o~J>C z{m6^YTf4lDY&DjHwZo{k4f~pEwtjVI60}MrNeQ0n81uVPxnD%vzF|X#no5lz7WxfZ z5W$y|+pLHY#Lql9Gf6>5NyPPO&Y)^gQK66X9Z1ywNx3Fw57-Qh5+-{`oGl(spgc;W z^(SLmp8x|vWJieGse}VD#)FqbNWv&rY>LpNWOWbM@fa6_#EOVE;buUgB}Y3()6`(C zBmw1;-AT_TS{n9waxqjMBUHPXSP88UgEY(>sjEoB3>%MPNvAiT8YeS~)8Q2-sg~53 zsMravWs&f4_jSV$*-3V~9}!p4dWpS&i049kdVv}Mg3qe375tgVN4<{;YQ*?=FiF7d zgp;i#Bc{JcN%RT%pZ|rky{HA={&KEtVg8#d?*GV~%}uQB|F6V(E2YnNg8?S=#si9| zV^vr`Z^$1A*byTXt;4$Gl8%AxfFhYzFd?x`j$gCo!c-yxXL+#NHo;f%&`j}u{v0S+ zUA4P`1Z1?|{34&qq=Feq(zBs#6UKh8seiZI7qi@g3quC$Eyr3-h zu)b%8FrKlopMoo&3z~=*l62Xi5=g>=Ef}V=sD97y-m!(86nOhV4H>^WG-t^2ebm!W zUwCAVz`jRBCrdyV%r?p!7Bk!@wK%=A%VU@+n_3Tv*iy9+v5r5%1cT=04zutH%7t8q zDwgz$%24PLbTX40o^_a)IVuBn6u5n@adC@BUCry?PMVU#1G($+tJXjT+i!#kEkXiZ zyPx?p{I?FS%T=R-kRPyF{Q#`8CDQ(ixKLpu2 zBvSafg70+iTiRe|7ILc0ap#HC$k1W$@E^{o>1W{nM7+BbB8A6xpcR? zdtKk;(NN|LJ|E|}17@%PB@d`{_1~f+7y!T;#sAzX))r3A|F=`7G;I2II(CBnam9&i+}|Uyng_?T@QTBLL#({(Myq#yCwHF-UHj zurZHg$cusi4~_`lWD^nNE8YR@-M&r#$|F;-t$;WSUbLuHgCIX-7{$=0_8iIKK}gdO zL@@K89io86_YBT6Mnql2RA-@K5kvAb4D=8zxyz6R|mm~C)xI6bG zrz4QY6NF|e9o%_sb%&vK1^&e>mBp!qvo!R6)$9Q6e3LN+=#+EbcJF*5r8*p3eXC$c zRKNuKvP#xJ=S7Zn)G%EOSQ*STzh+uO2B%;VU-G(u$$pXpSMpuzV;T*IaYeuHc$2Za zts!00o@9Iex*Ud3_yoVc!)UMjocmI6*+XMgHOQynIp^pn|<`QUZ7 zX>YzziAEh@-7NPGnACGa)xDdM$IlMY?jG(0;hS>}T|rF*5~La#Bhw@Ym%qb0Sy9?(oE^N5&l02g9TS?% z#Rp%FSN~2-Txz#8(3EJfFOs!n<0BU$qm+yfSQgT!S#tY#cdvbdx!ZyG~BgoT;c=u%w`^)#JPFB4A|!;j7v+|T{2#e z`UN3XA1*l|WmCjHwiJBzv%(HrU8U+rahsop+u0 zgOF-<(3;CmOzuR1AacD~v%w70PlE?HhdKk)$+!sX*9?7WOdk?Or*yNLPuAAX!oN0_ z4Z)jJEBUr7hua2igT8uM`0Ol@s^-x25J*HvHGeci-&9HVPqE>Uh$hl3b6WPJUY&xM zuZx<#Zy?ZEGCZD}Rgrd3E5o6Dv#q@ed%UrpWrxR) z-ndu~e%TTChJvw<{#sQ3!U@Y77y^aHcBdy5*r}JeE^Jmr|7g z8lRM}ruFa@ZBk48&nYGew7%T8EE9&G!nbA5D5q^$ry4b7l%zt-pk3SEw!Q)JimDw_ z8#1Xo^0e>Lv>o=;Sgkt(8FY3Q-fbq0ZA&Y&o>8iBNo+kL8Rk}c*Vm}vq#0njxZrF8 zS(+pnY6+V1Ler*;1}S9}lVLC|lEHi_2ugV4ZB-k3=xjwhgm~!lFZNX*oxvPSnLK_? z)bt_4_7kq2=gF4=;z)E_=ZhHnW15!Fh|$st)hpRQAxq9=9%t8n1Cx}9PN7G|R%oux zh6?zTGqDkte3~mZ%Gt20pyhh1H>od*I?J9^D*03U>&V@_T?EY*6Ak(7b`!OYYoB5x zGPhEvn-x3X-HOz0#7V zUA45Imz>|RU1o`kH8lD@)h>JJ4F;H|{#6JGEo7;|p?A?o?O-Ad*Jq5$~)7lohnKC0otulEq^*TD5ZNqub${^L?Jq@riF z!GPfRQaam-jve(k`-*h3ZmtG9ND%*DDN`^`E;e^Vv4mrz^<&44G52?35v@rbq~y`Z zN5@AtUCjj-I#gOfRCtEApyW0fiS8fh5sw+HMnp>y#ga)d4rQ;A^%F_u;Q3C*Xdp=m z1&MqZi*rL-ro2vuJd71UgL1S^MgeZ<5lq@>!014>q|F}6frKq)&oB;GYrpVUP?KCP z1%dYoR;4PS$vpCDwzQm(JXPoL15S`wHg&L&+^Pf|io^s-(&aqFGE+|ZE(>WgS5{v~ zfHUrxL_yi>-q{F=8NFr&6#XUEw<{($kpllV%J0|#(CuY+C%<<1)3#%2tNVP$eWAsd zyh+&pJ2;oH=Zip#Q=@ejtV#AWi`{nA=Mmq~1d>>zm(TXQ{GhQWkCyK{94?YDSu*mj zCk&y|3pg*wLabXKvtCR8a;eKj#?AWZp!>3VB^d^6B7u2EF4Gr#BP3s znDQU`xu4LY7UvHN=L`Hes{S7LgrZT%u=Pr!ToMVdtHpz}KmzZLnRFub2x|!iqs8rm zMQE2+EfNHJMqSZHMn76DVbREXU$rH3S{&cnWI-@5B z>_&&;&mryBVPv8Prn9j)>$-5HY8(md)+FCQd7x}+}q1VUd6pHoLX zD;`kpczV4Y1CJWvi3f|+g{B#{v%9VU=E%96)^M@Jl|w~;Tks5rD(S>*`Hkmna|OUY ziRA*g0sDQ2>ObR(-L1OIo7UmBMPIBm=iC<&5nGc-I0u-Sd?7@pQ=9`d7%dfy<(F@eKtfi&r@p zb&b_1jGuu2`NqG1B}ZZQyEdMFvEcvH+Hf-Yk89kPs!n940E*9xn)YR+P#mk)dmTa3 z7A9e-uEeC|WaVedKK?-Ml6fjI~tx)P+gYAZi3Cv*tlb^=o>4EVubDM-q=UWe``I zehrlwoHRISohtzR6Jx>?kDY2B<84jL zy{anpylcNoP>S!NYpXd$=Ng;11zhMQJ1Pd%F`PACYcvW;tP({3!0vK*(M>?&9B;LJ zGdl^UItqajdL!Q z{+y$V?Fsr48-0PCc>{RE7u*EI0?TdbH$bG<=(4fwCtj?-ksl7?i3Jn(iE!`OW8?7* z3j@`q>6d_6&RtLD%pc_X&3puGcDhg>`_ISGmZcb{{jJ$q{;I_S^3sPs8B&TeDD41y zkSRwTKD_G)L0ucIn7oasT=BUg=Z*gA4LtH!Il24XoK*gon__9XqRv1`O#yZBbfD7Z z;s#E=my1nP`<{Bt+Sec>2;st+n zvi2^4P1>&)Is~;VIl4G9Hohq9x(Uw3prke_^VEZdk^9w5UhlsNKRXAn+U>f0Cq1`N zQZMj1B)iZGhPse@H`Dvy=)?Im@Bg0!lK%~)2i1^NO#G!TGyJ|t|EK(X#)%}VBkqX*(M@-eIZ#SM@R1)0|XwE=1hx+v`+NT1n(S3@PJhMC_2bukt<8! zBqRmZbq+=9J{LIUf-Hl1vS692lxE;{Nsww&4JmS}LQZw+#U?fU7H+{8iwlXRtE#(f z;0skG)n}E=#;xs%I?IZDq~{fTXP6ok*NzQoikK_M{dama&IVg@z+(8)Ta zS;$P50yLbdeErePd)Wl0>3qkZRB(Yk`pCGYFS2-f*8YKmT&6g`Cq)&BJUMN1bf&^*TYX22 z^Z8vtZQI7T&%p7neS-E@&nH&?6 z!5Qy))%N`h_YeVGljAM&O@yQ93G1&<*tpG^xoRcxaSkk#al3qlg7-@MjGF8T9N5%g zfSAU3Q=|mvg*M&X=n<$sQ>tI_CnQ)^Bf6#a!zCJ*|B@rvukmsF!5`G#5B zRUd8t{igmxA7DszyK&4+cxfyuK-Lo4=&uwkj*Q;d{;9lPi$f=9r(jbq47=@lxsf=R z?W{gF0ULLH=37_SB;Js{tNiGi`f-7211Fw#D|2h|xm6SVo34KCNvz|q=T#h4Qc6@E zHKzGZpZJ4RZzrJ(=xRAe{OVU-V-i+kXd>dzDK0?(tRe%YZ-C#YdNfnAam{Q=lUwy7 zbnpRlew|M(%L?-`hmXBC4!BnIG$8YggD(EL%SAvp5=UHZ%g+$_xcARzDrp^?S}9dHgquyp4&Couh}IwTZ$1 zixAK`bNt`i*-q(fFh3j!yu<`&BBrExK8{Q&Bzx3|OVaS}Y9IJ-Y<%TFiucgC2s8(< zjkQ!qWwL9}Ov2H&K&#!hXbOAFR;1YJ)amInu%oHnz85pcRp_hP>FUX7&J-_y)Xb#x zN@wpk9>14C{e9yo8P1_y+IuJ&0{_FpORxi=mkGRG^1LBY0>!AW#*y{Fn^_=PqG zVQ&jML&0^tz!~Va`zRfA!tbNS$0di*B&8Rsmgg?F~WZ}>|oHS?>|ufhTLTG8RN zUF$_8F?`V(aPr}^wcRgp&vMFiLryl=0sjc8%=tpKv;s6%X|_XWL|I<7tay`zW3X~j ztV3nZ&$F-~^x?>X*0ABTy;Bcp4JB(I<~(?9>yVa{7WrL$o|y?!J=n;PzT0}#w}3BT z?AKDxOm2?ldmQc*4Lvo!I@8%g@v9Wzd~BgFELTkjW0)%k%TFdtl(~qp)xb?H%3) zUiDk}If9Wix$P`ppT@HVLN2>znZw7|<;j+-nP`ue(`&LrYnjj&k=4=>8UFN1jLjSv zTQ^O85^P~*qmrw6`xIu_u<$U$iBnK!1uE)g%qPTy>XyLGPLG%z_^C02kXTTo*%u-f z?CivWTJ7wq2fkq1)B6A+7hgeG;kP|V@rSUc?`z4*#Y?WDpliKCYy#Pg0=q0d7D{KY zL91gFi-pZ4h`S1>13VO^Q#RVdDPpAPGcS7OXH)FJsM{Ql_BThiv<13#-`i}`lxG4a z);siGPDpe%W5&Y4->ZJ{BlL9mZcnx*_zYuCQv`8~;zH6LqX%(nSo=y`tWY$4GawHl z$7pD33*etPE3Q_U4CnoJ4xeb1AgRm9HjF94UJ_!>CT}IVMYe;6H;iwc z?`Cl-*y|a2NpDGEANy}q(d=b{_D+Q&Ea%$=LunzGDd(aV@Z&(u(SMLlA;a(sf9+Er zMIw{JSm zU|6+QMLlF_FQR}$wiqPM*AB^~-W;d3`VEbb$Ii2l2O-*dMN zg($$)Xt5KCITKcsu8Y-yCSDksjL*3l(04+!w>kdukH?rWJL?CY2Js2-Ug+vw@5Ug> zQ*GrL2^jwkNezzaTZG7{7`l`v7vp6H*ok1oGF3{!qU$ zh?)=X%xueiKbjZ+G-N`@`-6>rjTgy}*WIbk)h#9Dm3S*V!W4Oz6u_qZOz4F@s7Pz0 zYAp}$;)FtP8_p=#X3Z|v*_`zi%;IX&?Q>zPq@D>ReMk9ZSgMguKK^GI=}{jm#E40g zWCTHls!i*X?l$GH%&?2gIRk$Bq4y_>zHV+y0)56gR@2F8fP&Rr*GSxZ855ARP9RLZ z=@zli?gL$f8*~RzA4EYIr{LOh{T>+0Tm^oRl$gW)FFFB7p+i5+W{&Lke&@3C9`-4K zM^$g3Gn(WzlNV$rvRitTN-I;r{oFV-gu8U{EpWi9?qFcoC7 zi7izpsu`RFT!6|3LiE9(g?qq6Axj(m;R2LF6 zOI0PZ84U_zsgD$8^~J*}Q_dBJ69bCYoa(qb&xscW)AyS{_A1^U?k=zMwy6M7Ni-z# zbj}KJK5^m)A#`dE^>B#@`8EcgGbl8s%rFB&s4eTbpquBMy8b`>w}!R{-xcwf74ee= zg>m034YX*;8GzY!!1&32-2n+zAexoyX^V04{+n5y7r+A!9*C2x;7wWe!}Tt?e_AC^ zs-vcIp}8$M4Np2u)kdphwl~CXljXAp7P6D`c<2d%ujQ*@rc?WEKBk$Z(T}g~qDnRd zlzlJCA}UUC!cs9hB!+1bO#dB_{3&iKqHeS0X9vw>Zx(ljz9L5}b~y%(D%n*x24_Gw z8-Ssjj73)=m12bs+$gybxo#7f9t^TLk)jalfR`!qFMeoLrlOfg>g$w4(YEhN*kQa9 z_GdJn?rlNvKyIHR@r&a-;4)bDib{SOM_!b=R*GOM#_Fw3LMOA@>_E9)mEOg*#&Jg# zbVFB}S3mNfSj6r%2TeT)BnWrn4=yti1r$x?zzeLWXX>AXe55S!2C_(1+Qhr@D4=9ic7)6ibUMZi zKqHZOjV5K{(2G#5(t3@X!f1jYw=TGJ)i$hZF(#*uI;ryGd?FPBP{7&zM~9;1>u3hy zs$ad6OubHqt!g;9bs{j8VRtgTo1CFqsctl&$(@o9Z*Q;ICjM@PPLp74_QW*?vlV(5 zdb*}5DGqz(MKv3`Lv1YCl?~lh&!(C*yNOyNSfE^$H6+}mwp-Du2gysWQwML`Y|kd(7WNUYupT$<1kTHHv@jr=u9PEqLq~QK@eJZH-Y;z%El?sR5e?-0 z)5-l3@s8Xd0gO4Hcz0M-F4N>pd+$6L;M_1(xf;E*rKFJzDjC(Xpw6YB*KS^}lklbY zSwnaV3-1x^UuMzu zW2L^LnBw|TywYnGI;@DU-_ci)o7&K?p|YR59X!GQBPd4-IRE}RsspNn8HLSsw0dXy z8b)Iu)pqVRERJl;gmrq6!|BT5Sbxcy(qyoSf;0A;)oUJ=@ksY?s(a?qN>lY+lS_SN zrGl`F+zvl9Zh5sfYfPSseXz~Y1gqIrmk0NH9Qj7)myU-Y7`$79lpQuhK2Q?pk8NPu z5Sj6%vhdVXoYE2i>!WrAtVa=znAGa03IvekM|A`lNo0$bOxGx(mX$oYm3hY>kANg{ z1#vesP(15xplCp2kYz=Vj6Z-rBWg@O@C!d4nB~4me?umpgvY_D=}T@SFYTH4dM@J6 zr~o263GRj&inBdJPznUjgQn_q0dHPZzOt*KI$XFExTZ zjlaN+>3gu~^!#aq45?%hitetK-BL!OrW4-TS!QbD;oh8P_~#17z;*VIG#%f<28F3> zOv+I(WiFqs~DDT z*^MBE_#~3X65x*LigatdGKofAkTMw@{V^YTO&yy_k=zaV+} zdAUgs3(NWk+o-uv=%fV%+X@vogc4ZB33ziN z`TsO|)~}p9yXMKE%mR+b9Nm^_d9|kl|7Ct@2#xJh`PCTJ#w-_XgJ$$;@jh? zeON7Nd@Ri$gWX<(d5ilceqX7Z_WHA_rFK%svA(Lm)6bX3d#1L!qje8J+|nG@o!Psb zzOe1%%1>*+cmQ+QH*0C+U6c~IzIE#_tYI^L4RIdm8b8qM?HnU({1BKcEdPLl183Ol zLX~sM6<9Bw*f*hl{+GO}W}~Bg`Cr*b*>8^$|7Qfk|4xe3`wzOEiMxS~y|u|N$2D1A zMrz;}(Rfpf;V8+N)It>^)Vm(ED+t>4muA7(kX^=g;If4EXM3s?55+m=VY}1g22_y1 zRr-_yw9r6{v1%Ey3T+|;fnB{LXb<)>ek%Zq3VROE*jlnIqtZL)0P?7#htCMOq6&t) z?WLo`7qt&A(lB=HF8%e-XK)lLry+H)I8+l_i$z5i6Bo-z783gg#BHJ?Sh+`4cs(#J zss;#E;BQder#?&bpn)=&m^|9LOUD-$nOY+lD0-Y60P3^Z`jKbzKiw+&RH?bX>^{2b zKX%OTSmNB!jGbUQkdFaZ%hO&xp+W3~os^W#a@3@Qj3A0BWlc$fwXCjaeznwD+a$s^ z^R8WEmQ-}#)pPc27t>Sv{ZevU&*3!_UF4qnuY50@ZSlzp>r*x#a%{QoEj({Dv&V!Lby7+`|k zctQ8ZGv@M?2$c;lAVy`CGjFl zAw-(Uz7%<>{GUNJ!!*DD^Z7iVnd|d$zH`sH_dfUDbDwh@xANEMYv)+$Y^2p$p6Tzz zzw=zyX7da17dtT{bB+A$8#YZd!aCUS`&HsaRf|BNu`` zw!X)=-_`)s^cZNn{Mb~5JdtAO21ND#-H3g3dz{+OQ#9}L)k#ZHcrTPrD{>bV-KC=c zR78^}xv2DrpH5Y4p|ygkE)zdL*%%77x2G?V`SfBpweDNjSdNVBI;GNx^M~oz{W(Jp z#PMd`0u?v@a)@(yGqFa-NN-A`EYMi{m}yc0E@8#UD^~Qb&bX;VQ8(mPZ_n;9F5?)fOaVC#GPFI#cx`q3j>BlpiL)dw;xX{o@k5g;SlejLTZTXApf9uTX zR>_;DXqW@6@`(@x62N8^FFP9_uWOFJb|_})HPA%HTWSH~ElF|c*;R%QsmXa5;}gA# z1`CGdvt+wG@LKBfZNC(`7q?uvC^lbp`eGA;>bmS|%YWWH&uF*k!*v=7?v(C4Z6AcD zkd^D?|?F{EX`Fzh%4?sYAb$WjWe1y zCs0EXTKI(0xS)n*y>Q-QN%GyTdio!ng*gpg`~faZ6QP@r>f%k{Txz>)se!H~*-&`~ zILqgor5yLrzw_1nk zpZM9jWX6U>aB()$Ix#SSq_5tc%wOr)_ptI-&TZEpHJcf1 z(E|~V*%tpgl!m7dfnI(VD9+qN*Fhfw*0}=2Bs)iUEELhc4z4-c1E~pqIlPvQr;rY> z;C_fzx4KAGq^fMLDy;0p$u=>^kHjoJ>XUzEF+_}-jNRy$>LZs?HAy-#+FmB-y$|HTbE8YQ4_v2Ao#vi$>iKkfe!v8F%FIG z$O)1@5!K2%4=;8}+nC!TGnL$Vd`~#(lHuHUwU#y2c-3?^wpRoj*vRqSBMSpJj4zrK zwq1Vpo))R-EH_v-oJcHfCF ze9KJOsNZD|YO8T0(fm0#r|PZbyAcL8atmz4cM7FtErRo7*3~@_1v7{pDMN_!3!YQe z21?{L_~W`8U+thnEP3r`uh!z`bVNxveCigvNV@;fxH&q=-$mY#5Rxt2>V^pa4q2j6`> zedU5)?~4I={SC?SgOAj5en|0%`X`$R=I$k4J>4>ExNy64T0QbpYAm)PY)YwVgfL%*Z<@JCT%6jJQ`IY7m<8T0Uv~Z{p%CF5^Jg`l?)Rz?sLQ= zFDKD*{}huqR&QDQzBxHRfre44aKm*~s+l^Uy*G<*V=%G_+A?QT5D&T9zO$_*QVx?0 z4O9ppE!)up5qDT<4@0k*R*+lTLLDaCI^j`^DMDOtL`4hdPA2Y#>bVQcdbeDThPmh6 zk5<2b$^o}{deC%QrUN!%@+Hkr{cU_qS#dcRB*0wU;f=&&7`4VnMqZW=9dpV7ri0FK z$I@Ion(|=lXR41B*{;+=T4ES`t_^!UT!z;Z&eF>aR;RlaGwu=cJdb>OzCy{91s%2#4kz7sJ}s+{8S>GRRxV>E-1wIaX* zGYj~(9_i1=A7uSA#4y^fTAOy%mlRAT_OuF^^7sRO@gu|Kkx`-r2&N94({9-{qL}Y^ zIgzX7MN5KsRm=0(Icr6a=C-wHsq_mkj_ao9ms+)>zqyXY0()Gvsb5s&*AtWar-Qu7 zI{!Bvbp2wp2Vt3r_Zz`s3F*C$P!X4tyl*{iJaX_ zJ@xVvGA8w45bB%rUd>r8F$9xJ-O48#r^p3*HWXQvq=$Wdp|Z_2{5skp&J^6t)~1#X zhSdr30g_v)wAP?&{>}bQrt44l*n`-_a_h9%SX2=O@He9i1TaVPamr*7)`)v;0v|$@ zbPyMUn8Uk9L=5dX|Fh0EPJT$74m% z;TSni&vMx@sKMLJ0fb*f_#@wk z>+*AJGx9|XSI(SKR=Z#RcV^g&;OLU7b2+%ubR*6)d!tX{U!YtCIJ96_xkq$MHgC2_XAuzsQ%iTUW~;wOBf zcHK*7zCTWqk^dAZa7EH3b)n;ZQQ2w9Touy*9fzKoPQRKk+eP|}p#fX@J<>fJyz@FQ zXS_gmB5vd0q&cut0$j)wAg4pN321=t$omX%BcFsq$i0$lLJ)|fo1-@b0<3=6z&h8( z)W8P7-@!USAa*NzI}+r>^*|1-sE+EnaUh}`@c+o0{HGis3G+2acQ-%)1akNt0s+f` zxy}m;pBEJ4+LwVO6M?v6YgE`R@kR}Rk~?ry{FH(6D>Uy2lZ2e7z~2q*xC!Gm{o68y z`JApGl~|4$RofKbKNp@%j%DL4i}pE8K52?=gDbIbaGbScJ-JTR`ze+E@=J}{ku1l> z-ppI~KuR~h{ab1^r0Wm(ka2MT-mg8Nk4#-v0Q>7A*k2tYi^yMty!N$20s`+Ba6q)6 zwa;6=Fz!K|pP!yEv6q2B`M21Es=eu^kIyCmfyjW-gyNT2q?}(n0>}XO9`0Vq6z8Y} zP_$odLe&5ct_|rtQu5C-DgkN8fUR={Dbyg4pApiS$Ym|^0t5F7z}^4h2idU?l4BpG zcpDFNGY;agR~}Lh%SS2eyK(;Cfq~H)IP6J=MyUm6RRn@e+T4|+L!+;fz;6|94hM49(F~oR;C5)0OLCDB@kdhR7DK|+S*4P zBpq;faj~=U2HNknqol*iIB1?cFv%a3#rn(zKT-n(3OB?#ViG(-RMKG#%s)`Bdt;{{ zcooXJfG*b{aCcQLJI>pEfl8?6m?ks-KvgvI?`VNWTy*EI0PIFNvIv;RnbA$`}t{g=fbhB zhwK=S++PO22hO-T0Xq6^?+`$EA-ZsDJ2x8#S1T`P;iJ_!wC`H@VKx3>?*ayHjnBXW z%K)G;{i+6%)Rcl&4eRUPb`T(OlcSsc&*OEdH3J>Wry9o2n=q-@wF7KY0}PsAG~PlQ z8t;G~%yLij5}5N`|%npy$0prHz-P(T-%n!lX+> z59}=T3_7T+?~l<~`(p!IJ{?Rr`!wu0^o%E{xaR8PxL-TOhfWMkME|GQiRclzQHgOa z$B8KMxiP^qPm*J`9D3AWR9ITuaoCX)rg zojr8kF()M^9P~YQ9J-eY6_<&NgW5`v4F;3X2@}+hh!U#-(LE`sP(J+s4MqA@Fi|6g zuv5{U5~x%|0xVPvE(%QaG8lF``sO<-edz=iI;O37O!%fk?0EE@TvR+0@p1g`eO*kf z$qno*^c_=F7Awhd){&hNOu!X7Dy;1ZecuHYs7Zzf{C}`m;xV!QwDLt|Nl>D(P*$Lr z>NI76Y0mVgmGMsw7??BN_{sUbPDTTKdxW^mHHgzEaR@jp|DHW#S~jBs+J6H6RE5JO d)4I&jB@+Sg@;TBu$__dWOq-RdfLSf*{{W)JaJB#d literal 0 HcmV?d00001 diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index b7284487c511d..f2d9e6b568a9b 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -28,6 +28,6 @@ export SPARK_CONF_DIR="${SPARK_CONF_DIR:-"${SPARK_HOME}/conf"}" # Add the PySpark classes to the 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 PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.4-src.zip:${PYTHONPATH}" export PYSPARK_PYTHONPATH_SET=1 fi 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 ea4e1160b7672..6e4f68c74c365 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 @@ -1179,7 +1179,7 @@ private[spark] class Client( val pyArchivesFile = new File(pyLibPath, "pyspark.zip") require(pyArchivesFile.exists(), s"$pyArchivesFile not found; cannot run pyspark application in YARN mode.") - val py4jFile = new File(pyLibPath, "py4j-0.10.3-src.zip") + val py4jFile = new File(pyLibPath, "py4j-0.10.4-src.zip") require(py4jFile.exists(), s"$py4jFile 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 d245acf49aa91..99fb58a28934a 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 @@ -242,7 +242,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { // needed locations. val sparkHome = sys.props("spark.test.home") val pythonPath = Seq( - s"$sparkHome/python/lib/py4j-0.10.3-src.zip", + s"$sparkHome/python/lib/py4j-0.10.4-src.zip", s"$sparkHome/python") val extraEnvVars = Map( "PYSPARK_ARCHIVES_PATH" -> pythonPath.map("local:" + _).mkString(File.pathSeparator), From a8ea4da8d04c1ed621a96668118f20739145edd2 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Fri, 21 Oct 2016 09:49:37 +0100 Subject: [PATCH 17/48] [SPARK-17331][FOLLOWUP][ML][CORE] Avoid allocating 0-length arrays ## What changes were proposed in this pull request? `Array[T]()` -> `Array.empty[T]` to avoid allocating 0-length arrays. Use regex `find . -name '*.scala' | xargs -i bash -c 'egrep "Array\[[A-Za-z]+\]\(\)" -n {} && echo {}'` to find modification candidates. cc srowen ## How was this patch tested? existing tests Author: Zheng RuiFeng Closes #15564 from zhengruifeng/avoid_0_length_array. --- .../org/apache/spark/CheckpointSuite.scala | 2 +- .../spark/deploy/JsonProtocolSuite.scala | 2 +- .../spark/deploy/SparkSubmitSuite.scala | 2 +- .../history/HistoryServerArgumentsSuite.scala | 2 +- .../spark/io/ChunkedByteBufferSuite.scala | 4 ++-- .../serializer/KryoSerializerSuite.scala | 2 +- .../spark/ml/linalg/MatricesSuite.scala | 4 ++-- .../spark/ml/util/TestingUtilsSuite.scala | 24 +++++++++---------- .../classification/LogisticRegression.scala | 2 +- .../stat/test/KolmogorovSmirnovTest.scala | 3 ++- .../MultilayerPerceptronClassifierSuite.scala | 2 +- .../apache/spark/ml/python/MLSerDeSuite.scala | 2 +- .../ml/tree/impl/RandomForestSuite.scala | 4 ++-- .../api/python/PythonMLLibAPISuite.scala | 2 +- .../evaluation/RankingMetricsSuite.scala | 4 ++-- .../spark/mllib/linalg/MatricesSuite.scala | 4 ++-- .../spark/mllib/util/TestingUtilsSuite.scala | 24 +++++++++---------- .../expressions/StringExpressionsSuite.scala | 10 ++++---- .../spark/sql/DataFrameFunctionsSuite.scala | 2 +- 19 files changed, 51 insertions(+), 50 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index 9f94e36324536..b117c7709b46f 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -500,7 +500,7 @@ class CheckpointSuite extends SparkFunSuite with RDDCheckpointTester with LocalS } runTest("CheckpointRDD with zero partitions") { reliableCheckpoint: Boolean => - val rdd = new BlockRDD[Int](sc, Array[BlockId]()) + val rdd = new BlockRDD[Int](sc, Array.empty[BlockId]) assert(rdd.partitions.size === 0) assert(rdd.isCheckpointed === false) assert(rdd.isCheckpointedAndMaterialized === false) diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 2d48e75cfbd96..7093dad05c5f6 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -65,7 +65,7 @@ class JsonProtocolSuite extends SparkFunSuite with JsonTestUtils { test("writeMasterState") { val workers = Array(createWorkerInfo(), createWorkerInfo()) val activeApps = Array(createAppInfo()) - val completedApps = Array[ApplicationInfo]() + val completedApps = Array.empty[ApplicationInfo] val activeDrivers = Array(createDriverInfo()) val completedDrivers = Array(createDriverInfo()) val stateResponse = new MasterStateResponse( 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 732cbfaaeea46..7c649e305a37e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -91,7 +91,7 @@ class SparkSubmitSuite // scalastyle:off println test("prints usage on empty input") { - testPrematureExit(Array[String](), "Usage: spark-submit") + testPrematureExit(Array.empty[String], "Usage: spark-submit") } test("prints usage with only --help") { diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala index 34f27ecaa07a3..de321db845a66 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala @@ -33,7 +33,7 @@ class HistoryServerArgumentsSuite extends SparkFunSuite { .set("spark.testing", "true") test("No Arguments Parsing") { - val argStrings = Array[String]() + val argStrings = Array.empty[String] val hsa = new HistoryServerArguments(conf, argStrings) assert(conf.get("spark.history.fs.logDirectory") === logDir.getAbsolutePath) assert(conf.get("spark.history.fs.updateInterval") === "1") 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 38b48a4c9e654..3b798e36b0499 100644 --- a/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala @@ -57,7 +57,7 @@ class ChunkedByteBufferSuite extends SparkFunSuite { } test("toArray()") { - val empty = ByteBuffer.wrap(Array[Byte]()) + val empty = ByteBuffer.wrap(Array.empty[Byte]) val bytes = ByteBuffer.wrap(Array.tabulate(8)(_.toByte)) val chunkedByteBuffer = new ChunkedByteBuffer(Array(bytes, bytes, empty)) assert(chunkedByteBuffer.toArray === bytes.array() ++ bytes.array()) @@ -74,7 +74,7 @@ class ChunkedByteBufferSuite extends SparkFunSuite { } test("toInputStream()") { - val empty = ByteBuffer.wrap(Array[Byte]()) + val empty = ByteBuffer.wrap(Array.empty[Byte]) val bytes1 = ByteBuffer.wrap(Array.tabulate(256)(_.toByte)) val bytes2 = ByteBuffer.wrap(Array.tabulate(128)(_.toByte)) val chunkedByteBuffer = new ChunkedByteBuffer(Array(empty, bytes1, bytes2)) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 57a82312008e9..bc6e98365daef 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -100,7 +100,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { check(Array("aaa", "bbb", null)) check(Array(true, false, true)) check(Array('a', 'b', 'c')) - check(Array[Int]()) + check(Array.empty[Int]) check(Array(Array("1", "2"), Array("1", "2", "3", "4"))) } 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 2796fcf2cbc22..9c0aa73938478 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 @@ -287,7 +287,7 @@ class MatricesSuite extends SparkMLFunSuite { val spHorz2 = Matrices.horzcat(Array(spMat1, deMat2)) val spHorz3 = Matrices.horzcat(Array(deMat1, spMat2)) val deHorz1 = Matrices.horzcat(Array(deMat1, deMat2)) - val deHorz2 = Matrices.horzcat(Array[Matrix]()) + val deHorz2 = Matrices.horzcat(Array.empty[Matrix]) assert(deHorz1.numRows === 3) assert(spHorz2.numRows === 3) @@ -341,7 +341,7 @@ class MatricesSuite extends SparkMLFunSuite { val deVert1 = Matrices.vertcat(Array(deMat1, deMat3)) val spVert2 = Matrices.vertcat(Array(spMat1, deMat3)) val spVert3 = Matrices.vertcat(Array(deMat1, spMat3)) - val deVert2 = Matrices.vertcat(Array[Matrix]()) + val deVert2 = Matrices.vertcat(Array.empty[Matrix]) assert(deVert1.numRows === 5) assert(spVert2.numRows === 5) diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/util/TestingUtilsSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/util/TestingUtilsSuite.scala index 5cbf2f04e6269..2dc0ee32d5762 100644 --- a/mllib-local/src/test/scala/org/apache/spark/ml/util/TestingUtilsSuite.scala +++ b/mllib-local/src/test/scala/org/apache/spark/ml/util/TestingUtilsSuite.scala @@ -110,9 +110,9 @@ class TestingUtilsSuite extends SparkMLFunSuite { assert(!(Vectors.dense(Array(3.1, 3.5)) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01)) assert(!(Vectors.dense(Array(3.1, 3.5)) ~= Vectors.dense(Array(3.135, 3.534)) relTol 0.01)) assert(Vectors.dense(Array(3.1)) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) - assert(Vectors.dense(Array[Double]()) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array.empty[Double]) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) assert(Vectors.dense(Array(3.1)) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) - assert(Vectors.dense(Array[Double]()) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array.empty[Double]) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) // Should throw exception with message when test fails. intercept[TestFailedException]( @@ -125,7 +125,7 @@ class TestingUtilsSuite extends SparkMLFunSuite { Vectors.dense(Array(3.1)) ~== Vectors.dense(Array(3.535, 3.534)) relTol 0.01) intercept[TestFailedException]( - Vectors.dense(Array[Double]()) ~== Vectors.dense(Array(3.135)) relTol 0.01) + Vectors.dense(Array.empty[Double]) ~== Vectors.dense(Array(3.135)) relTol 0.01) // Comparing against zero should fail the test and throw exception with message // saying that the relative error is meaningless in this situation. @@ -145,7 +145,7 @@ class TestingUtilsSuite extends SparkMLFunSuite { assert(Vectors.dense(Array(3.1)) !~== Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01) - assert(Vectors.dense(Array[Double]()) !~== + assert(Vectors.dense(Array.empty[Double]) !~== Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01) } @@ -176,14 +176,14 @@ class TestingUtilsSuite extends SparkMLFunSuite { assert(!(Vectors.dense(Array(3.1)) ~= Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5)) - assert(Vectors.dense(Array[Double]()) !~= + assert(Vectors.dense(Array.empty[Double]) !~= Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5) - assert(!(Vectors.dense(Array[Double]()) ~= + assert(!(Vectors.dense(Array.empty[Double]) ~= Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5)) - assert(Vectors.dense(Array[Double]()) ~= - Vectors.dense(Array[Double]()) absTol 1E-5) + assert(Vectors.dense(Array.empty[Double]) ~= + Vectors.dense(Array.empty[Double]) absTol 1E-5) // Should throw exception with message when test fails. intercept[TestFailedException](Vectors.dense(Array(3.1, 3.5, 0.0)) !~== @@ -195,7 +195,7 @@ class TestingUtilsSuite extends SparkMLFunSuite { intercept[TestFailedException](Vectors.dense(Array(3.1)) ~== Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7)) absTol 1E-6) - intercept[TestFailedException](Vectors.dense(Array[Double]()) ~== + intercept[TestFailedException](Vectors.dense(Array.empty[Double]) ~== Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7)) absTol 1E-6) // Comparisons of two sparse vectors @@ -214,7 +214,7 @@ class TestingUtilsSuite extends SparkMLFunSuite { assert(Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-6, 2.4)) !~== Vectors.sparse(1, Array(0), Array(3.1)) absTol 1E-3) - assert(Vectors.sparse(0, Array[Int](), Array[Double]()) !~== + assert(Vectors.sparse(0, Array.empty[Int], Array.empty[Double]) !~== Vectors.sparse(1, Array(0), Array(3.1)) absTol 1E-3) // Comparisons of a dense vector and a sparse vector @@ -230,14 +230,14 @@ class TestingUtilsSuite extends SparkMLFunSuite { assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) !~== Vectors.dense(Array(3.1)) absTol 1E-6) - assert(Vectors.dense(Array[Double]()) !~== + assert(Vectors.dense(Array.empty[Double]) !~== Vectors.sparse(3, Array(0, 2), Array(0, 2.4)) absTol 1E-6) assert(Vectors.sparse(1, Array(0), Array(3.1)) !~== Vectors.dense(Array(3.1, 3.2)) absTol 1E-6) assert(Vectors.dense(Array(3.1)) !~== - Vectors.sparse(0, Array[Int](), Array[Double]()) absTol 1E-6) + Vectors.sparse(0, Array.empty[Int], Array.empty[Double]) absTol 1E-6) } test("Comparing Matrices using absolute error.") { 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 862a468745fbd..8fdaae04c42ec 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 @@ -622,7 +622,7 @@ class LogisticRegression @Since("1.2.0") ( rawCoefficients(coefIndex) } } else { - Array[Double]() + Array.empty[Double] } val interceptVector = if (interceptsArray.nonEmpty && isMultinomial) { // The intercepts are never regularized, so we always center the mean. 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 c3de5d75f4f7d..a8b5955a7285d 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 @@ -124,7 +124,8 @@ private[stat] object KolmogorovSmirnovTest extends Logging { val pResults = partDiffs.foldLeft(initAcc) { case ((pMin, pMax, pCt), (dl, dp)) => (math.min(pMin, dl), math.max(pMax, dp), pCt + 1) } - val results = if (pResults == initAcc) Array[(Double, Double, Double)]() else Array(pResults) + val results = + if (pResults == initAcc) Array.empty[(Double, Double, Double)] else Array(pResults) results.iterator } 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 c08cb695806d0..41684d92be33a 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 @@ -51,7 +51,7 @@ class MultilayerPerceptronClassifierSuite test("Input Validation") { val mlpc = new MultilayerPerceptronClassifier() intercept[IllegalArgumentException] { - mlpc.setLayers(Array[Int]()) + mlpc.setLayers(Array.empty[Int]) } intercept[IllegalArgumentException] { mlpc.setLayers(Array[Int](1)) 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 index 5eaef9aabda50..3bb760f2ecc1d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/python/MLSerDeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/python/MLSerDeSuite.scala @@ -54,7 +54,7 @@ class MLSerDeSuite extends SparkFunSuite { assert(matrix === nm) // Test conversion for empty matrix - val empty = Array[Double]() + val empty = Array.empty[Double] val emptyMatrix = Matrices.dense(0, 0, empty) val ne = MLSerDe.loads(MLSerDe.dumps(emptyMatrix)).asInstanceOf[DenseMatrix] assert(emptyMatrix == ne) 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 499d386e66413..3bded9c01760a 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 @@ -154,10 +154,10 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { val featureSamples = Array(0, 0, 0).map(_.toDouble) val featureSamplesEmpty = Array.empty[Double] val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) - assert(splits === Array[Double]()) + assert(splits === Array.empty[Double]) val splitsEmpty = RandomForest.findSplitsForContinuousFeature(featureSamplesEmpty, fakeMetadata, 0) - assert(splitsEmpty === Array[Double]()) + assert(splitsEmpty === Array.empty[Double]) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala index 0eb839f20c003..5f85c0d65ff2d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala @@ -72,7 +72,7 @@ class PythonMLLibAPISuite extends SparkFunSuite { assert(matrix === nm) // Test conversion for empty matrix - val empty = Array[Double]() + val empty = Array.empty[Double] val emptyMatrix = Matrices.dense(0, 0, empty) val ne = SerDe.loads(SerDe.dumps(emptyMatrix)).asInstanceOf[DenseMatrix] assert(emptyMatrix == ne) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala index 8e9d910e646c9..f334be2c2ba83 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala @@ -28,7 +28,7 @@ class RankingMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { Seq( (Array(1, 6, 2, 7, 8, 3, 9, 10, 4, 5), Array(1, 2, 3, 4, 5)), (Array(4, 1, 5, 6, 2, 7, 3, 8, 9, 10), Array(1, 2, 3)), - (Array(1, 2, 3, 4, 5), Array[Int]()) + (Array(1, 2, 3, 4, 5), Array.empty[Int]) ), 2) val eps = 1.0E-5 @@ -55,7 +55,7 @@ class RankingMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { val predictionAndLabels = sc.parallelize( Seq( (Array(1, 6, 2), Array(1, 2, 3, 4, 5)), - (Array[Int](), Array(1, 2, 3)) + (Array.empty[Int], Array(1, 2, 3)) ), 2) val eps = 1.0E-5 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 d0c4dd28e14ee..563756907d201 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 @@ -289,7 +289,7 @@ class MatricesSuite extends SparkFunSuite { val spHorz2 = Matrices.horzcat(Array(spMat1, deMat2)) val spHorz3 = Matrices.horzcat(Array(deMat1, spMat2)) val deHorz1 = Matrices.horzcat(Array(deMat1, deMat2)) - val deHorz2 = Matrices.horzcat(Array[Matrix]()) + val deHorz2 = Matrices.horzcat(Array.empty[Matrix]) assert(deHorz1.numRows === 3) assert(spHorz2.numRows === 3) @@ -343,7 +343,7 @@ class MatricesSuite extends SparkFunSuite { val deVert1 = Matrices.vertcat(Array(deMat1, deMat3)) val spVert2 = Matrices.vertcat(Array(spMat1, deMat3)) val spVert3 = Matrices.vertcat(Array(deMat1, spMat3)) - val deVert2 = Matrices.vertcat(Array[Matrix]()) + val deVert2 = Matrices.vertcat(Array.empty[Matrix]) assert(deVert1.numRows === 5) assert(spVert2.numRows === 5) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala index 1aff44480aac9..3fcf1cf2c2635 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala @@ -110,9 +110,9 @@ class TestingUtilsSuite extends SparkFunSuite { assert(!(Vectors.dense(Array(3.1, 3.5)) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01)) assert(!(Vectors.dense(Array(3.1, 3.5)) ~= Vectors.dense(Array(3.135, 3.534)) relTol 0.01)) assert(Vectors.dense(Array(3.1)) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) - assert(Vectors.dense(Array[Double]()) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array.empty[Double]) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) assert(Vectors.dense(Array(3.1)) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) - assert(Vectors.dense(Array[Double]()) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array.empty[Double]) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) // Should throw exception with message when test fails. intercept[TestFailedException]( @@ -125,7 +125,7 @@ class TestingUtilsSuite extends SparkFunSuite { Vectors.dense(Array(3.1)) ~== Vectors.dense(Array(3.535, 3.534)) relTol 0.01) intercept[TestFailedException]( - Vectors.dense(Array[Double]()) ~== Vectors.dense(Array(3.135)) relTol 0.01) + Vectors.dense(Array.empty[Double]) ~== Vectors.dense(Array(3.135)) relTol 0.01) // Comparing against zero should fail the test and throw exception with message // saying that the relative error is meaningless in this situation. @@ -145,7 +145,7 @@ class TestingUtilsSuite extends SparkFunSuite { assert(Vectors.dense(Array(3.1)) !~== Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01) - assert(Vectors.dense(Array[Double]()) !~== + assert(Vectors.dense(Array.empty[Double]) !~== Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01) } @@ -176,14 +176,14 @@ class TestingUtilsSuite extends SparkFunSuite { assert(!(Vectors.dense(Array(3.1)) ~= Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5)) - assert(Vectors.dense(Array[Double]()) !~= + assert(Vectors.dense(Array.empty[Double]) !~= Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5) - assert(!(Vectors.dense(Array[Double]()) ~= + assert(!(Vectors.dense(Array.empty[Double]) ~= Vectors.dense(Array(3.1 + 1E-6, 3.5 + 2E-7)) absTol 1E-5)) - assert(Vectors.dense(Array[Double]()) ~= - Vectors.dense(Array[Double]()) absTol 1E-5) + assert(Vectors.dense(Array.empty[Double]) ~= + Vectors.dense(Array.empty[Double]) absTol 1E-5) // Should throw exception with message when test fails. intercept[TestFailedException](Vectors.dense(Array(3.1, 3.5, 0.0)) !~== @@ -195,7 +195,7 @@ class TestingUtilsSuite extends SparkFunSuite { intercept[TestFailedException](Vectors.dense(Array(3.1)) ~== Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7)) absTol 1E-6) - intercept[TestFailedException](Vectors.dense(Array[Double]()) ~== + intercept[TestFailedException](Vectors.dense(Array.empty[Double]) ~== Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7)) absTol 1E-6) // Comparisons of two sparse vectors @@ -214,7 +214,7 @@ class TestingUtilsSuite extends SparkFunSuite { assert(Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-6, 2.4)) !~== Vectors.sparse(1, Array(0), Array(3.1)) absTol 1E-3) - assert(Vectors.sparse(0, Array[Int](), Array[Double]()) !~== + assert(Vectors.sparse(0, Array.empty[Int], Array.empty[Double]) !~== Vectors.sparse(1, Array(0), Array(3.1)) absTol 1E-3) // Comparisons of a dense vector and a sparse vector @@ -230,14 +230,14 @@ class TestingUtilsSuite extends SparkFunSuite { assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) !~== Vectors.dense(Array(3.1)) absTol 1E-6) - assert(Vectors.dense(Array[Double]()) !~== + assert(Vectors.dense(Array.empty[Double]) !~== Vectors.sparse(3, Array(0, 2), Array(0, 2.4)) absTol 1E-6) assert(Vectors.sparse(1, Array(0), Array(3.1)) !~== Vectors.dense(Array(3.1, 3.2)) absTol 1E-6) assert(Vectors.dense(Array(3.1)) !~== - Vectors.sparse(0, Array[Int](), Array[Double]()) absTol 1E-6) + Vectors.sparse(0, Array.empty[Int], Array.empty[Double]) absTol 1E-6) } test("Comparing Matrices using absolute error.") { 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 fdb9fa31f09c8..26978a0482fc7 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 @@ -215,13 +215,13 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Substring(bytes, 2, 2), Array[Byte](2, 3)) checkEvaluation(Substring(bytes, 3, 2), Array[Byte](3, 4)) checkEvaluation(Substring(bytes, 4, 2), Array[Byte](4)) - checkEvaluation(Substring(bytes, 8, 2), Array[Byte]()) + checkEvaluation(Substring(bytes, 8, 2), Array.empty[Byte]) checkEvaluation(Substring(bytes, -1, 2), Array[Byte](4)) checkEvaluation(Substring(bytes, -2, 2), Array[Byte](3, 4)) checkEvaluation(Substring(bytes, -3, 2), Array[Byte](2, 3)) checkEvaluation(Substring(bytes, -4, 2), Array[Byte](1, 2)) checkEvaluation(Substring(bytes, -5, 2), Array[Byte](1)) - checkEvaluation(Substring(bytes, -8, 2), Array[Byte]()) + checkEvaluation(Substring(bytes, -8, 2), Array.empty[Byte]) } test("string substring_index function") { @@ -275,7 +275,7 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Base64(UnBase64(a)), "AQIDBA==", create_row("AQIDBA==")) checkEvaluation(Base64(b), "AQIDBA==", create_row(bytes)) - checkEvaluation(Base64(b), "", create_row(Array[Byte]())) + checkEvaluation(Base64(b), "", create_row(Array.empty[Byte])) checkEvaluation(Base64(b), null, create_row(null)) checkEvaluation(Base64(Literal.create(null, BinaryType)), null, create_row("abdef")) @@ -526,13 +526,13 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // non ascii characters are not allowed in the source code, so we disable the scalastyle. checkEvaluation(Length(Literal("a花花c")), 4, create_row(string)) // scalastyle:on - checkEvaluation(Length(Literal(bytes)), 5, create_row(Array[Byte]())) + checkEvaluation(Length(Literal(bytes)), 5, create_row(Array.empty[Byte])) checkEvaluation(Length(a), 5, create_row(string)) checkEvaluation(Length(b), 5, create_row(bytes)) checkEvaluation(Length(a), 0, create_row("")) - checkEvaluation(Length(b), 0, create_row(Array[Byte]())) + checkEvaluation(Length(b), 0, create_row(Array.empty[Byte])) checkEvaluation(Length(a), null, create_row(null)) checkEvaluation(Length(b), null, create_row(null)) 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 45db61515e9b6..586a0fffeb7a1 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 @@ -273,7 +273,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { test("sort_array function") { val df = Seq( (Array[Int](2, 1, 3), Array("b", "c", "a")), - (Array[Int](), Array[String]()), + (Array.empty[Int], Array.empty[String]), (null, null) ).toDF("a", "b") checkAnswer( From 3a237512b162d192b5503c08d121134a2dac6ff1 Mon Sep 17 00:00:00 2001 From: Alex Bozarth Date: Fri, 21 Oct 2016 11:39:32 +0100 Subject: [PATCH 18/48] [SPARK-13275][WEB UI] Visually clarified executors start time in timeline ## What changes were proposed in this pull request? Updated the Executors added/removed bubble in the time line so it's clearer where it starts. Now the bubble is left justified on the start time (still also denoted by the line) rather than center justified. ## How was this patch tested? Manually tested UI screen shot 2016-10-17 at 6 04 36 pm screen shot 2016-10-17 at 5 54 09 pm Author: Alex Bozarth Closes #15536 from ajbozarth/spark13275. --- .../main/resources/org/apache/spark/ui/static/timeline-view.js | 2 ++ 1 file changed, 2 insertions(+) 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 a6153ceda75e2..705a08f0293d3 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 @@ -24,6 +24,7 @@ function drawApplicationTimeline(groupArray, eventObjArray, startTime, offset) { return a.value - b.value }, editable: false, + align: 'left', showCurrentTime: false, min: startTime, zoomable: false, @@ -99,6 +100,7 @@ function drawJobTimeline(groupArray, eventObjArray, startTime, offset) { return a.value - b.value; }, editable: false, + align: 'left', showCurrentTime: false, min: startTime, zoomable: false, From b3b4b9542223de3495a7a7e0dd27634ddb9f929d Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 21 Oct 2016 11:25:01 -0700 Subject: [PATCH 19/48] [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/MimaExcludes.scala | 7 ++++++- project/plugins.sbt | 2 +- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index facf034ea7e7d..350b144f8294b 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -81,7 +81,12 @@ object MimaExcludes { // [SPARK-17338][SQL] add global temp view ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropGlobalTempView"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.catalog.Catalog.dropTempView"), - ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropTempView") + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.dropTempView"), + + // [SPARK-18034] Upgrade to MiMa 0.1.11 to fix flakiness. + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasAggregationDepth.aggregationDepth"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasAggregationDepth.getAggregationDepth"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasAggregationDepth.org$apache$spark$ml$param$shared$HasAggregationDepth$_setter_$aggregationDepth_=") ) } 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 4efdc764edfbc4971f0e863947258482ca2017df Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Fri, 21 Oct 2016 12:34:14 -0700 Subject: [PATCH 20/48] [SPARK-17674][SPARKR] check for warning in test output ## What changes were proposed in this pull request? testthat library we are using for testing R is redirecting warning (and disabling `options("warn" = 2)`), we need to have a way to detect any new warning and fail ## How was this patch tested? manual testing, Jenkins Author: Felix Cheung Closes #15576 from felixcheung/rtestwarning. --- R/run-tests.sh | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/R/run-tests.sh b/R/run-tests.sh index 1a1e8ab9ffe18..5e4dafaf76f3d 100755 --- a/R/run-tests.sh +++ b/R/run-tests.sh @@ -26,6 +26,8 @@ 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)) +NUM_TEST_WARNING="$(grep -c -e 'Warnings ----------------' $LOGFILE)" + # Also run the documentation tests for CRAN CRAN_CHECK_LOG_FILE=$FWDIR/cran-check.out rm -f $CRAN_CHECK_LOG_FILE @@ -37,10 +39,10 @@ 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 +if [[ $FAILED != 0 || $NUM_TEST_WARNING != 0 ]]; then cat $LOGFILE echo -en "\033[31m" # Red - echo "Had test failures; see logs." + echo "Had test warnings or failures; see logs." echo -en "\033[0m" # No color exit -1 else From e21e1c946c4b7448fb150cfa2d9419864ae6f9b5 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Fri, 21 Oct 2016 12:35:37 -0700 Subject: [PATCH 21/48] [SPARK-18013][SPARKR] add crossJoin API ## What changes were proposed in this pull request? Add crossJoin and do not default to cross join if joinExpr is left out ## How was this patch tested? unit test Author: Felix Cheung Closes #15559 from felixcheung/rcrossjoin. --- R/pkg/NAMESPACE | 1 + R/pkg/R/DataFrame.R | 59 ++++++++++++++++++----- R/pkg/R/generics.R | 4 ++ R/pkg/inst/tests/testthat/test_sparkSQL.R | 11 ++++- docs/sparkr.md | 4 ++ 5 files changed, 64 insertions(+), 15 deletions(-) diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 5960c6206a6f1..87181851714e0 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -71,6 +71,7 @@ exportMethods("arrange", "covar_samp", "covar_pop", "createOrReplaceTempView", + "crossJoin", "crosstab", "dapply", "dapplyCollect", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 801d2ed4e7500..8910a4b138a37 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -2271,12 +2271,13 @@ setMethod("dropDuplicates", #' Join #' -#' Join two SparkDataFrames based on the given join expression. +#' Joins two SparkDataFrames based on the given join expression. #' #' @param x A SparkDataFrame #' @param y A SparkDataFrame #' @param joinExpr (Optional) The expression used to perform the join. joinExpr must be a -#' Column expression. If joinExpr is omitted, join() will perform a Cartesian join +#' Column expression. If joinExpr is omitted, the default, inner join is attempted and an error is +#' thrown if it would be a Cartesian Product. For Cartesian join, use crossJoin instead. #' @param joinType The type of join to perform. The following join types are available: #' 'inner', 'outer', 'full', 'fullouter', leftouter', 'left_outer', 'left', #' 'right_outer', 'rightouter', 'right', and 'leftsemi'. The default joinType is "inner". @@ -2285,23 +2286,24 @@ setMethod("dropDuplicates", #' @aliases join,SparkDataFrame,SparkDataFrame-method #' @rdname join #' @name join -#' @seealso \link{merge} +#' @seealso \link{merge} \link{crossJoin} #' @export #' @examples #'\dontrun{ #' sparkR.session() #' 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") +#' join(df1, df2) # Attempts an inner join #' } #' @note join since 1.4.0 setMethod("join", signature(x = "SparkDataFrame", y = "SparkDataFrame"), function(x, y, joinExpr = NULL, joinType = NULL) { if (is.null(joinExpr)) { - sdf <- callJMethod(x@sdf, "crossJoin", y@sdf) + # this may not fail until the planner checks for Cartesian join later on. + sdf <- callJMethod(x@sdf, "join", y@sdf) } else { if (class(joinExpr) != "Column") stop("joinExpr must be a Column") if (is.null(joinType)) { @@ -2322,22 +2324,52 @@ setMethod("join", dataFrame(sdf) }) +#' CrossJoin +#' +#' Returns Cartesian Product on two SparkDataFrames. +#' +#' @param x A SparkDataFrame +#' @param y A SparkDataFrame +#' @return A SparkDataFrame containing the result of the join operation. +#' @family SparkDataFrame functions +#' @aliases crossJoin,SparkDataFrame,SparkDataFrame-method +#' @rdname crossJoin +#' @name crossJoin +#' @seealso \link{merge} \link{join} +#' @export +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df1 <- read.json(path) +#' df2 <- read.json(path2) +#' crossJoin(df1, df2) # Performs a Cartesian +#' } +#' @note crossJoin since 2.1.0 +setMethod("crossJoin", + signature(x = "SparkDataFrame", y = "SparkDataFrame"), + function(x, y) { + sdf <- callJMethod(x@sdf, "crossJoin", y@sdf) + dataFrame(sdf) + }) + #' Merges two data frames #' #' @name merge -#' @param x the first data frame to be joined -#' @param y the second data frame to be joined +#' @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 #' 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} #' 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 +#' 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 +#' 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}. @@ -2351,20 +2383,21 @@ setMethod("join", #' @family SparkDataFrame functions #' @aliases merge,SparkDataFrame,SparkDataFrame-method #' @rdname merge -#' @seealso \link{join} +#' @seealso \link{join} \link{crossJoin} #' @export #' @examples #'\dontrun{ #' 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", @@ -2401,7 +2434,7 @@ setMethod("merge", joinY <- by } else { # if by or both by.x and by.y have length 0, use Cartesian Product - joinRes <- join(x, y) + joinRes <- crossJoin(x, y) return (joinRes) } diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 810aea9017743..5549cd7cac516 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -468,6 +468,10 @@ setGeneric("createOrReplaceTempView", standardGeneric("createOrReplaceTempView") }) +# @rdname crossJoin +# @export +setGeneric("crossJoin", function(x, y) { standardGeneric("crossJoin") }) + #' @rdname dapply #' @export setGeneric("dapply", function(x, func, schema) { standardGeneric("dapply") }) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index 1c806869e9fbe..3a987cd86213f 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -1572,7 +1572,7 @@ test_that("filter() on a DataFrame", { #expect_true(is.ts(filter(1:100, rep(1, 3)))) # nolint }) -test_that("join() and merge() on a DataFrame", { +test_that("join(), crossJoin() and merge() on a DataFrame", { df <- read.json(jsonPath) mockLines2 <- c("{\"name\":\"Michael\", \"test\": \"yes\"}", @@ -1583,7 +1583,14 @@ test_that("join() and merge() on a DataFrame", { writeLines(mockLines2, jsonPath2) df2 <- read.json(jsonPath2) - joined <- join(df, df2) + # inner join, not cartesian join + expect_equal(count(where(join(df, df2), df$name == df2$name)), 3) + # cartesian join + expect_error(tryCatch(count(join(df, df2)), error = function(e) { stop(e) }), + paste0(".*(org.apache.spark.sql.AnalysisException: Detected cartesian product for", + " INNER join between logical plans).*")) + + joined <- crossJoin(df, df2) expect_equal(names(joined), c("age", "name", "name", "test")) expect_equal(count(joined), 12) expect_equal(names(collect(joined)), c("age", "name", "name", "test")) diff --git a/docs/sparkr.md b/docs/sparkr.md index 340e7f7cb1a0b..c1829efd18f44 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -591,3 +591,7 @@ 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` + +## Upgrading to SparkR 2.1.0 + + - `join` no longer performs Cartesian Product by default, use `crossJoin` instead. From e371040a0150e4ed748a7c25465965840b61ca63 Mon Sep 17 00:00:00 2001 From: Hossein Date: Fri, 21 Oct 2016 12:38:52 -0700 Subject: [PATCH 22/48] [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. --- 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 3a987cd86213f..b4b43fdba42ce 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -390,6 +390,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 7a531e3054f8d4820216ed379433559f57f571b8 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 21 Oct 2016 13:07:29 -0700 Subject: [PATCH 23/48] [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. --- 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 ce47bd1640fb1..35fc469291684 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 c1f344f1a09b8834bec70c1ece30b9bff63e55ea Mon Sep 17 00:00:00 2001 From: w00228970 Date: Fri, 21 Oct 2016 14:43:55 -0700 Subject: [PATCH 24/48] [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. --- .../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 0dae0e614e17d..10d55c87fb8de 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 140570252fd3739d6bdcadd6d4d5a180e480d3e0 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 21 Oct 2016 15:28:16 -0700 Subject: [PATCH 25/48] [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 92b1fff7d8127..17da606580eea 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 @@ -75,7 +75,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() @@ -186,8 +186,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 => @@ -197,14 +200,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.") @@ -217,7 +220,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) @@ -246,7 +249,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( @@ -266,7 +270,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 614a6261e7c28..115edf7ab2b61 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 3e1e1126f9e6b..4a47c04d3f084 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 268ccb9a48dfefc4d7bc85155e7e20a2dfe89307 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Fri, 21 Oct 2016 15:55:04 -0700 Subject: [PATCH 26/48] [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. --- .../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.
- - - - + + + + - + + + + + + From eff4aed1ac1e500d4aa40665dd06b527dffbc111 Mon Sep 17 00:00:00 2001 From: Tejas Patil Date: Sat, 22 Oct 2016 20:43:43 -0700 Subject: [PATCH 37/48] [SPARK-18035][SQL] Introduce performant and memory efficient APIs to create ArrayBasedMapData ## What changes were proposed in this pull request? Jira: https://issues.apache.org/jira/browse/SPARK-18035 In HiveInspectors, I saw that converting Java map to Spark's `ArrayBasedMapData` spent quite sometime in buffer copying : https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala#L658 The reason being `map.toSeq` allocates a new buffer and copies the map entries to it: https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/MapLike.scala#L323 This copy is not needed as we get rid of it once we extract the key and value arrays. Here is the call trace: ``` org.apache.spark.sql.hive.HiveInspectors$$anonfun$unwrapperFor$41.apply(HiveInspectors.scala:664) scala.collection.AbstractMap.toSeq(Map.scala:59) scala.collection.MapLike$class.toSeq(MapLike.scala:323) scala.collection.AbstractMap.toBuffer(Map.scala:59) scala.collection.MapLike$class.toBuffer(MapLike.scala:326) scala.collection.AbstractTraversable.copyToBuffer(Traversable.scala:104) scala.collection.TraversableOnce$class.copyToBuffer(TraversableOnce.scala:275) scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48) scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104) scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59) scala.collection.AbstractIterable.foreach(Iterable.scala:54) scala.collection.IterableLike$class.foreach(IterableLike.scala:72) scala.collection.AbstractIterator.foreach(Iterator.scala:1336) scala.collection.Iterator$class.foreach(Iterator.scala:893) scala.collection.generic.Growable$$anonfun$$plus$plus$eq$1.apply(Growable.scala:59) scala.collection.generic.Growable$$anonfun$$plus$plus$eq$1.apply(Growable.scala:59) ``` Also, earlier code was populating keys and values arrays separately by iterating twice. The PR avoids double iteration of the map and does it in one iteration. EDIT: During code review, there were several more places in the code which were found to do similar thing. The PR dedupes those instances and introduces convenient APIs which are performant and memory efficient ## Performance gains The number is subjective and depends on how many map columns are accessed in the query and average entries per map. For one the queries that I tried out, I saw 3% CPU savings (end-to-end) for the query. ## How was this patch tested? This does not change the end result produced so relying on existing tests. Author: Tejas Patil Closes #15573 from tejasapatil/SPARK-18035_avoid_toSeq. --- .../sql/catalyst/CatalystTypeConverters.scala | 53 +++--------- .../expressions/complexTypeCreator.scala | 32 +++++--- .../sql/catalyst/util/ArrayBasedMapData.scala | 81 ++++++++++++++++++- .../sql/execution/python/EvaluatePython.scala | 10 +-- .../spark/sql/hive/HiveInspectors.scala | 11 +-- 5 files changed, 119 insertions(+), 68 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 f542f5cf40506..5b9161551a7af 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 @@ -199,34 +199,14 @@ object CatalystTypeConverters { private[this] val keyConverter = getConverterForType(keyType) private[this] val valueConverter = getConverterForType(valueType) - override def toCatalystImpl(scalaValue: Any): MapData = scalaValue match { - case m: Map[_, _] => - val length = m.size - val convertedKeys = new Array[Any](length) - val convertedValues = new Array[Any](length) - - var i = 0 - for ((key, value) <- m) { - convertedKeys(i) = keyConverter.toCatalyst(key) - convertedValues(i) = valueConverter.toCatalyst(value) - i += 1 - } - ArrayBasedMapData(convertedKeys, convertedValues) - - case jmap: JavaMap[_, _] => - val length = jmap.size() - val convertedKeys = new Array[Any](length) - val convertedValues = new Array[Any](length) - - var i = 0 - val iter = jmap.entrySet.iterator - while (iter.hasNext) { - val entry = iter.next() - convertedKeys(i) = keyConverter.toCatalyst(entry.getKey) - convertedValues(i) = valueConverter.toCatalyst(entry.getValue) - i += 1 - } - ArrayBasedMapData(convertedKeys, convertedValues) + override def toCatalystImpl(scalaValue: Any): MapData = { + val keyFunction = (k: Any) => keyConverter.toCatalyst(k) + val valueFunction = (k: Any) => valueConverter.toCatalyst(k) + + scalaValue match { + case map: Map[_, _] => ArrayBasedMapData(map, keyFunction, valueFunction) + case javaMap: JavaMap[_, _] => ArrayBasedMapData(javaMap, keyFunction, valueFunction) + } } override def toScala(catalystValue: MapData): Map[Any, Any] = { @@ -433,18 +413,11 @@ object CatalystTypeConverters { case seq: Seq[Any] => new GenericArrayData(seq.map(convertToCatalyst).toArray) case r: Row => InternalRow(r.toSeq.map(convertToCatalyst): _*) case arr: Array[Any] => new GenericArrayData(arr.map(convertToCatalyst)) - case m: Map[_, _] => - val length = m.size - val convertedKeys = new Array[Any](length) - val convertedValues = new Array[Any](length) - - var i = 0 - for ((key, value) <- m) { - convertedKeys(i) = convertToCatalyst(key) - convertedValues(i) = convertToCatalyst(value) - i += 1 - } - ArrayBasedMapData(convertedKeys, convertedValues) + case map: Map[_, _] => + ArrayBasedMapData( + map, + (key: Any) => convertToCatalyst(key), + (value: Any) => convertToCatalyst(value)) case other => other } 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 09e22aaf3e3d8..917aa0873130b 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 @@ -427,18 +427,28 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E } } - 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 - } + override def nullSafeEval( + inputString: Any, + stringDelimiter: Any, + keyValueDelimiter: Any): Any = { + val keyValues = + inputString.asInstanceOf[UTF8String].split(stringDelimiter.asInstanceOf[UTF8String], -1) + + val iterator = new Iterator[(UTF8String, UTF8String)] { + var index = 0 + val keyValueDelimiterUTF8String = keyValueDelimiter.asInstanceOf[UTF8String] + + override def hasNext: Boolean = { + keyValues.length > index } - ArrayBasedMapData(array.map(_ (0)), array.map(_ (1))) + + override def next(): (UTF8String, UTF8String) = { + val keyValueArray = keyValues(index).split(keyValueDelimiterUTF8String, 2) + index += 1 + (keyValueArray(0), if (keyValueArray.length < 2) null else keyValueArray(1)) + } + } + ArrayBasedMapData(iterator, keyValues.size, identity, identity) } override def prettyName: String = "str_to_map" 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 4449da13c083c..91b3139443696 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.util +import java.util.{Map => JavaMap} + class ArrayBasedMapData(val keyArray: ArrayData, val valueArray: ArrayData) extends MapData { require(keyArray.numElements() == valueArray.numElements()) @@ -30,12 +32,83 @@ class ArrayBasedMapData(val keyArray: ArrayData, val valueArray: ArrayData) exte } object ArrayBasedMapData { - def apply(map: Map[Any, Any]): ArrayBasedMapData = { - val array = map.toArray - ArrayBasedMapData(array.map(_._1), array.map(_._2)) + /** + * Creates a [[ArrayBasedMapData]] by applying the given converters over + * each (key -> value) pair of the input [[java.util.Map]] + * + * @param javaMap Input map + * @param keyConverter This function is applied over all the keys of the input map to + * obtain the output map's keys + * @param valueConverter This function is applied over all the values of the input map to + * obtain the output map's values + */ + def apply( + javaMap: JavaMap[_, _], + keyConverter: (Any) => Any, + valueConverter: (Any) => Any): ArrayBasedMapData = { + import scala.language.existentials + + val keys: Array[Any] = new Array[Any](javaMap.size()) + val values: Array[Any] = new Array[Any](javaMap.size()) + + var i: Int = 0 + val iterator = javaMap.entrySet().iterator() + while (iterator.hasNext) { + val entry = iterator.next() + keys(i) = keyConverter(entry.getKey) + values(i) = valueConverter(entry.getValue) + i += 1 + } + ArrayBasedMapData(keys, values) + } + + /** + * Creates a [[ArrayBasedMapData]] by applying the given converters over + * each (key -> value) pair of the input map + * + * @param map Input map + * @param keyConverter This function is applied over all the keys of the input map to + * obtain the output map's keys + * @param valueConverter This function is applied over all the values of the input map to + * obtain the output map's values + */ + def apply( + map: scala.collection.Map[_, _], + keyConverter: (Any) => Any = identity, + valueConverter: (Any) => Any = identity): ArrayBasedMapData = { + ArrayBasedMapData(map.iterator, map.size, keyConverter, valueConverter) + } + + /** + * Creates a [[ArrayBasedMapData]] by applying the given converters over + * each (key -> value) pair from the given iterator + * + * @param iterator Input iterator + * @param size Number of elements + * @param keyConverter This function is applied over all the keys extracted from the + * given iterator to obtain the output map's keys + * @param valueConverter This function is applied over all the values extracted from the + * given iterator to obtain the output map's values + */ + def apply( + iterator: Iterator[(_, _)], + size: Int, + keyConverter: (Any) => Any, + valueConverter: (Any) => Any): ArrayBasedMapData = { + + val keys: Array[Any] = new Array[Any](size) + val values: Array[Any] = new Array[Any](size) + + var i = 0 + for ((key, value) <- iterator) { + keys(i) = keyConverter(key) + values(i) = valueConverter(value) + i += 1 + } + ArrayBasedMapData(keys, values) } - def apply(keys: Array[Any], values: Array[Any]): ArrayBasedMapData = { + def apply(keys: Array[_], values: Array[_]): ArrayBasedMapData = { new ArrayBasedMapData(new GenericArrayData(keys), new GenericArrayData(values)) } 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 724025b4647f4..46fd54e5c7420 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 @@ -124,11 +124,11 @@ object EvaluatePython { case (c, ArrayType(elementType, _)) if c.getClass.isArray => new GenericArrayData(c.asInstanceOf[Array[_]].map(e => fromJava(e, elementType))) - case (c: java.util.Map[_, _], MapType(keyType, valueType, _)) => - val keyValues = c.asScala.toSeq - val keys = keyValues.map(kv => fromJava(kv._1, keyType)).toArray - val values = keyValues.map(kv => fromJava(kv._2, valueType)).toArray - ArrayBasedMapData(keys, values) + case (javaMap: java.util.Map[_, _], MapType(keyType, valueType, _)) => + ArrayBasedMapData( + javaMap, + (key: Any) => fromJava(key, keyType), + (value: Any) => fromJava(value, valueType)) case (c, StructType(fields)) if c.getClass.isArray => val array = c.asInstanceOf[Array[_]] 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 1625116803505..e303065127c3b 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 @@ -473,10 +473,8 @@ private[hive] trait HiveInspectors { case mi: StandardConstantMapObjectInspector => val keyUnwrapper = unwrapperFor(mi.getMapKeyObjectInspector) val valueUnwrapper = unwrapperFor(mi.getMapValueObjectInspector) - val keyValues = mi.getWritableConstantValue.asScala.toSeq - val keys = keyValues.map(kv => keyUnwrapper(kv._1)).toArray - val values = keyValues.map(kv => valueUnwrapper(kv._2)).toArray - val constant = ArrayBasedMapData(keys, values) + val keyValues = mi.getWritableConstantValue + val constant = ArrayBasedMapData(keyValues, keyUnwrapper, valueUnwrapper) _ => constant case li: StandardConstantListObjectInspector => val unwrapper = unwrapperFor(li.getListElementObjectInspector) @@ -655,10 +653,7 @@ private[hive] trait HiveInspectors { if (map == null) { null } else { - val keyValues = map.asScala.toSeq - val keys = keyValues.map(kv => keyUnwrapper(kv._1)).toArray - val values = keyValues.map(kv => valueUnwrapper(kv._2)).toArray - ArrayBasedMapData(keys, values) + ArrayBasedMapData(map, keyUnwrapper, valueUnwrapper) } } else { null From 21c7539a5274a7e77686d17a6261d56592b85c2d Mon Sep 17 00:00:00 2001 From: Tejas Patil Date: Sun, 23 Oct 2016 13:25:47 +0200 Subject: [PATCH 38/48] [SPARK-18038][SQL] Move output partitioning definition from UnaryNodeExec to its children ## What changes were proposed in this pull request? Jira : https://issues.apache.org/jira/browse/SPARK-18038 This was a suggestion by rxin over one of the dev list discussion : http://apache-spark-developers-list.1001551.n3.nabble.com/Project-not-preserving-child-partitioning-td19417.html His words: >> It would be better (safer) to move the output partitioning definition into each of the operator and remove it from UnaryExecNode. With this PR, following is the output partitioning and ordering for all the impls of `UnaryExecNode`. UnaryExecNode's impl | outputPartitioning | outputOrdering | comment ------------ | ------------- | ------------ | ------------ AppendColumnsExec | child's | Nil | child's ordering can be used AppendColumnsWithObjectExec | child's | Nil | child's ordering can be used BroadcastExchangeExec | BroadcastPartitioning | Nil | - CoalesceExec | UnknownPartitioning | Nil | - CollectLimitExec | SinglePartition | Nil | - DebugExec | child's | Nil | child's ordering can be used DeserializeToObjectExec | child's | Nil | child's ordering can be used ExpandExec | UnknownPartitioning | Nil | - FilterExec | child's | child's | - FlatMapGroupsInRExec | child's | Nil | child's ordering can be used GenerateExec | child's | Nil | need to dig more GlobalLimitExec | child's | child's | - HashAggregateExec | child's | Nil | - InputAdapter | child's | child's | - InsertIntoHiveTable | child's | Nil | terminal node, doesn't need partitioning LocalLimitExec | child's | child's | - MapElementsExec | child's | child's | - MapGroupsExec | child's | Nil | child's ordering can be used MapPartitionsExec | child's | Nil | child's ordering can be used ProjectExec | child's | child's | - SampleExec | child's | Nil | child's ordering can be used ScriptTransformation | child's | Nil | child's ordering can be used SerializeFromObjectExec | child's | Nil | child's ordering can be used ShuffleExchange | custom | Nil | - SortAggregateExec | child's | sort over grouped exprs | - SortExec | child's | custom | - StateStoreRestoreExec | child's | Nil | child's ordering can be used StateStoreSaveExec | child's | Nil | child's ordering can be used SubqueryExec | child's | child's | - TakeOrderedAndProjectExec | SinglePartition | custom | - WholeStageCodegenExec | child's | child's | - WindowExec | child's | child's | - ## How was this patch tested? This does NOT change any existing functionality so relying on existing tests Author: Tejas Patil Closes #15575 from tejasapatil/SPARK-18038_UnaryNodeExec_output_partitioning. --- .../spark/sql/execution/GenerateExec.scala | 3 +++ .../apache/spark/sql/execution/SortExec.scala | 6 +++++- .../apache/spark/sql/execution/SparkPlan.scala | 2 -- .../sql/execution/WholeStageCodegenExec.scala | 4 ++++ .../execution/aggregate/HashAggregateExec.scala | 2 ++ .../execution/aggregate/SortAggregateExec.scala | 4 +++- .../sql/execution/basicPhysicalOperators.scala | 8 ++++++++ .../spark/sql/execution/debug/package.scala | 4 +++- .../org/apache/spark/sql/execution/limit.scala | 16 +++++++++++----- .../apache/spark/sql/execution/objects.scala | 17 +++++++++++++++++ .../execution/streaming/StatefulAggregate.scala | 6 ++++++ .../spark/sql/execution/window/WindowExec.scala | 2 ++ .../spark/sql/execution/ReferenceSort.scala | 2 ++ .../hive/execution/InsertIntoHiveTable.scala | 4 +++- .../hive/execution/ScriptTransformation.scala | 3 +++ .../execution/ScriptTransformationSuite.scala | 4 ++++ 16 files changed, 76 insertions(+), 11 deletions(-) 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 39189a2b0c72c..2663129562660 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution 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.plans.physical.Partitioning import org.apache.spark.sql.execution.metric.SQLMetrics /** @@ -60,6 +61,8 @@ case class GenerateExec( override def producedAttributes: AttributeSet = AttributeSet(output) + override def outputPartitioning: Partitioning = child.outputPartitioning + val boundGenerator = BindReferences.bindReference(generator, child.output) protected override def doExecute(): RDD[InternalRow] = { 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 d8e0675e3eb65..cc576bbc4c802 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 @@ -23,7 +23,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.{CodegenContext, ExprCode} -import org.apache.spark.sql.catalyst.plans.physical.{Distribution, OrderedDistribution, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.metric.SQLMetrics /** @@ -45,6 +45,10 @@ case class SortExec( override def outputOrdering: Seq[SortOrder] = sortOrder + // sort performed is local within a given partition so will retain + // child operator's partitioning + override def outputPartitioning: Partitioning = child.outputPartitioning + override def requiredChildDistribution: Seq[Distribution] = if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil 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 48d6ef6dcd44a..24d0cffef82a2 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 @@ -395,8 +395,6 @@ trait UnaryExecNode extends SparkPlan { def child: SparkPlan override final def children: Seq[SparkPlan] = child :: Nil - - override def outputPartitioning: Partitioning = child.outputPartitioning } trait BinaryExecNode extends SparkPlan { 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 62bf6f4a81eec..6303483f22fd3 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 @@ -218,7 +218,9 @@ trait CodegenSupport extends SparkPlan { case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupport { override def output: Seq[Attribute] = child.output + override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = child.outputOrdering override def doExecute(): RDD[InternalRow] = { @@ -292,7 +294,9 @@ object WholeStageCodegenExec { case class WholeStageCodegenExec(child: SparkPlan) extends UnaryExecNode with CodegenSupport { override def output: Seq[Attribute] = child.output + override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = child.outputOrdering override lazy val metrics = Map( 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 06199ef3e8243..4529ed067e565 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 @@ -63,6 +63,8 @@ case class HashAggregateExec( override def output: Seq[Attribute] = resultExpressions.map(_.toAttribute) + override def outputPartitioning: Partitioning = child.outputPartitioning + override def producedAttributes: AttributeSet = AttributeSet(aggregateAttributes) ++ AttributeSet(resultExpressions.diff(groupingExpressions).map(_.toAttribute)) ++ 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 2a81a823c44b3..be3198b8e7d82 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 @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ 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.catalyst.plans.physical._ import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.util.Utils @@ -66,6 +66,8 @@ case class SortAggregateExec( groupingExpressions.map(SortOrder(_, Ascending)) :: Nil } + override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = { groupingExpressions.map(SortOrder(_, Ascending)) } 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 dd78a784915d2..37d750e621c25 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 @@ -78,6 +78,8 @@ case class ProjectExec(projectList: Seq[NamedExpression], child: SparkPlan) } override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + override def outputPartitioning: Partitioning = child.outputPartitioning } @@ -214,6 +216,8 @@ case class FilterExec(condition: Expression, child: SparkPlan) } override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + override def outputPartitioning: Partitioning = child.outputPartitioning } /** @@ -234,6 +238,8 @@ case class SampleExec( child: SparkPlan) extends UnaryExecNode with CodegenSupport { override def output: Seq[Attribute] = child.output + override def outputPartitioning: Partitioning = child.outputPartitioning + override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) @@ -517,7 +523,9 @@ case class SubqueryExec(name: String, child: SparkPlan) extends UnaryExecNode { "collectTime" -> SQLMetrics.createMetric(sparkContext, "time to collect (ms)")) override def output: Seq[Attribute] = child.output + override def outputPartitioning: Partitioning = child.outputPartitioning + override def outputOrdering: Seq[SortOrder] = child.outputOrdering override def sameResult(o: SparkPlan): Boolean = o match { 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 dd9d83767e221..0395c43ba2cbc 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 @@ -27,8 +27,8 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow 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.plans.physical.Partitioning import org.apache.spark.sql.catalyst.trees.TreeNodeRef -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.{AccumulatorV2, LongAccumulator} /** @@ -162,6 +162,8 @@ package object debug { } } + override def outputPartitioning: Partitioning = child.outputPartitioning + override def inputRDDs(): Seq[RDD[InternalRow]] = { child.asInstanceOf[CodegenSupport].inputRDDs() } 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 86a8770715600..9918ac327f2dd 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 @@ -26,7 +26,6 @@ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.util.Utils - /** * Take the first `limit` elements and collect them to a single partition. * @@ -54,8 +53,7 @@ case class CollectLimitExec(limit: Int, child: SparkPlan) extends UnaryExecNode trait BaseLimitExec extends UnaryExecNode with CodegenSupport { val limit: Int override def output: Seq[Attribute] = child.output - override def outputOrdering: Seq[SortOrder] = child.outputOrdering - override def outputPartitioning: Partitioning = child.outputPartitioning + protected override def doExecute(): RDD[InternalRow] = child.execute().mapPartitions { iter => iter.take(limit) } @@ -95,14 +93,22 @@ trait BaseLimitExec extends UnaryExecNode with CodegenSupport { * Take the first `limit` elements of each child partition, but do not collect or shuffle them. */ case class LocalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec { + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + override def outputPartitioning: Partitioning = child.outputPartitioning } /** * Take the first `limit` elements of the child's single output partition. */ case class GlobalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec { + override def requiredChildDistribution: List[Distribution] = AllTuples :: Nil + + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering } /** @@ -122,8 +128,6 @@ case class TakeOrderedAndProjectExec( projectList.map(_.toAttribute) } - override def outputPartitioning: Partitioning = SinglePartition - override def executeCollect(): Array[InternalRow] = { val ord = new LazilyGeneratedOrdering(sortOrder, child.output) val data = child.execute().map(_.copy()).takeOrdered(limit)(ord) @@ -160,6 +164,8 @@ case class TakeOrderedAndProjectExec( override def outputOrdering: Seq[SortOrder] = sortOrder + override def outputPartitioning: Partitioning = SinglePartition + override def simpleString: String = { val orderByString = Utils.truncatedString(sortOrder, "[", ",", "]") val outputString = Utils.truncatedString(output, "[", ",", "]") 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 2acc5110e8950..9df56bbf1ef87 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 @@ -68,6 +68,8 @@ case class DeserializeToObjectExec( outputObjAttr: Attribute, child: SparkPlan) extends UnaryExecNode with ObjectProducerExec with CodegenSupport { + override def outputPartitioning: Partitioning = child.outputPartitioning + override def inputRDDs(): Seq[RDD[InternalRow]] = { child.asInstanceOf[CodegenSupport].inputRDDs() } @@ -102,6 +104,8 @@ case class SerializeFromObjectExec( override def output: Seq[Attribute] = serializer.map(_.toAttribute) + override def outputPartitioning: Partitioning = child.outputPartitioning + override def inputRDDs(): Seq[RDD[InternalRow]] = { child.asInstanceOf[CodegenSupport].inputRDDs() } @@ -171,6 +175,8 @@ case class MapPartitionsExec( child: SparkPlan) extends ObjectConsumerExec with ObjectProducerExec { + override def outputPartitioning: Partitioning = child.outputPartitioning + override protected def doExecute(): RDD[InternalRow] = { child.execute().mapPartitionsInternal { iter => val getObject = ObjectOperator.unwrapObjectFromRow(child.output.head.dataType) @@ -231,6 +237,8 @@ case class MapElementsExec( } override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + override def outputPartitioning: Partitioning = child.outputPartitioning } /** @@ -244,6 +252,8 @@ case class AppendColumnsExec( override def output: Seq[Attribute] = child.output ++ serializer.map(_.toAttribute) + override def outputPartitioning: Partitioning = child.outputPartitioning + private def newColumnSchema = serializer.map(_.toAttribute).toStructType override protected def doExecute(): RDD[InternalRow] = { @@ -272,6 +282,8 @@ case class AppendColumnsWithObjectExec( override def output: Seq[Attribute] = (inputSerializer ++ newColumnsSerializer).map(_.toAttribute) + override def outputPartitioning: Partitioning = child.outputPartitioning + private def inputSchema = inputSerializer.map(_.toAttribute).toStructType private def newColumnSchema = newColumnsSerializer.map(_.toAttribute).toStructType @@ -304,6 +316,8 @@ case class MapGroupsExec( outputObjAttr: Attribute, child: SparkPlan) extends UnaryExecNode with ObjectProducerExec { + override def outputPartitioning: Partitioning = child.outputPartitioning + override def requiredChildDistribution: Seq[Distribution] = ClusteredDistribution(groupingAttributes) :: Nil @@ -347,6 +361,9 @@ case class FlatMapGroupsInRExec( child: SparkPlan) extends UnaryExecNode with ObjectProducerExec { override def output: Seq[Attribute] = outputObjAttr :: Nil + + override def outputPartitioning: Partitioning = child.outputPartitioning + override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) override def requiredChildDistribution: Seq[Distribution] = 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 587ea7d02acab..ad8238f189c64 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 @@ -22,6 +22,7 @@ import org.apache.spark.sql.catalyst.InternalRow 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.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.streaming.state._ @@ -80,7 +81,10 @@ case class StateStoreRestoreExec( } } } + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = child.outputPartitioning } /** @@ -116,6 +120,8 @@ case class StateStoreSaveExec( override def output: Seq[Attribute] = child.output + override def outputPartitioning: Partitioning = child.outputPartitioning + /** * 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 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala index 1dd281ebf1034..80b87d5ffa797 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala @@ -103,6 +103,8 @@ case class WindowExec( override def outputOrdering: Seq[SortOrder] = child.outputOrdering + override def outputPartitioning: Partitioning = child.outputPartitioning + /** * Create a bound ordering object for a given frame type and offset. A bound ordering object is * used to determine which input row lies within the frame boundaries of an output row. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala index a19ea51af7c01..6abcb1f067968 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReferenceSort.scala @@ -57,4 +57,6 @@ case class ReferenceSort( override def output: Seq[Attribute] = child.output override def outputOrdering: Seq[SortOrder] = sortOrder + + override def outputPartitioning: Partitioning = child.outputPartitioning } 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 53bb3b93db738..c3c4e2925b90c 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 @@ -20,7 +20,6 @@ package org.apache.spark.sql.hive.execution import java.io.IOException import java.net.URI import java.text.SimpleDateFormat -import java.util import java.util.{Date, Random} import scala.collection.JavaConverters._ @@ -36,6 +35,7 @@ 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.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.HiveShim.{ShimFileSinkDesc => FileSinkDesc} @@ -291,6 +291,8 @@ case class InsertIntoHiveTable( Seq.empty[InternalRow] } + override def outputPartitioning: Partitioning = child.outputPartitioning + override def executeCollect(): Array[InternalRow] = sideEffectResult.toArray protected override def doExecute(): RDD[InternalRow] = { 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 1025b8f70d9ff..50855e48bc8fe 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 @@ -38,6 +38,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.ScriptInputOutputSchema +import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution._ import org.apache.spark.sql.hive.HiveInspectors import org.apache.spark.sql.hive.HiveShim._ @@ -61,6 +62,8 @@ case class ScriptTransformation( override def producedAttributes: AttributeSet = outputSet -- inputSet + override def outputPartitioning: Partitioning = child.outputPartitioning + protected override def doExecute(): RDD[InternalRow] = { def processIterator(inputIterator: Iterator[InternalRow], hadoopConf: Configuration) : Iterator[InternalRow] = { 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 a8e81d7a3c42a..0e837766e2ea4 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 @@ -24,6 +24,7 @@ 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} +import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest, UnaryExecNode} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.types.StringType @@ -135,5 +136,8 @@ private case class ExceptionInjectingOperator(child: SparkPlan) extends UnaryExe throw new IllegalArgumentException("intentional exception") } } + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = child.outputPartitioning } From b158256c2e719edde3dbdfe27a9a65cd3b3039f4 Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Sun, 23 Oct 2016 13:28:35 +0200 Subject: [PATCH 39/48] [SPARK-18045][SQL][TESTS] Move `HiveDataFrameAnalyticsSuite` to package `sql` ## What changes were proposed in this pull request? The testsuite `HiveDataFrameAnalyticsSuite` has nothing to do with HIVE, we should move it to package `sql`. The original test cases in that suite are splited into two existing testsuites: `DataFrameAggregateSuite` tests for the functions and ~~`SQLQuerySuite`~~`SQLQueryTestSuite` tests for the SQL statements. ## How was this patch tested? ~~Modified `SQLQuerySuite` in package `sql`.~~ Add query file for `SQLQueryTestSuite`. Author: jiangxingbo Closes #15582 from jiangxb1987/group-analytics-test. --- .../sql-tests/inputs/group-analytics.sql | 13 +++ .../sql-tests/results/group-analytics.sql.out | 87 +++++++++++++++++++ .../hive/HiveDataFrameAnalyticsSuite.scala | 72 --------------- 3 files changed, 100 insertions(+), 72 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/group-analytics.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out delete mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-analytics.sql b/sql/core/src/test/resources/sql-tests/inputs/group-analytics.sql new file mode 100644 index 0000000000000..2f783495ddf96 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/group-analytics.sql @@ -0,0 +1,13 @@ +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData(a, b); + +-- CUBE on overlapping columns +SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH CUBE; + +SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH CUBE; + +-- ROLLUP on overlapping columns +SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP; + +SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH ROLLUP; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out new file mode 100644 index 0000000000000..8ea7de809d19d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/group-analytics.sql.out @@ -0,0 +1,87 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 5 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData(a, b) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH CUBE +-- !query 1 schema +struct<(a + b):int,b:int,sum((a - b)):bigint> +-- !query 1 output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL 1 3 +NULL 2 0 +NULL NULL 3 + + + +-- !query 2 +SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH CUBE +-- !query 2 schema +struct +-- !query 2 output +1 1 1 +1 2 2 +1 NULL 3 +2 1 1 +2 2 2 +2 NULL 3 +3 1 1 +3 2 2 +3 NULL 3 +NULL 1 3 +NULL 2 6 +NULL NULL 9 + + +-- !query 3 +SELECT a + b, b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP +-- !query 3 schema +struct<(a + b):int,b:int,sum((a - b)):bigint> +-- !query 3 output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL NULL 3 + + +-- !query 4 +SELECT a, b, SUM(b) FROM testData GROUP BY a, b WITH ROLLUP +-- !query 4 schema +struct +-- !query 4 output +1 1 1 +1 2 2 +1 NULL 3 +2 1 1 +2 2 2 +2 NULL 3 +3 1 1 +3 2 2 +3 NULL 3 +NULL NULL 9 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 deleted file mode 100644 index 6477974fe713a..0000000000000 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala +++ /dev/null @@ -1,72 +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.scalatest.BeforeAndAfterAll - -import org.apache.spark.sql.{DataFrame, QueryTest, Row} -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.hive.test.TestHiveSingleton - -// TODO ideally we should put the test suite into the package `sql`, as -// `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 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") - testData.createOrReplaceTempView("mytable") - } - - override def afterAll(): Unit = { - try { - spark.catalog.dropTempView("mytable") - } finally { - super.afterAll() - } - } - - test("rollup") { - checkAnswer( - testData.rollup($"a" + $"b", $"b").agg(sum($"a" - $"b")), - sql("select a + b, b, sum(a - b) from mytable group by a + b, b with rollup").collect() - ) - - checkAnswer( - testData.rollup("a", "b").agg(sum("b")), - sql("select a, b, sum(b) from mytable group by a, b with rollup").collect() - ) - } - - test("cube") { - checkAnswer( - testData.cube($"a" + $"b", $"b").agg(sum($"a" - $"b")), - sql("select a + b, b, sum(a - b) from mytable group by a + b, b with cube").collect() - ) - - checkAnswer( - testData.cube("a", "b").agg(sum("b")), - sql("select a, b, sum(b) from mytable group by a, b with cube").collect() - ) - } -} From a81fba048fabcd413730548ab65955802508d4e4 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Sun, 23 Oct 2016 19:42:11 +0200 Subject: [PATCH 40/48] [SPARK-18058][SQL] 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 #15595 from CodingCat/SPARK-18058. --- .../sql/catalyst/analysis/CheckAnalysis.scala | 3 +- .../plans/logical/basicLogicalOperators.scala | 30 +++++++------------ .../sql/catalyst/analysis/AnalysisSuite.scala | 19 ++++++++++++ 3 files changed, 31 insertions(+), 21 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 9c06069f24f76..9a7c2a944b588 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 @@ -287,7 +287,8 @@ trait CheckAnalysis extends PredicateHelper { } // Check if the data types match. dataTypes(child).zip(ref).zipWithIndex.foreach { case ((dt1, dt2), ci) => - if (dt1 != dt2) { + // SPARK-18058: we shall not care about the nullability of columns + if (dt1.asNullable != dt2.asNullable) { failAnalysis( s""" |${operator.nodeName} can only be performed on tables with the compatible 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 d2d33e40a8c8f..64a787a7ae351 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 @@ -117,6 +117,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] = { @@ -126,6 +128,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 { @@ -134,8 +143,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) @@ -144,14 +151,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 @@ -172,19 +171,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() } @@ -219,9 +210,8 @@ 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 50ebad25cd258..590774c043040 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,23 @@ 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)()) + + val unionPlan = Union(firstTable, secondTable) + assertAnalysisSuccess(unionPlan) + + val r1 = Except(firstTable, secondTable) + val r2 = Intersect(firstTable, secondTable) + + assertAnalysisSuccess(r1) + assertAnalysisSuccess(r2) + } } From 3a423f5a0373de87ddfb4744852b2fda14fcc3cb Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Sun, 23 Oct 2016 10:53:27 -0700 Subject: [PATCH 41/48] [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. (cherry picked from commit 0e0d83a597885ab1773cb69d6dcc10346d6976a3) Signed-off-by: Felix Cheung --- R/pkg/R/DataFrame.R | 2 +- R/pkg/inst/tests/testthat/test_sparkSQL.R | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 8910a4b138a37..b6ce838969a44 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -365,7 +365,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)) diff --git a/R/pkg/inst/tests/testthat/test_sparkSQL.R b/R/pkg/inst/tests/testthat/test_sparkSQL.R index b4b43fdba42ce..e77dbde44ee66 100644 --- a/R/pkg/inst/tests/testthat/test_sparkSQL.R +++ b/R/pkg/inst/tests/testthat/test_sparkSQL.R @@ -845,7 +845,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 c64a8ff39794d60c596c0d34130019c09c9c8012 Mon Sep 17 00:00:00 2001 From: Zheng RuiFeng Date: Mon, 24 Oct 2016 10:25:24 +0100 Subject: [PATCH 42/48] [SPARK-18049][MLLIB][TEST] Add missing tests for truePositiveRate and weightedTruePositiveRate ## What changes were proposed in this pull request? Add missing tests for `truePositiveRate` and `weightedTruePositiveRate` in `MulticlassMetricsSuite` ## How was this patch tested? added testing Author: Zheng RuiFeng Closes #15585 from zhengruifeng/mc_missing_test. --- .../api/python/WriteInputFormatTestDataGenerator.scala | 2 +- .../main/scala/org/apache/spark/ml/util/ReadWrite.scala | 2 +- .../apache/spark/mllib/evaluation/RegressionMetrics.scala | 2 +- .../spark/mllib/linalg/distributed/BlockMatrix.scala | 4 ++-- .../spark/mllib/evaluation/MulticlassMetricsSuite.scala | 8 ++++++++ .../spark/mllib/evaluation/MultilabelMetricsSuite.scala | 2 +- 6 files changed, 14 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala index 34cb7c61d7034..86965dbc2e778 100644 --- a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala +++ b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala @@ -144,7 +144,7 @@ object WriteInputFormatTestDataGenerator { // Create test data for ArrayWritable val data = Seq( - (1, Array()), + (1, Array.empty[Double]), (2, Array(3.0, 4.0, 5.0)), (3, Array(4.0, 5.0, 6.0)) ) 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 4413fefdea3ca..bc4f9e6716ee8 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 @@ -474,7 +474,7 @@ private[ml] object MetaAlgorithmReadWrite { case ovr: OneVsRest => Array(ovr.getClassifier) case ovrModel: OneVsRestModel => Array(ovrModel.getClassifier) ++ ovrModel.models case rformModel: RFormulaModel => Array(rformModel.pipelineModel) - case _: Params => Array() + case _: Params => Array.empty[Params] } val subStageMaps = subStages.flatMap(getUidMapImpl) List((instance.uid, instance)) ++ subStageMaps diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala index ce4421515126c..8f777cc35b93f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala @@ -73,7 +73,7 @@ class RegressionMetrics @Since("2.0.0") ( /** * Returns the variance explained by regression. - * explainedVariance = $\sum_i (\hat{y_i} - \bar{y})^2 / n$ + * explainedVariance = $\sum_i (\hat{y_i} - \bar{y})^2^ / n$ * @see [[https://en.wikipedia.org/wiki/Fraction_of_variance_unexplained]] */ @Since("1.2.0") 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 ff1068417d94f..377be6bfb9886 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 @@ -429,14 +429,14 @@ class BlockMatrix @Since("1.3.0") ( val rightCounterpartsHelper = rightMatrix.groupBy(_._1).mapValues(_.map(_._2)) val leftDestinations = leftMatrix.map { case (rowIndex, colIndex) => - val rightCounterparts = rightCounterpartsHelper.getOrElse(colIndex, Array()) + val rightCounterparts = rightCounterpartsHelper.getOrElse(colIndex, Array.empty[Int]) 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 = leftCounterpartsHelper.getOrElse(rowIndex, Array()) + val leftCounterparts = leftCounterpartsHelper.getOrElse(rowIndex, Array.empty[Int]) val partitions = leftCounterparts.map(b => partitioner.getPartition((b, colIndex))) ((rowIndex, colIndex), partitions.toSet) }.toMap 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 f316c67234f18..142d1e9812ef1 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 @@ -36,6 +36,9 @@ class MulticlassMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { (1.0, 1.0), (1.0, 1.0), (2.0, 2.0), (2.0, 0.0)), 2) val metrics = new MulticlassMetrics(predictionAndLabels) val delta = 0.0000001 + val tpRate0 = 2.0 / (2 + 2) + val tpRate1 = 3.0 / (3 + 1) + val tpRate2 = 1.0 / (1 + 0) val fpRate0 = 1.0 / (9 - 4) val fpRate1 = 1.0 / (9 - 4) val fpRate2 = 1.0 / (9 - 1) @@ -53,6 +56,9 @@ class MulticlassMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { val f2measure2 = (1 + 2 * 2) * precision2 * recall2 / (2 * 2 * precision2 + recall2) assert(metrics.confusionMatrix.toArray.sameElements(confusionMatrix.toArray)) + assert(math.abs(metrics.truePositiveRate(0.0) - tpRate0) < delta) + assert(math.abs(metrics.truePositiveRate(1.0) - tpRate1) < delta) + assert(math.abs(metrics.truePositiveRate(2.0) - tpRate2) < delta) assert(math.abs(metrics.falsePositiveRate(0.0) - fpRate0) < delta) assert(math.abs(metrics.falsePositiveRate(1.0) - fpRate1) < delta) assert(math.abs(metrics.falsePositiveRate(2.0) - fpRate2) < delta) @@ -75,6 +81,8 @@ class MulticlassMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { 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.weightedTruePositiveRate - + ((4.0 / 9) * tpRate0 + (4.0 / 9) * tpRate1 + (1.0 / 9) * tpRate2)) < delta) assert(math.abs(metrics.weightedFalsePositiveRate - ((4.0 / 9) * fpRate0 + (4.0 / 9) * fpRate1 + (1.0 / 9) * fpRate2)) < delta) assert(math.abs(metrics.weightedPrecision - diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala index f3b19aeb42f84..a660492c7ae59 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala @@ -47,7 +47,7 @@ class MultilabelMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { val scoreAndLabels: RDD[(Array[Double], Array[Double])] = sc.parallelize( Seq((Array(0.0, 1.0), Array(0.0, 2.0)), (Array(0.0, 2.0), Array(0.0, 1.0)), - (Array(), Array(0.0)), + (Array.empty[Double], Array(0.0)), (Array(2.0), Array(2.0)), (Array(2.0, 0.0), Array(2.0, 0.0)), (Array(0.0, 1.0, 2.0), Array(0.0, 1.0)), From 4ecbe1b92f4c4c5b2d734895c09d8ded0ed48d4d Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 24 Oct 2016 10:44:45 +0100 Subject: [PATCH 43/48] [SPARK-17810][SQL] Default spark.sql.warehouse.dir is relative to local FS but can resolve as HDFS path ## What changes were proposed in this pull request? Always resolve spark.sql.warehouse.dir as a local path, and as relative to working dir not home dir ## How was this patch tested? Existing tests. Author: Sean Owen Closes #15382 from srowen/SPARK-17810. --- 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 | 23 ++++++------- .../spark/sql/internal/SQLConfSuite.scala | 6 ++-- .../sql/hive/execution/HiveQuerySuite.scala | 4 ++- .../spark/sql/sources/BucketedReadSuite.scala | 5 +-- 9 files changed, 29 insertions(+), 51 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index d334a86bc73d7..064af41965b70 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -904,50 +904,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 98b48908b5a12..ad83fe1cf14b5 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 a6e2fa26cb5ef..f47ec7f3963a4 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 a6da8a86c1623..d593bfb4ce19a 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 @@ -43,8 +43,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { // drop all databases, tables and functions after each test spark.sessionState.catalog.reset() } finally { - val path = System.getProperty("user.dir") + "/spark-warehouse" - Utils.deleteRecursively(new File(path)) + Utils.deleteRecursively(new File("spark-warehouse")) super.afterEach() } } @@ -116,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 @@ -148,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`") @@ -159,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, "", @@ -184,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, "", @@ -204,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) @@ -227,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`") @@ -237,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, "", @@ -476,7 +473,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`") @@ -484,7 +481,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 df640ffab91de..a89a43fa1e777 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 @@ -19,11 +19,11 @@ package org.apache.spark.sql.internal import org.apache.hadoop.fs.Path -import org.apache.spark.SparkContext import org.apache.spark.sql._ import org.apache.spark.sql.execution.WholeStageCodegenExec import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext} +import org.apache.spark.util.Utils class SQLConfSuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -219,8 +219,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") } 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 2b945dbbe03dd..6fbbed1d47e04 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.execution import java.io.File +import java.net.URI import java.sql.Timestamp import java.util.{Locale, TimeZone} @@ -954,7 +955,8 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd .mkString("/") // Loads partition data to a temporary table to verify contents - val path = s"${sparkSession.getWarehousePath}/dynamic_part_table/$partFolder/part-00000" + val warehousePathFile = new URI(sparkSession.getWarehousePath()).getPath + val path = s"$warehousePathFile/dynamic_part_table/$partFolder/part-00000" sql("DROP TABLE IF EXISTS dp_verify") sql("CREATE TABLE dp_verify(intcol INT)") 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 9ed454e578d69..d9ddcbd57ca83 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.sources import java.io.File +import java.net.URI import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog.BucketSpec @@ -489,8 +490,8 @@ 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.getWarehousePath, "bucketed_table") + val warehouseFilePath = new URI(hiveContext.sparkSession.getWarehousePath).getPath + val tableDir = new File(warehouseFilePath, "bucketed_table") Utils.deleteRecursively(tableDir) df1.write.parquet(tableDir.getAbsolutePath) From 81d6933e75579343b1dd14792c18149e97e92cdd Mon Sep 17 00:00:00 2001 From: Eren Avsarogullari Date: Mon, 24 Oct 2016 15:33:02 -0700 Subject: [PATCH 44/48] [SPARK-17894][CORE] Ensure uniqueness of TaskSetManager name. `TaskSetManager` should have unique name to avoid adding duplicate ones to parent `Pool` via `SchedulableBuilder`. This problem has been surfaced with following discussion: [[PR: Avoid adding duplicate schedulables]](https://github.com/apache/spark/pull/15326) **Proposal** : There is 1x1 relationship between `stageAttemptId` and `TaskSetManager` so `taskSet.Id` covering both `stageId` and `stageAttemptId` looks to be used for uniqueness of `TaskSetManager` name instead of just `stageId`. **Current TaskSetManager Name** : `var name = "TaskSet_" + taskSet.stageId.toString` **Sample**: TaskSet_0 **Proposed TaskSetManager Name** : `val name = "TaskSet_" + taskSet.Id ` `// taskSet.Id = (stageId + "." + stageAttemptId)` **Sample** : TaskSet_0.0 Added new Unit Test. Author: erenavsarogullari Closes #15463 from erenavsarogullari/SPARK-17894. --- .../spark/scheduler/TaskSetManager.scala | 2 +- .../org/apache/spark/scheduler/FakeTask.scala | 13 ++++++++---- .../spark/scheduler/TaskSetManagerSuite.scala | 20 ++++++++++++++++++- 3 files changed, 29 insertions(+), 6 deletions(-) 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 9491bc7a0497e..b766e4148e496 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -79,7 +79,7 @@ private[spark] class TaskSetManager( var minShare = 0 var priority = taskSet.priority var stageId = taskSet.stageId - var name = "TaskSet_" + taskSet.stageId.toString + val name = "TaskSet_" + taskSet.id var parent: Pool = null var totalResultSize = 0L var calculatedTasks = 0 diff --git a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala index 87600fe504b98..f395fe9804c91 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala @@ -22,7 +22,7 @@ import org.apache.spark.TaskContext class FakeTask( stageId: Int, partitionId: Int, - prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId, 0, partitionId) { + prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId, stageAttemptId = 0, partitionId) { override def runTask(context: TaskContext): Int = 0 override def preferredLocations: Seq[TaskLocation] = prefLocs } @@ -33,16 +33,21 @@ object FakeTask { * locations for each task (given as varargs) if this sequence is not empty. */ def createTaskSet(numTasks: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { - createTaskSet(numTasks, 0, prefLocs: _*) + createTaskSet(numTasks, stageAttemptId = 0, prefLocs: _*) } def createTaskSet(numTasks: Int, stageAttemptId: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { + createTaskSet(numTasks, stageId = 0, stageAttemptId, prefLocs: _*) + } + + def createTaskSet(numTasks: Int, stageId: Int, stageAttemptId: Int, prefLocs: Seq[TaskLocation]*): + TaskSet = { if (prefLocs.size != 0 && prefLocs.size != numTasks) { throw new IllegalArgumentException("Wrong number of task locations") } val tasks = Array.tabulate[Task[_]](numTasks) { i => - new FakeTask(0, i, if (prefLocs.size != 0) prefLocs(i) else Nil) + new FakeTask(stageId, i, if (prefLocs.size != 0) prefLocs(i) else Nil) } - new TaskSet(tasks, 0, stageAttemptId, 0, null) + new TaskSet(tasks, stageId, stageAttemptId, priority = 0, null) } } 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 69edcf3347243..b49ba085ca5d2 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -904,7 +904,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg task.index == index && !sched.endedTasks.contains(task.taskId) }.getOrElse { throw new RuntimeException(s"couldn't find index $index in " + - s"tasks: ${tasks.map{t => t.index -> t.taskId}} with endedTasks:" + + s"tasks: ${tasks.map { t => t.index -> t.taskId }} with endedTasks:" + s" ${sched.endedTasks.keys}") } } @@ -974,6 +974,24 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(manager.isZombie) } + test("SPARK-17894: Verify TaskSetManagers for different stage attempts have unique names") { + sc = new SparkContext("local", "test") + sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + val taskSet = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0) + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, new ManualClock) + assert(manager.name === "TaskSet_0.0") + + // Make sure a task set with the same stage ID but different attempt ID has a unique name + val taskSet2 = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 1) + val manager2 = new TaskSetManager(sched, taskSet2, MAX_TASK_FAILURES, new ManualClock) + assert(manager2.name === "TaskSet_0.1") + + // Make sure a task set with the same attempt ID but different stage ID also has a unique name + val taskSet3 = FakeTask.createTaskSet(numTasks = 1, stageId = 1, stageAttemptId = 1) + val manager3 = new TaskSetManager(sched, taskSet3, MAX_TASK_FAILURES, new ManualClock) + assert(manager3.name === "TaskSet_1.1") + } + private def createTaskResult( id: Int, accumUpdates: Seq[AccumulatorV2[_, _]] = Seq.empty): DirectTaskResult[Int] = { From 407c3cedf29a4413339dcde758295dc3225a0054 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 24 Oct 2016 17:21:16 -0700 Subject: [PATCH 45/48] [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. --- .../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 7d71f5242c27d..f1e7f1d113ce7 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 */ @@ -493,10 +497,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 { @@ -560,7 +566,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 84a33999082af88ea6365cdb5c7232ed0933b1c6 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 25 Oct 2016 08:42:21 +0800 Subject: [PATCH 46/48] [SPARK-18028][SQL] simplify TableFileCatalog ## What changes were proposed in this pull request? Simplify/cleanup TableFileCatalog: 1. pass a `CatalogTable` instead of `databaseName` and `tableName` into `TableFileCatalog`, so that we don't need to fetch table metadata from metastore again 2. In `TableFileCatalog.filterPartitions0`, DO NOT set `PartitioningAwareFileCatalog.BASE_PATH_PARAM`. According to the [classdoc](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala#L189-L209), the default value of `basePath` already satisfies our need. What's more, if we set this parameter, we may break the case 2 which is metioned in the classdoc. 3. add `equals` and `hashCode` to `TableFileCatalog` 4. add `SessionCatalog.listPartitionsByFilter` which handles case sensitivity. ## How was this patch tested? existing tests. Author: Wenchen Fan Closes #15568 from cloud-fan/table-file-catalog. --- .../sql/catalyst/catalog/SessionCatalog.scala | 14 +++++ .../datasources/TableFileCatalog.scala | 54 ++++++++++--------- .../spark/sql/hive/HiveMetastoreCatalog.scala | 4 +- .../spark/sql/hive/CachedTableSuite.scala | 41 +++++++++++++- .../PruneFileSourcePartitionsSuite.scala | 7 +-- 5 files changed, 84 insertions(+), 36 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 9711131d88a05..3d6eec81c03c8 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 @@ -755,6 +755,20 @@ class SessionCatalog( externalCatalog.listPartitions(db, table, partialSpec) } + /** + * List the metadata of partitions that belong to the specified table, assuming it exists, that + * satisfy the given partition-pruning predicate expressions. + */ + def listPartitionsByFilter( + tableName: TableIdentifier, + predicates: Seq[Expression]): Seq[CatalogTablePartition] = { + val db = formatDatabaseName(tableName.database.getOrElse(getCurrentDatabase)) + val table = formatTableName(tableName.table) + requireDbExists(db) + requireTableExists(TableIdentifier(table, Option(db))) + externalCatalog.listPartitionsByFilter(db, table, predicates) + } + /** * Verify if the input partition spec exactly matches the existing defined partition spec * The columns must be the same but the orders could be different. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala index 31a01bc6db082..667379b222c48 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala @@ -20,36 +20,30 @@ package org.apache.spark.sql.execution.datasources import org.apache.hadoop.fs.Path import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types.StructType /** * A [[FileCatalog]] for a metastore catalog table. * * @param sparkSession a [[SparkSession]] - * @param db the table's database name - * @param table the table's (unqualified) name - * @param partitionSchema the schema of a partitioned table's partition columns + * @param table the metadata of the table * @param sizeInBytes the table's data size in bytes - * @param fileStatusCache optional cache implementation to use for file listing */ class TableFileCatalog( sparkSession: SparkSession, - db: String, - table: String, - partitionSchema: Option[StructType], + val table: CatalogTable, override val sizeInBytes: Long) extends FileCatalog { protected val hadoopConf = sparkSession.sessionState.newHadoopConf private val fileStatusCache = FileStatusCache.newCache(sparkSession) - private val externalCatalog = sparkSession.sharedState.externalCatalog + assert(table.identifier.database.isDefined, + "The table identifier must be qualified in TableFileCatalog") - private val catalogTable = externalCatalog.getTable(db, table) - - private val baseLocation = catalogTable.storage.locationUri + private val baseLocation = table.storage.locationUri override def rootPaths: Seq[Path] = baseLocation.map(new Path(_)).toSeq @@ -66,24 +60,32 @@ class TableFileCatalog( * @param filters partition-pruning filters */ def filterPartitions(filters: Seq[Expression]): ListingFileCatalog = { - val parameters = baseLocation - .map(loc => Map(PartitioningAwareFileCatalog.BASE_PATH_PARAM -> loc)) - .getOrElse(Map.empty) - partitionSchema match { - case Some(schema) => - val selectedPartitions = externalCatalog.listPartitionsByFilter(db, table, filters) - val partitions = selectedPartitions.map { p => - PartitionPath(p.toRow(schema), p.storage.locationUri.get) - } - val partitionSpec = PartitionSpec(schema, partitions) - new PrunedTableFileCatalog( - sparkSession, new Path(baseLocation.get), fileStatusCache, partitionSpec) - case None => - new ListingFileCatalog(sparkSession, rootPaths, parameters, None, fileStatusCache) + if (table.partitionColumnNames.nonEmpty) { + val selectedPartitions = sparkSession.sessionState.catalog.listPartitionsByFilter( + table.identifier, filters) + val partitionSchema = table.partitionSchema + val partitions = selectedPartitions.map { p => + PartitionPath(p.toRow(partitionSchema), p.storage.locationUri.get) + } + val partitionSpec = PartitionSpec(partitionSchema, partitions) + new PrunedTableFileCatalog( + sparkSession, new Path(baseLocation.get), fileStatusCache, partitionSpec) + } else { + new ListingFileCatalog(sparkSession, rootPaths, table.storage.properties, None) } } override def inputFiles: Array[String] = filterPartitions(Nil).inputFiles + + // `TableFileCatalog` may be a member of `HadoopFsRelation`, `HadoopFsRelation` may be a member + // of `LogicalRelation`, and `LogicalRelation` may be used as the cache key. So we need to + // implement `equals` and `hashCode` here, to make it work with cache lookup. + override def equals(o: Any): Boolean = o match { + case other: TableFileCatalog => this.table.identifier == other.table.identifier + case _ => false + } + + override def hashCode(): Int = table.identifier.hashCode() } /** 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 44089335e1a1d..6c1585d5f5617 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 @@ -226,12 +226,10 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log Some(partitionSchema)) val logicalRelation = cached.getOrElse { - val db = metastoreRelation.databaseName - val table = metastoreRelation.tableName val sizeInBytes = metastoreRelation.statistics.sizeInBytes.toLong val fileCatalog = { val catalog = new TableFileCatalog( - sparkSession, db, table, Some(partitionSchema), sizeInBytes) + sparkSession, metastoreRelation.catalogTable, sizeInBytes) if (lazyPruningEnabled) { catalog } else { 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 7d4ef6f26a600..ecdf4f14b3985 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 @@ -19,12 +19,15 @@ package org.apache.spark.sql.hive import java.io.File -import org.apache.spark.sql.{AnalysisException, QueryTest, SaveMode} +import org.apache.spark.sql.{AnalysisException, Dataset, QueryTest, SaveMode} import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, TableFileCatalog} +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types.StructType import org.apache.spark.storage.RDDBlockId import org.apache.spark.util.Utils @@ -317,4 +320,40 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto sql("DROP TABLE cachedTable") } + + test("cache a table using TableFileCatalog") { + withTable("test") { + sql("CREATE TABLE test(i int) PARTITIONED BY (p int) STORED AS parquet") + val tableMeta = spark.sharedState.externalCatalog.getTable("default", "test") + val tableFileCatalog = new TableFileCatalog(spark, tableMeta, 0) + + val dataSchema = StructType(tableMeta.schema.filterNot { f => + tableMeta.partitionColumnNames.contains(f.name) + }) + val relation = HadoopFsRelation( + location = tableFileCatalog, + partitionSchema = tableMeta.partitionSchema, + dataSchema = dataSchema, + bucketSpec = None, + fileFormat = new ParquetFileFormat(), + options = Map.empty)(sparkSession = spark) + + val plan = LogicalRelation(relation, catalogTable = Some(tableMeta)) + spark.sharedState.cacheManager.cacheQuery(Dataset.ofRows(spark, plan)) + + assert(spark.sharedState.cacheManager.lookupCachedData(plan).isDefined) + + val sameCatalog = new TableFileCatalog(spark, tableMeta, 0) + val sameRelation = HadoopFsRelation( + location = sameCatalog, + partitionSchema = tableMeta.partitionSchema, + dataSchema = dataSchema, + bucketSpec = None, + fileFormat = new ParquetFileFormat(), + options = Map.empty)(sparkSession = spark) + val samePlan = LogicalRelation(sameRelation, catalogTable = Some(tableMeta)) + + assert(spark.sharedState.cacheManager.lookupCachedData(samePlan).isDefined) + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala index 346ea0ca4367e..59639aacf3a3f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala @@ -45,12 +45,7 @@ class PruneFileSourcePartitionsSuite extends QueryTest with SQLTestUtils with Te |LOCATION '${dir.getAbsolutePath}'""".stripMargin) val tableMeta = spark.sharedState.externalCatalog.getTable("default", "test") - val tableFileCatalog = new TableFileCatalog( - spark, - tableMeta.database, - tableMeta.identifier.table, - Some(tableMeta.partitionSchema), - 0) + val tableFileCatalog = new TableFileCatalog(spark, tableMeta, 0) val dataSchema = StructType(tableMeta.schema.filterNot { f => tableMeta.partitionColumnNames.contains(f.name) From d479c5262276b47302659bd877a9e3467400bdb6 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Tue, 25 Oct 2016 10:47:11 +0800 Subject: [PATCH 47/48] [SPARK-17409][SQL][FOLLOW-UP] Do Not Optimize Query in CTAS More Than Once ### What changes were proposed in this pull request? This follow-up PR is for addressing the [comment](https://github.com/apache/spark/pull/15048). We added two test cases based on the suggestion from yhuai . One is a new test case using the `saveAsTable` API to create a data source table. Another is for CTAS on Hive serde table. Note: No need to backport this PR to 2.0. Will submit a new PR to backport the whole fix with new test cases to Spark 2.0 ### How was this patch tested? N/A Author: gatorsmile Closes #15459 from gatorsmile/ctasOptimizedTestCases. --- .../org/apache/spark/sql/DataFrameSuite.scala | 18 +++++++++++++++++ .../sources/CreateTableAsSelectSuite.scala | 2 +- .../sql/hive/MetastoreRelationSuite.scala | 20 +++++++++++++++++-- 3 files changed, 37 insertions(+), 3 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 e87baa454c8b3..3fb7eeefba67f 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 @@ -1599,6 +1599,24 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { assert(df.persist.take(1).apply(0).toSeq(100).asInstanceOf[Long] == 100) } + 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.provider == Some("json"), + "the expected table is a data source table using json") + } + } + } + } + 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) 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 c39005f6a1063..5cc9467395adc 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 @@ -238,7 +238,7 @@ class CreateTableAsSelectSuite } } - test("CTAS of decimal calculation") { + test("SPARK-17409: CTAS of decimal calculation") { withTable("tab2") { withTempView("tab1") { spark.range(99, 101).createOrReplaceTempView("tab1") 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 c28e41a85c39d..91ff711445e82 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,12 +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.{CatalogStorageFormat, CatalogTable, CatalogTableType} +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types.{IntegerType, StructField, StructType} -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")), @@ -36,4 +38,18 @@ 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(tableMetadata.provider == Some("hive"), "the expected table is a Hive serde table") + } + } + } } From 483c37c581fedc64b218e294ecde1a7bb4b2af9c Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Mon, 24 Oct 2016 20:16:00 -0700 Subject: [PATCH 48/48] [SPARK-17894][HOTFIX] Fix broken build from The named parameter in an overridden class isn't supported in Scala 2.10 so was breaking the build. cc zsxwing Author: Kay Ousterhout Closes #15617 from kayousterhout/hotfix. --- core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala index f395fe9804c91..a757041299411 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala @@ -22,7 +22,7 @@ import org.apache.spark.TaskContext class FakeTask( stageId: Int, partitionId: Int, - prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId, stageAttemptId = 0, partitionId) { + prefLocs: Seq[TaskLocation] = Nil) extends Task[Int](stageId, 0, partitionId) { override def runTask(context: TaskContext): Int = 0 override def preferredLocations: Seq[TaskLocation] = prefLocs }
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 c9720b2195a465653690b3e221ce789142217b0d Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Fri, 21 Oct 2016 16:27:19 -0700 Subject: [PATCH 27/48] [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. --- 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 3fbf5a58c236fc5d5fee39cb29e7f5c7e01c0ee7 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 21 Oct 2016 17:27:18 -0700 Subject: [PATCH 28/48] [SPARK-18042][SQL] OutputWriter should expose file path written ## What changes were proposed in this pull request? This patch adds a new "path" method on OutputWriter that returns the path of the file written by the OutputWriter. This is part of the necessary work to consolidate structured streaming and batch write paths. The batch write path has a nice feature that each data source can define the extension of the files, and allow Spark to specify the staging directory and the prefix for the files. However, in the streaming path we need to collect the list of files written, and there is no interface right now to do that. ## How was this patch tested? N/A - there is no behavior change and this should be covered by existing tests. Author: Reynold Xin Closes #15580 from rxin/SPARK-18042. --- .../ml/source/libsvm/LibSVMRelation.scala | 8 ++++- .../execution/datasources/OutputWriter.scala | 17 ++++++----- .../datasources/csv/CSVRelation.scala | 8 ++++- .../datasources/json/JsonFileFormat.scala | 8 ++++- .../parquet/ParquetFileFormat.scala | 2 +- .../datasources/parquet/ParquetOptions.scala | 2 +- .../parquet/ParquetOutputWriter.scala | 24 ++++++++------- .../datasources/text/TextFileFormat.scala | 25 ++++++++++++++-- .../spark/sql/hive/orc/OrcFileFormat.scala | 29 +++++++++---------- .../sql/sources/CommitFailureTestSource.scala | 3 ++ .../sql/sources/SimpleTextRelation.scala | 3 ++ 11 files changed, 90 insertions(+), 39 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 fff86686b550c..5e9e6ff1a5690 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 @@ -35,6 +35,7 @@ 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.datasources._ +import org.apache.spark.sql.execution.datasources.text.TextOutputWriter import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ import org.apache.spark.util.SerializableConfiguration @@ -46,12 +47,17 @@ private[libsvm] class LibSVMOutputWriter( context: TaskAttemptContext) extends OutputWriter { + override val path: String = { + val compressionExtension = TextOutputWriter.getCompressionExtension(context) + new Path(stagingDir, fileNamePrefix + ".libsvm" + compressionExtension).toString + } + private[this] val buffer = new Text() private val recordWriter: RecordWriter[NullWritable, Text] = { new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - new Path(stagingDir, fileNamePrefix + extension) + new Path(path) } }.getRecordWriter(context) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala index f4cefdab077e9..fbf6e96d3f850 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/OutputWriter.scala @@ -42,11 +42,12 @@ abstract class OutputWriterFactory extends Serializable { * @param fileNamePrefix Prefix of the file name. The returned OutputWriter must make sure this * prefix is used in the actual file name. For example, if the prefix is * "part-1-2-3", then the file name must start with "part_1_2_3" but can - * end in arbitrary extension. + * end in arbitrary extension that is deterministic given the configuration + * (i.e. the suffix extension should not depend on any task id, attempt id, + * or partition id). * @param dataSchema Schema of the rows to be written. Partition columns are not included in the * schema if the relation being written is partitioned. * @param context The Hadoop MapReduce task context. - * @since 1.4.0 */ def newInstance( stagingDir: String, @@ -62,7 +63,6 @@ abstract class OutputWriterFactory extends Serializable { * 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 */ def newWriter(path: String): OutputWriter = { throw new UnsupportedOperationException("newInstance with just path not supported") @@ -77,19 +77,22 @@ abstract class OutputWriterFactory extends Serializable { * executor side. This instance is used to persist rows to this single output file. */ abstract class OutputWriter { + + /** + * The path of the file to be written out. This path should include the staging directory and + * the file name prefix passed into the associated createOutputWriter function. + */ + def path: String + /** * Persists a single row. Invoked on the executor side. When writing to dynamically partitioned * tables, dynamic partition columns are not included in rows to be written. - * - * @since 1.4.0 */ def write(row: Row): Unit /** * Closes the [[OutputWriter]]. Invoked on the executor side after all rows are persisted, before * the task output is committed. - * - * @since 1.4.0 */ def close(): Unit 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 eefacbf05ba0d..a35cfdb2c234f 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 @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory, PartitionedFile} +import org.apache.spark.sql.execution.datasources.text.TextOutputWriter import org.apache.spark.sql.types._ object CSVRelation extends Logging { @@ -185,6 +186,11 @@ private[csv] class CsvOutputWriter( context: TaskAttemptContext, params: CSVOptions) extends OutputWriter with Logging { + override val path: String = { + val compressionExtension = TextOutputWriter.getCompressionExtension(context) + new Path(stagingDir, fileNamePrefix + ".csv" + compressionExtension).toString + } + // create the Generator without separator inserted between 2 records private[this] val text = new Text() @@ -199,7 +205,7 @@ private[csv] class CsvOutputWriter( private val recordWriter: RecordWriter[NullWritable, Text] = { new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - new Path(stagingDir, s"$fileNamePrefix.csv$extension") + new Path(path) } }.getRecordWriter(context) } 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 cdbb2f7292613..651fa78a4e924 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 @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.json.{JacksonParser, JSONOptions} import org.apache.spark.sql.catalyst.util.CompressionCodecs import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.text.TextOutputWriter import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -160,6 +161,11 @@ private[json] class JsonOutputWriter( context: TaskAttemptContext) extends OutputWriter with Logging { + override val path: String = { + val compressionExtension = TextOutputWriter.getCompressionExtension(context) + new Path(stagingDir, fileNamePrefix + ".json" + compressionExtension).toString + } + private[this] val writer = new CharArrayWriter() // create the Generator without separator inserted between 2 records private[this] val gen = new JacksonGenerator(dataSchema, writer, options) @@ -168,7 +174,7 @@ private[json] class JsonOutputWriter( private val recordWriter: RecordWriter[NullWritable, Text] = { new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - new Path(stagingDir, s"$fileNamePrefix.json$extension") + new Path(path) } }.getRecordWriter(context) } 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 87b944ba523ca..502dd0e8d4cf9 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 @@ -121,7 +121,7 @@ class ParquetFileFormat sparkSession.sessionState.conf.writeLegacyParquetFormat.toString) // Sets compression scheme - conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodec) + conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodecClassName) // SPARK-15719: Disables writing Parquet summary files by default. if (conf.get(ParquetOutputFormat.ENABLE_JOB_SUMMARY) == 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 615731889dfad..d0fd23605bea8 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 @@ -35,7 +35,7 @@ private[parquet] class ParquetOptions( * Compression codec to use. By default use the value specified in SQLConf. * Acceptable values are defined in [[shortParquetCompressionCodecNames]]. */ - val compressionCodec: String = { + val compressionCodecClassName: String = { val codecName = parameters.getOrElse("compression", sqlConf.parquetCompressionCodec).toLowerCase if (!shortParquetCompressionCodecNames.contains(codecName)) { val availableCodecs = shortParquetCompressionCodecNames.keys.map(_.toLowerCase) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala index 39c199784cd6d..1300069c42b05 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.parquet.hadoop.{ParquetOutputFormat, ParquetRecordWriter} +import org.apache.parquet.hadoop.codec.CodecConfig import org.apache.parquet.hadoop.util.ContextUtil import org.apache.spark.sql.Row @@ -80,7 +81,7 @@ private[parquet] class ParquetOutputWriterFactory( sqlConf.writeLegacyParquetFormat.toString) // Sets compression scheme - conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodec) + conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodecClassName) new SerializableConfiguration(conf) } @@ -88,7 +89,7 @@ private[parquet] class ParquetOutputWriterFactory( * Returns a [[OutputWriter]] that writes data to the give path without using * [[OutputCommitter]]. */ - override def newWriter(path: String): OutputWriter = new OutputWriter { + override def newWriter(path1: String): OutputWriter = new OutputWriter { // Create TaskAttemptContext that is used to pass on Configuration to the ParquetRecordWriter private val hadoopTaskAttemptId = new TaskAttemptID(new TaskID(new JobID, TaskType.MAP, 0), 0) @@ -98,6 +99,8 @@ private[parquet] class ParquetOutputWriterFactory( // Instance of ParquetRecordWriter that does not use OutputCommitter private val recordWriter = createNoCommitterRecordWriter(path, hadoopAttemptContext) + override def path: String = path1 + override def write(row: Row): Unit = { throw new UnsupportedOperationException("call writeInternal") } @@ -140,16 +143,17 @@ private[parquet] class ParquetOutputWriter( context: TaskAttemptContext) extends OutputWriter { + override val path: String = { + val filename = fileNamePrefix + CodecConfig.from(context).getCodec.getExtension + ".parquet" + new Path(stagingDir, filename).toString + } + private val recordWriter: RecordWriter[Void, InternalRow] = { - val outputFormat = { - new ParquetOutputFormat[InternalRow]() { - override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - new Path(stagingDir, fileNamePrefix + extension) - } + new ParquetOutputFormat[InternalRow]() { + override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { + new Path(path) } - } - - outputFormat.getRecordWriter(context) + }.getRecordWriter(context) } override def write(row: Row): Unit = throw new UnsupportedOperationException("call writeInternal") 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 6cd2351c5749a..d40b5725199a8 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 @@ -20,8 +20,10 @@ package org.apache.spark.sql.execution.datasources.text import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.io.{NullWritable, Text} +import org.apache.hadoop.io.compress.GzipCodec import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext} -import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat +import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat, TextOutputFormat} +import org.apache.hadoop.util.ReflectionUtils import org.apache.spark.TaskContext import org.apache.spark.sql.{AnalysisException, Row, SparkSession} @@ -128,12 +130,17 @@ class TextOutputWriter( context: TaskAttemptContext) extends OutputWriter { + override val path: String = { + val compressionExtension = TextOutputWriter.getCompressionExtension(context) + new Path(stagingDir, fileNamePrefix + ".txt" + compressionExtension).toString + } + private[this] val buffer = new Text() private val recordWriter: RecordWriter[NullWritable, Text] = { new TextOutputFormat[NullWritable, Text]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - new Path(stagingDir, s"$fileNamePrefix.txt$extension") + new Path(path) } }.getRecordWriter(context) } @@ -150,3 +157,17 @@ class TextOutputWriter( recordWriter.close(context) } } + + +object TextOutputWriter { + /** Returns the compression codec extension to be used in a file name, e.g. ".gzip"). */ + def getCompressionExtension(context: TaskAttemptContext): String = { + // Set the compression extension, similar to code in TextOutputFormat.getDefaultWorkFile + if (FileOutputFormat.getCompressOutput(context)) { + val codecClass = FileOutputFormat.getOutputCompressorClass(context, classOf[GzipCodec]) + ReflectionUtils.newInstance(codecClass, context.getConfiguration).getDefaultExtension + } else { + "" + } + } +} 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 1ceacb458ae6e..eba7aa386ade2 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 @@ -216,9 +216,18 @@ private[orc] class OrcOutputWriter( context: TaskAttemptContext) extends OutputWriter { - private[this] val conf = context.getConfiguration + override val path: String = { + val compressionExtension: String = { + val name = context.getConfiguration.get(OrcRelation.ORC_COMPRESSION) + OrcRelation.extensionsForCompressionCodecNames.getOrElse(name, "") + } + // It has the `.orc` extension at the end because (de)compression tools + // such as gunzip would not be able to decompress this as the compression + // is not applied on this whole file but on each "stream" in ORC format. + new Path(stagingDir, fileNamePrefix + compressionExtension + ".orc").toString + } - private[this] val serializer = new OrcSerializer(dataSchema, conf) + private[this] val serializer = new OrcSerializer(dataSchema, context.getConfiguration) // `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. @@ -226,20 +235,10 @@ private[orc] class OrcOutputWriter( private lazy val recordWriter: RecordWriter[NullWritable, Writable] = { recordWriterInstantiated = true - - val compressionExtension = { - val name = conf.get(OrcRelation.ORC_COMPRESSION) - OrcRelation.extensionsForCompressionCodecNames.getOrElse(name, "") - } - // It has the `.orc` extension at the end because (de)compression tools - // such as gunzip would not be able to decompress this as the compression - // is not applied on this whole file but on each "stream" in ORC format. - val filename = s"$fileNamePrefix$compressionExtension.orc" - new OrcOutputFormat().getRecordWriter( - new Path(stagingDir, filename).getFileSystem(conf), - conf.asInstanceOf[JobConf], - new Path(stagingDir, filename).toString, + new Path(path).getFileSystem(context.getConfiguration), + context.getConfiguration.asInstanceOf[JobConf], + path, Reporter.NULL ).asInstanceOf[RecordWriter[NullWritable, Writable]] } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala index d5044684020e2..731540db17eeb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/CommitFailureTestSource.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.sources +import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import org.apache.spark.TaskContext @@ -50,6 +51,8 @@ class CommitFailureTestSource extends SimpleTextSource { SimpleTextRelation.callbackCalled = true } + override val path: String = new Path(stagingDir, fileNamePrefix).toString + override def write(row: Row): Unit = { if (SimpleTextRelation.failWriter) { sys.error("Intentional task writer failure for testing purpose.") 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 9e13b217ec305..9896b9bde99c8 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 @@ -123,6 +123,9 @@ class SimpleTextSource extends TextBasedFileFormat with DataSourceRegister { class SimpleTextOutputWriter( stagingDir: String, fileNamePrefix: String, context: TaskAttemptContext) extends OutputWriter { + + override val path: String = new Path(stagingDir, fileNamePrefix).toString + private val recordWriter: RecordWriter[NullWritable, Text] = new AppendingTextOutputFormat(new Path(stagingDir), fileNamePrefix).getRecordWriter(context) From 7178c56433cd138dae53db9194c55e3f4fa0fa69 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 21 Oct 2016 22:20:52 -0700 Subject: [PATCH 29/48] [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. --- .../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 baae55013787d..3045eb69f427f 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 @@ -814,7 +814,7 @@ object SparkSession { if ((session ne null) && !session.sparkContext.isStopped) { options.foreach { case (k, v) => session.sessionState.conf.setConfString(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 } @@ -826,7 +826,7 @@ object SparkSession { if ((session ne null) && !session.sparkContext.isStopped) { options.foreach { case (k, v) => session.sessionState.conf.setConfString(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 625fdddacd58ad54fdbb17409987812176abc812 Mon Sep 17 00:00:00 2001 From: Erik O'Shaughnessy Date: Sat, 22 Oct 2016 09:37:53 +0100 Subject: [PATCH 30/48] [SPARK-17944][DEPLOY] sbin/start-* scripts use of `hostname -f` fail with Solaris ## What changes were proposed in this pull request? Modify sbin/start-master.sh, sbin/start-mesos-dispatcher.sh and sbin/start-slaves.sh to use the output of 'uname' to select which OS-specific command-line is used to determine the host's fully qualified host name. ## How was this patch tested? Tested by hand; starting on Solaris, Linux and macOS. Author: Erik O'Shaughnessy Closes #15557 from JnyJny/SPARK-17944. --- sbin/start-master.sh | 9 ++++++++- sbin/start-mesos-dispatcher.sh | 9 ++++++++- sbin/start-slaves.sh | 9 ++++++++- 3 files changed, 24 insertions(+), 3 deletions(-) diff --git a/sbin/start-master.sh b/sbin/start-master.sh index d970fcc45e2c1..97ee32159b6de 100755 --- a/sbin/start-master.sh +++ b/sbin/start-master.sh @@ -48,7 +48,14 @@ if [ "$SPARK_MASTER_PORT" = "" ]; then fi if [ "$SPARK_MASTER_HOST" = "" ]; then - SPARK_MASTER_HOST=`hostname -f` + case `uname` in + (SunOS) + SPARK_MASTER_HOST="`/usr/sbin/check-hostname | awk '{print $NF}'`" + ;; + (*) + SPARK_MASTER_HOST="`hostname -f`" + ;; + esac fi if [ "$SPARK_MASTER_WEBUI_PORT" = "" ]; then diff --git a/sbin/start-mesos-dispatcher.sh b/sbin/start-mesos-dispatcher.sh index ef65fb9539146..ecaad7ad09634 100755 --- a/sbin/start-mesos-dispatcher.sh +++ b/sbin/start-mesos-dispatcher.sh @@ -34,7 +34,14 @@ if [ "$SPARK_MESOS_DISPATCHER_PORT" = "" ]; then fi if [ "$SPARK_MESOS_DISPATCHER_HOST" = "" ]; then - SPARK_MESOS_DISPATCHER_HOST=`hostname -f` + case `uname` in + (SunOS) + SPARK_MESOS_DISPATCHER_HOST="`/usr/sbin/check-hostname | awk '{print $NF}'`" + ;; + (*) + SPARK_MESOS_DISPATCHER_HOST="`hostname -f`" + ;; + esac fi if [ "$SPARK_MESOS_DISPATCHER_NUM" = "" ]; then diff --git a/sbin/start-slaves.sh b/sbin/start-slaves.sh index 7d8871251f81b..f5269df523dac 100755 --- a/sbin/start-slaves.sh +++ b/sbin/start-slaves.sh @@ -32,7 +32,14 @@ if [ "$SPARK_MASTER_PORT" = "" ]; then fi if [ "$SPARK_MASTER_HOST" = "" ]; then - SPARK_MASTER_HOST="`hostname -f`" + case `uname` in + (SunOS) + SPARK_MASTER_HOST="`/usr/sbin/check-hostname | awk '{print $NF}'`" + ;; + (*) + SPARK_MASTER_HOST="`hostname -f`" + ;; + esac fi # Launch the slaves From 01b26a06436b4c8020f22be3e1da4995b44c9b03 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 22 Oct 2016 09:39:07 +0100 Subject: [PATCH 31/48] [SPARK-17898][DOCS] repositories needs username and password ## What changes were proposed in this pull request? Document `user:password` syntax as possible means of specifying credentials for password-protected `--repositories` ## How was this patch tested? Doc build Author: Sean Owen Closes #15584 from srowen/SPARK-17898. --- docs/programming-guide.md | 8 ++++---- docs/submitting-applications.md | 2 ++ 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 20b4bee0f58e1..7516579ec6dbf 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -182,7 +182,7 @@ variable called `sc`. Making your own SparkContext will not work. You can set wh context connects to using the `--master` argument, and you can add JARs to the classpath by passing a comma-separated list to the `--jars` argument. You can also add dependencies (e.g. Spark Packages) to your shell session by supplying a comma-separated list of maven coordinates -to the `--packages` argument. Any additional repositories where dependencies might exist (e.g. SonaType) +to the `--packages` argument. Any additional repositories where dependencies might exist (e.g. Sonatype) can be passed to the `--repositories` argument. For example, to run `bin/spark-shell` on exactly four cores, use: @@ -214,9 +214,9 @@ variable called `sc`. Making your own SparkContext will not work. You can set wh context connects to using the `--master` argument, and you can add Python .zip, .egg or .py files to the runtime path by passing a comma-separated list to `--py-files`. You can also add dependencies (e.g. Spark Packages) to your shell session by supplying a comma-separated list of maven coordinates -to the `--packages` argument. Any additional repositories where dependencies might exist (e.g. SonaType) -can be passed to the `--repositories` argument. Any python dependencies a Spark Package has (listed in -the requirements.txt of that package) must be manually installed using pip when necessary. +to the `--packages` argument. Any additional repositories where dependencies might exist (e.g. Sonatype) +can be passed to the `--repositories` argument. Any Python dependencies a Spark package has (listed in +the requirements.txt of that package) must be manually installed using `pip` when necessary. For example, to run `bin/pyspark` on exactly four cores, use: {% highlight bash %} diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index 6fe3049995876..b738194eac9aa 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -190,6 +190,8 @@ is handled automatically, and with Spark standalone, automatic cleanup can be co Users may also include any other dependencies by supplying a comma-delimited list of maven coordinates with `--packages`. All transitive dependencies will be handled when using this command. Additional repositories (or resolvers in SBT) can be added in a comma-delimited fashion with the flag `--repositories`. +(Note that credentials for password-protected repositories can be supplied in some cases in the repository URI, +such as in `https://user:password@host/...`. Be careful when supplying credentials this way.) These commands can be used with `pyspark`, `spark-shell`, and `spark-submit` to include Spark Packages. For Python, the equivalent `--py-files` option can be used to distribute `.egg`, `.zip` and `.py` libraries From ab3363e9f6b1f7fc26682509fe7382c570f91778 Mon Sep 17 00:00:00 2001 From: Drew Robb Date: Sat, 22 Oct 2016 01:59:36 -0700 Subject: [PATCH 32/48] [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. --- .../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 23464073e6edb..753f890c48301 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 3eca283aca68ac81c127d60ad5699f854d5f14b7 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Sat, 22 Oct 2016 22:08:28 +0800 Subject: [PATCH 33/48] [SPARK-17994][SQL] Add back a file status cache for catalog tables ## What changes were proposed in this pull request? In SPARK-16980, we removed the full in-memory cache of table partitions in favor of loading only needed partitions from the metastore. This greatly improves the initial latency of queries that only read a small fraction of table partitions. However, since the metastore does not store file statistics, we need to discover those from remote storage. With the loss of the in-memory file status cache this has to happen on each query, increasing the latency of repeated queries over the same partitions. The proposal is to add back a per-table cache of partition contents, i.e. Map[Path, Array[FileStatus]]. This cache would be retained per-table, and can be invalidated through refreshTable() and refreshByPath(). Unlike the prior cache, it can be incrementally updated as new partitions are read. ## How was this patch tested? Existing tests and new tests in `HiveTablePerfStatsSuite`. cc mallman Author: Eric Liang Author: Michael Allman Author: Eric Liang Closes #15539 from ericl/meta-cache. --- .../spark/metrics/source/StaticSources.scala | 7 + .../datasources/FileStatusCache.scala | 149 ++++++++++++++++++ .../datasources/ListingFileCatalog.scala | 13 +- .../PartitioningAwareFileCatalog.scala | 115 ++++++++------ .../datasources/TableFileCatalog.scala | 36 ++--- .../apache/spark/sql/internal/SQLConf.scala | 16 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../spark/sql/hive/HiveDDLCommandSuite.scala | 16 +- ...te.scala => HiveTablePerfStatsSuite.scala} | 127 +++++++++++++-- 9 files changed, 385 insertions(+), 96 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala rename sql/hive/src/test/scala/org/apache/spark/sql/hive/{HiveDataFrameSuite.scala => HiveTablePerfStatsSuite.scala} (50%) 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 cf92a10deabd5..b54885b7ff8b0 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 @@ -80,15 +80,22 @@ object HiveCatalogMetrics extends Source { */ val METRIC_FILES_DISCOVERED = metricRegistry.counter(MetricRegistry.name("filesDiscovered")) + /** + * Tracks the total number of files served from the file status cache instead of discovered. + */ + val METRIC_FILE_CACHE_HITS = metricRegistry.counter(MetricRegistry.name("fileCacheHits")) + /** * Resets the values of all metrics to zero. This is useful in tests. */ def reset(): Unit = { METRIC_PARTITIONS_FETCHED.dec(METRIC_PARTITIONS_FETCHED.getCount()) METRIC_FILES_DISCOVERED.dec(METRIC_FILES_DISCOVERED.getCount()) + METRIC_FILE_CACHE_HITS.dec(METRIC_FILE_CACHE_HITS.getCount()) } // clients can use these to avoid classloader issues with the codahale classes def incrementFetchedPartitions(n: Int): Unit = METRIC_PARTITIONS_FETCHED.inc(n) def incrementFilesDiscovered(n: Int): Unit = METRIC_FILES_DISCOVERED.inc(n) + def incrementFileCacheHits(n: Int): Unit = METRIC_FILE_CACHE_HITS.inc(n) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala new file mode 100644 index 0000000000000..e0ec748a0b34d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileStatusCache.scala @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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 java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.atomic.AtomicBoolean + +import scala.collection.JavaConverters._ + +import com.google.common.cache._ +import org.apache.hadoop.fs.{FileStatus, Path} + +import org.apache.spark.internal.Logging +import org.apache.spark.metrics.source.HiveCatalogMetrics +import org.apache.spark.sql.SparkSession +import org.apache.spark.util.{SerializableConfiguration, SizeEstimator} + +/** + * A cache of the leaf files of partition directories. We cache these files in order to speed + * up iterated queries over the same set of partitions. Otherwise, each query would have to + * hit remote storage in order to gather file statistics for physical planning. + * + * Each resolved catalog table has its own FileStatusCache. When the backing relation for the + * table is refreshed via refreshTable() or refreshByPath(), this cache will be invalidated. + */ +abstract class FileStatusCache { + /** + * @return the leaf files for the specified path from this cache, or None if not cached. + */ + def getLeafFiles(path: Path): Option[Array[FileStatus]] = None + + /** + * Saves the given set of leaf files for a path in this cache. + */ + def putLeafFiles(path: Path, leafFiles: Array[FileStatus]): Unit + + /** + * Invalidates all data held by this cache. + */ + def invalidateAll(): Unit +} + +object FileStatusCache { + private var sharedCache: SharedInMemoryCache = null + + /** + * @return a new FileStatusCache based on session configuration. Cache memory quota is + * shared across all clients. + */ + def newCache(session: SparkSession): FileStatusCache = { + synchronized { + if (session.sqlContext.conf.filesourcePartitionPruning && + session.sqlContext.conf.filesourcePartitionFileCacheSize > 0) { + if (sharedCache == null) { + sharedCache = new SharedInMemoryCache( + session.sqlContext.conf.filesourcePartitionFileCacheSize) + } + sharedCache.getForNewClient() + } else { + NoopCache + } + } + } + + def resetForTesting(): Unit = synchronized { + sharedCache = null + } +} + +/** + * An implementation that caches partition file statuses in memory. + * + * @param maxSizeInBytes max allowable cache size before entries start getting evicted + */ +private class SharedInMemoryCache(maxSizeInBytes: Long) extends Logging { + import FileStatusCache._ + + // Opaque object that uniquely identifies a shared cache user + private type ClientId = Object + + private val warnedAboutEviction = new AtomicBoolean(false) + + // we use a composite cache key in order to distinguish entries inserted by different clients + private val cache: Cache[(ClientId, Path), Array[FileStatus]] = CacheBuilder.newBuilder() + .weigher(new Weigher[(ClientId, Path), Array[FileStatus]] { + override def weigh(key: (ClientId, Path), value: Array[FileStatus]): Int = { + (SizeEstimator.estimate(key) + SizeEstimator.estimate(value)).toInt + }}) + .removalListener(new RemovalListener[(ClientId, Path), Array[FileStatus]]() { + override def onRemoval(removed: RemovalNotification[(ClientId, Path), Array[FileStatus]]) = { + if (removed.getCause() == RemovalCause.SIZE && + warnedAboutEviction.compareAndSet(false, true)) { + logWarning( + "Evicting cached table partition metadata from memory due to size constraints " + + "(spark.sql.hive.filesourcePartitionFileCacheSize = " + maxSizeInBytes + " bytes). " + + "This may impact query planning performance.") + } + }}) + .maximumWeight(maxSizeInBytes) + .build() + + /** + * @return a FileStatusCache that does not share any entries with any other client, but does + * share memory resources for the purpose of cache eviction. + */ + def getForNewClient(): FileStatusCache = new FileStatusCache { + val clientId = new Object() + + override def getLeafFiles(path: Path): Option[Array[FileStatus]] = { + Option(cache.getIfPresent((clientId, path))) + } + + override def putLeafFiles(path: Path, leafFiles: Array[FileStatus]): Unit = { + cache.put((clientId, path), leafFiles.toArray) + } + + override def invalidateAll(): Unit = { + cache.asMap.asScala.foreach { case (key, value) => + if (key._1 == clientId) { + cache.invalidate(key) + } + } + } + } +} + +/** + * A non-caching implementation used when partition file status caching is disabled. + */ +object NoopCache extends FileStatusCache { + override def getLeafFiles(path: Path): Option[Array[FileStatus]] = None + override def putLeafFiles(path: Path, leafFiles: Array[FileStatus]): Unit = {} + override def invalidateAll(): Unit = {} +} 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 6d10501b7265d..d9d588388aaf1 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 @@ -38,14 +38,16 @@ class ListingFileCatalog( sparkSession: SparkSession, override val rootPaths: Seq[Path], parameters: Map[String, String], - partitionSchema: Option[StructType]) - extends PartitioningAwareFileCatalog(sparkSession, parameters, partitionSchema) { + partitionSchema: Option[StructType], + fileStatusCache: FileStatusCache = NoopCache) + extends PartitioningAwareFileCatalog( + sparkSession, parameters, partitionSchema, fileStatusCache) { @volatile private var cachedLeafFiles: mutable.LinkedHashMap[Path, FileStatus] = _ @volatile private var cachedLeafDirToChildrenFiles: Map[Path, Array[FileStatus]] = _ @volatile private var cachedPartitionSpec: PartitionSpec = _ - refresh() + refresh0() override def partitionSpec(): PartitionSpec = { if (cachedPartitionSpec == null) { @@ -64,6 +66,11 @@ class ListingFileCatalog( } override def refresh(): Unit = { + refresh0() + fileStatusCache.invalidateAll() + } + + private def refresh0(): Unit = { val files = listLeafFiles(rootPaths) cachedLeafFiles = new mutable.LinkedHashMap[Path, FileStatus]() ++= files.map(f => f.getPath -> f) 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 5c8eff7ec46b4..9b1903c47119e 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 @@ -33,7 +33,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.util.SerializableConfiguration - /** * 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. @@ -45,7 +44,8 @@ import org.apache.spark.util.SerializableConfiguration abstract class PartitioningAwareFileCatalog( sparkSession: SparkSession, parameters: Map[String, String], - partitionSchema: Option[StructType]) extends FileCatalog with Logging { + partitionSchema: Option[StructType], + fileStatusCache: FileStatusCache = NoopCache) extends FileCatalog with Logging { import PartitioningAwareFileCatalog.BASE_PATH_PARAM /** Returns the specification of the partitions inferred from the data. */ @@ -238,15 +238,29 @@ abstract class PartitioningAwareFileCatalog( * This is publicly visible for testing. */ def listLeafFiles(paths: Seq[Path]): mutable.LinkedHashSet[FileStatus] = { - val files = - if (paths.length >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { - PartitioningAwareFileCatalog.listLeafFilesInParallel(paths, hadoopConf, sparkSession) - } else { - PartitioningAwareFileCatalog.listLeafFilesInSerial(paths, hadoopConf) + val output = mutable.LinkedHashSet[FileStatus]() + val pathsToFetch = mutable.ArrayBuffer[Path]() + for (path <- paths) { + fileStatusCache.getLeafFiles(path) match { + case Some(files) => + HiveCatalogMetrics.incrementFileCacheHits(files.length) + output ++= files + case None => + pathsToFetch += path } - - HiveCatalogMetrics.incrementFilesDiscovered(files.size) - mutable.LinkedHashSet(files: _*) + } + val discovered = if (pathsToFetch.length >= + sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { + PartitioningAwareFileCatalog.listLeafFilesInParallel(pathsToFetch, hadoopConf, sparkSession) + } else { + PartitioningAwareFileCatalog.listLeafFilesInSerial(pathsToFetch, hadoopConf) + } + discovered.foreach { case (path, leafFiles) => + HiveCatalogMetrics.incrementFilesDiscovered(leafFiles.size) + fileStatusCache.putLeafFiles(path, leafFiles.toArray) + output ++= leafFiles + } + output } } @@ -276,14 +290,14 @@ object PartitioningAwareFileCatalog extends Logging { */ private def listLeafFilesInSerial( paths: Seq[Path], - hadoopConf: Configuration): Seq[FileStatus] = { + hadoopConf: Configuration): Seq[(Path, Seq[FileStatus])] = { // Dummy jobconf to get to the pathFilter defined in configuration val jobConf = new JobConf(hadoopConf, this.getClass) val filter = FileInputFormat.getInputPathFilter(jobConf) - paths.flatMap { path => + paths.map { path => val fs = path.getFileSystem(hadoopConf) - listLeafFiles0(fs, path, filter) + (path, listLeafFiles0(fs, path, filter)) } } @@ -294,7 +308,7 @@ object PartitioningAwareFileCatalog extends Logging { private def listLeafFilesInParallel( paths: Seq[Path], hadoopConf: Configuration, - sparkSession: SparkSession): Seq[FileStatus] = { + sparkSession: SparkSession): Seq[(Path, Seq[FileStatus])] = { assert(paths.size >= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) logInfo(s"Listing leaf files and directories in parallel under: ${paths.mkString(", ")}") @@ -306,47 +320,54 @@ object PartitioningAwareFileCatalog extends Logging { // in case of large #defaultParallelism. val numParallelism = Math.min(paths.size, 10000) - val statuses = sparkContext + val statusMap = sparkContext .parallelize(serializedPaths, numParallelism) .mapPartitions { paths => val hadoopConf = serializableConfiguration.value listLeafFilesInSerial(paths.map(new Path(_)).toSeq, hadoopConf).iterator - }.map { status => - // Turn FileStatus into SerializableFileStatus so we can send it back to the driver - val blockLocations = status match { - case f: LocatedFileStatus => - f.getBlockLocations.map { loc => - SerializableBlockLocation( - loc.getNames, - loc.getHosts, - loc.getOffset, - loc.getLength) - } - - case _ => - Array.empty[SerializableBlockLocation] - } + }.map { case (path, statuses) => + val serializableStatuses = statuses.map { status => + // Turn FileStatus into SerializableFileStatus so we can send it back to the driver + val blockLocations = status match { + case f: LocatedFileStatus => + f.getBlockLocations.map { loc => + SerializableBlockLocation( + loc.getNames, + loc.getHosts, + loc.getOffset, + loc.getLength) + } + + case _ => + Array.empty[SerializableBlockLocation] + } - SerializableFileStatus( - status.getPath.toString, - status.getLen, - status.isDirectory, - status.getReplication, - status.getBlockSize, - status.getModificationTime, - status.getAccessTime, - blockLocations) + SerializableFileStatus( + status.getPath.toString, + status.getLen, + status.isDirectory, + status.getReplication, + status.getBlockSize, + status.getModificationTime, + status.getAccessTime, + blockLocations) + } + (path.toString, serializableStatuses) }.collect() - // Turn SerializableFileStatus back to Status - statuses.map { f => - val blockLocations = f.blockLocations.map { loc => - new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length) + // turn SerializableFileStatus back to Status + statusMap.map { case (path, serializableStatuses) => + val statuses = serializableStatuses.map { f => + val blockLocations = f.blockLocations.map { loc => + new BlockLocation(loc.names, loc.hosts, loc.offset, loc.length) + } + new LocatedFileStatus( + new FileStatus( + f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, + new Path(f.path)), + blockLocations) } - new LocatedFileStatus( - new FileStatus( - f.length, f.isDir, f.blockReplication, f.blockSize, f.modificationTime, new Path(f.path)), - blockLocations) + (new Path(path), statuses) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala index fc08c3798ee06..31a01bc6db082 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/TableFileCatalog.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.types.StructType * @param table the table's (unqualified) name * @param partitionSchema the schema of a partitioned table's partition columns * @param sizeInBytes the table's data size in bytes + * @param fileStatusCache optional cache implementation to use for file listing */ class TableFileCatalog( sparkSession: SparkSession, @@ -42,24 +43,21 @@ class TableFileCatalog( protected val hadoopConf = sparkSession.sessionState.newHadoopConf + private val fileStatusCache = FileStatusCache.newCache(sparkSession) + private val externalCatalog = sparkSession.sharedState.externalCatalog private val catalogTable = externalCatalog.getTable(db, table) private val baseLocation = catalogTable.storage.locationUri - // Populated on-demand by calls to cachedAllPartitions - private var cachedAllPartitions: ListingFileCatalog = null - override def rootPaths: Seq[Path] = baseLocation.map(new Path(_)).toSeq override def listFiles(filters: Seq[Expression]): Seq[PartitionDirectory] = { filterPartitions(filters).listFiles(Nil) } - override def refresh(): Unit = synchronized { - cachedAllPartitions = null - } + override def refresh(): Unit = fileStatusCache.invalidateAll() /** * Returns a [[ListingFileCatalog]] for this table restricted to the subset of partitions @@ -68,14 +66,6 @@ class TableFileCatalog( * @param filters partition-pruning filters */ def filterPartitions(filters: Seq[Expression]): ListingFileCatalog = { - if (filters.isEmpty) { - allPartitions - } else { - filterPartitions0(filters) - } - } - - private def filterPartitions0(filters: Seq[Expression]): ListingFileCatalog = { val parameters = baseLocation .map(loc => Map(PartitioningAwareFileCatalog.BASE_PATH_PARAM -> loc)) .getOrElse(Map.empty) @@ -87,21 +77,13 @@ class TableFileCatalog( } val partitionSpec = PartitionSpec(schema, partitions) new PrunedTableFileCatalog( - sparkSession, new Path(baseLocation.get), partitionSpec) + sparkSession, new Path(baseLocation.get), fileStatusCache, partitionSpec) case None => - new ListingFileCatalog(sparkSession, rootPaths, parameters, None) - } - } - - // Not used in the hot path of queries when metastore partition pruning is enabled - def allPartitions: ListingFileCatalog = synchronized { - if (cachedAllPartitions == null) { - cachedAllPartitions = filterPartitions0(Nil) + new ListingFileCatalog(sparkSession, rootPaths, parameters, None, fileStatusCache) } - cachedAllPartitions } - override def inputFiles: Array[String] = allPartitions.inputFiles + override def inputFiles: Array[String] = filterPartitions(Nil).inputFiles } /** @@ -114,9 +96,11 @@ class TableFileCatalog( private class PrunedTableFileCatalog( sparkSession: SparkSession, tableBasePath: Path, + fileStatusCache: FileStatusCache, override val partitionSpec: PartitionSpec) extends ListingFileCatalog( sparkSession, partitionSpec.partitions.map(_.path), Map.empty, - Some(partitionSpec.partitionColumns)) + Some(partitionSpec.partitionColumns), + fileStatusCache) 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 ebf4fad5cbcff..a6e2fa26cb5ef 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 @@ -265,17 +265,27 @@ object SQLConf { val HIVE_METASTORE_PARTITION_PRUNING = SQLConfigBuilder("spark.sql.hive.metastorePartitionPruning") .doc("When true, some predicates will be pushed down into the Hive metastore so that " + - "unmatching partitions can be eliminated earlier.") + "unmatching partitions can be eliminated earlier. This only affects Hive tables " + + "not converted to filesource relations (see HiveUtils.CONVERT_METASTORE_PARQUET and " + + "HiveUtils.CONVERT_METASTORE_ORC for more information).") .booleanConf .createWithDefault(true) val HIVE_FILESOURCE_PARTITION_PRUNING = SQLConfigBuilder("spark.sql.hive.filesourcePartitionPruning") - .doc("When true, enable metastore partition pruning for file source tables as well. " + + .doc("When true, enable metastore partition pruning for filesource relations as well. " + "This is currently implemented for converted Hive tables only.") .booleanConf .createWithDefault(true) + val HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE = + SQLConfigBuilder("spark.sql.hive.filesourcePartitionFileCacheSize") + .doc("When nonzero, enable caching of partition file metadata in memory. All table share " + + "a cache that can use up to specified num bytes for file metadata. This conf only " + + "applies if filesource partition pruning is also enabled.") + .longConf + .createWithDefault(250 * 1024 * 1024) + val OPTIMIZER_METADATA_ONLY = SQLConfigBuilder("spark.sql.optimizer.metadataOnly") .doc("When true, enable the metadata-only query optimization that use the table's metadata " + "to produce the partition columns instead of table scans. It applies when all the columns " + @@ -670,6 +680,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def filesourcePartitionPruning: Boolean = getConf(HIVE_FILESOURCE_PARTITION_PRUNING) + def filesourcePartitionFileCacheSize: Long = getConf(HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE) + def gatherFastStats: Boolean = getConf(GATHER_FASTSTAT) def optimizerMetadataOnly: Boolean = getConf(OPTIMIZER_METADATA_ONLY) 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 c909eb5d20bcd..44089335e1a1d 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 @@ -235,7 +235,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log if (lazyPruningEnabled) { catalog } else { - catalog.allPartitions + catalog.filterPartitions(Nil) // materialize all the partitions in memory } } val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet 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 81337493c7f28..d13e29b3029b1 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 @@ -577,5 +577,19 @@ class HiveDDLCommandSuite extends PlanTest with SQLTestUtils with TestHiveSingle assert(output == Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat")) assert(serde == Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) } - } + } + + test("table name with schema") { + // regression test for SPARK-11778 + 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") + } + + test("SPARK-15887: hive-site.xml should be loaded") { + val hiveClient = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client + assert(hiveClient.getConf("hive.in.test", "") == "true") + } } 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/HiveTablePerfStatsSuite.scala similarity index 50% rename from sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameSuite.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveTablePerfStatsSuite.scala index 15523437a3404..82ee813c6a95f 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/HiveTablePerfStatsSuite.scala @@ -19,25 +19,26 @@ package org.apache.spark.sql.hive import java.io.File +import org.scalatest.BeforeAndAfterEach + import org.apache.spark.metrics.source.HiveCatalogMetrics +import org.apache.spark.sql.execution.datasources.FileStatusCache import org.apache.spark.sql.QueryTest import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils -class HiveDataFrameSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { - test("table name with schema") { - // regression test for SPARK-11778 - 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") +class HiveTablePerfStatsSuite + extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach { + + override def beforeEach(): Unit = { + super.beforeEach() + FileStatusCache.resetForTesting() } - test("SPARK-15887: hive-site.xml should be loaded") { - val hiveClient = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client - assert(hiveClient.getConf("hive.in.test", "") == "true") + override def afterEach(): Unit = { + super.afterEach() + FileStatusCache.resetForTesting() } private def setupPartitionedTable(tableName: String, dir: File): Unit = { @@ -79,7 +80,9 @@ class HiveDataFrameSuite extends QueryTest with TestHiveSingleton with SQLTestUt } test("lazy partition pruning reads only necessary partition data") { - withSQLConf(SQLConf.HIVE_FILESOURCE_PARTITION_PRUNING.key -> "true") { + withSQLConf( + SQLConf.HIVE_FILESOURCE_PARTITION_PRUNING.key -> "true", + SQLConf.HIVE_FILESOURCE_PARTITION_FILE_CACHE_SIZE.key -> "0") { withTable("test") { withTempDir { dir => setupPartitionedTable("test", dir) @@ -104,11 +107,103 @@ class HiveDataFrameSuite extends QueryTest with TestHiveSingleton with SQLTestUt assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) - // read all should be cached + // read all should not be cached HiveCatalogMetrics.reset() spark.sql("select * from test").count() + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + + // cache should be disabled + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + } + } + } + } + + test("lazy partition pruning with file status caching enabled") { + withSQLConf( + "spark.sql.hive.filesourcePartitionPruning" -> "true", + "spark.sql.hive.filesourcePartitionFileCacheSize" -> "9999999") { + withTable("test") { + withTempDir { dir => + setupPartitionedTable("test", dir) + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test where partCol1 = 999").count() == 0) assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 0) assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test where partCol1 < 2").count() == 2) + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 2) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 2) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test where partCol1 < 3").count() == 3) + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 3) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 1) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 2) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 2) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 3) + + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 5) + } + } + } + } + + test("file status caching respects refresh table and refreshByPath") { + withSQLConf( + "spark.sql.hive.filesourcePartitionPruning" -> "true", + "spark.sql.hive.filesourcePartitionFileCacheSize" -> "9999999") { + withTable("test") { + withTempDir { dir => + setupPartitionedTable("test", dir) + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + + HiveCatalogMetrics.reset() + spark.sql("refresh table test") + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + + spark.catalog.cacheTable("test") + HiveCatalogMetrics.reset() + spark.catalog.refreshByPath(dir.getAbsolutePath) + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + } + } + } + } + + test("file status cache respects size limit") { + withSQLConf( + "spark.sql.hive.filesourcePartitionPruning" -> "true", + "spark.sql.hive.filesourcePartitionFileCacheSize" -> "1" /* 1 byte */) { + withTable("test") { + withTempDir { dir => + setupPartitionedTable("test", dir) + HiveCatalogMetrics.reset() + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) + assert(spark.sql("select * from test").count() == 5) + assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 10) + assert(HiveCatalogMetrics.METRIC_FILE_CACHE_HITS.getCount() == 0) } } } @@ -124,18 +219,18 @@ class HiveDataFrameSuite extends QueryTest with TestHiveSingleton with SQLTestUt // mode. This is kind of terrible, but is needed to preserve the legacy behavior // of doing plan cache validation based on the entire partition set. HiveCatalogMetrics.reset() - spark.sql("select * from test where partCol1 = 999").count() + assert(spark.sql("select * from test where partCol1 = 999").count() == 0) // 5 from table resolution, another 5 from ListingFileCatalog assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 10) assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 5) HiveCatalogMetrics.reset() - spark.sql("select * from test where partCol1 < 2").count() + assert(spark.sql("select * from test where partCol1 < 2").count() == 2) assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) HiveCatalogMetrics.reset() - spark.sql("select * from test").count() + assert(spark.sql("select * from test").count() == 5) assert(HiveCatalogMetrics.METRIC_PARTITIONS_FETCHED.getCount() == 5) assert(HiveCatalogMetrics.METRIC_FILES_DISCOVERED.getCount() == 0) } From 5fa9f8795a71e08bcbef5975ba8c072db5be8866 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Sat, 22 Oct 2016 20:09:04 +0200 Subject: [PATCH 34/48] [SPARK-17123][SQL] Use type-widened encoder for DataFrame rather than existing encoder to allow type-widening from set operations # What changes were proposed in this pull request? This PR fixes set operations in `DataFrame` to be performed fine without exceptions when the types are non-scala native types. (e.g, `TimestampType`, `DateType` and `DecimalType`). The problem is, it seems set operations such as `union`, `intersect` and `except` uses the encoder belonging to the `Dataset` in caller. So, `Dataset` of the caller holds `ExpressionEncoder[Row]` as it is when the set operations are performed. However, the return types can be actually widen. So, we should use `ExpressionEncoder[Row]` constructed from executed plan rather than using existing one. Otherwise, this will generate some codes wrongly via `StaticInvoke`. Running the codes below: ```scala val dates = Seq( (new Date(0), BigDecimal.valueOf(1), new Timestamp(2)), (new Date(3), BigDecimal.valueOf(4), new Timestamp(5)) ).toDF("date", "timestamp", "decimal") val widenTypedRows = Seq( (new Timestamp(2), 10.5D, "string") ).toDF("date", "timestamp", "decimal") val results = dates.union(widenTypedRows).collect() results.foreach(println) ``` prints below: **Before** ```java 23:08:54.490 ERROR org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 28, Column 107: No applicable constructor/method found for actual parameters "long"; candidates are: "public static java.sql.Date org.apache.spark.sql.catalyst.util.DateTimeUtils.toJavaDate(int)" /* 001 */ public java.lang.Object generate(Object[] references) { /* 002 */ return new SpecificSafeProjection(references); /* 003 */ } /* 004 */ /* 005 */ class SpecificSafeProjection extends org.apache.spark.sql.catalyst.expressions.codegen.BaseProjection { /* 006 */ /* 007 */ private Object[] references; /* 008 */ private MutableRow mutableRow; /* 009 */ private Object[] values; /* 010 */ private org.apache.spark.sql.types.StructType schema; /* 011 */ /* 012 */ /* 013 */ public SpecificSafeProjection(Object[] references) { /* 014 */ this.references = references; /* 015 */ mutableRow = (MutableRow) references[references.length - 1]; /* 016 */ /* 017 */ this.schema = (org.apache.spark.sql.types.StructType) references[0]; /* 018 */ } /* 019 */ /* 020 */ public java.lang.Object apply(java.lang.Object _i) { /* 021 */ InternalRow i = (InternalRow) _i; /* 022 */ /* 023 */ values = new Object[3]; /* 024 */ /* 025 */ boolean isNull2 = i.isNullAt(0); /* 026 */ long value2 = isNull2 ? -1L : (i.getLong(0)); /* 027 */ boolean isNull1 = isNull2; /* 028 */ final java.sql.Date value1 = isNull1 ? null : org.apache.spark.sql.catalyst.util.DateTimeUtils.toJavaDate(value2); /* 029 */ isNull1 = value1 == null; /* 030 */ if (isNull1) { /* 031 */ values[0] = null; /* 032 */ } else { /* 033 */ values[0] = value1; /* 034 */ } /* 035 */ /* 036 */ boolean isNull4 = i.isNullAt(1); /* 037 */ double value4 = isNull4 ? -1.0 : (i.getDouble(1)); /* 038 */ /* 039 */ boolean isNull3 = isNull4; /* 040 */ java.math.BigDecimal value3 = null; /* 041 */ if (!isNull3) { /* 042 */ /* 043 */ Object funcResult = null; /* 044 */ funcResult = value4.toJavaBigDecimal(); /* 045 */ if (funcResult == null) { /* 046 */ isNull3 = true; /* 047 */ } else { /* 048 */ value3 = (java.math.BigDecimal) funcResult; /* 049 */ } /* 050 */ /* 051 */ } /* 052 */ isNull3 = value3 == null; /* 053 */ if (isNull3) { /* 054 */ values[1] = null; /* 055 */ } else { /* 056 */ values[1] = value3; /* 057 */ } /* 058 */ /* 059 */ boolean isNull6 = i.isNullAt(2); /* 060 */ UTF8String value6 = isNull6 ? null : (i.getUTF8String(2)); /* 061 */ boolean isNull5 = isNull6; /* 062 */ final java.sql.Timestamp value5 = isNull5 ? null : org.apache.spark.sql.catalyst.util.DateTimeUtils.toJavaTimestamp(value6); /* 063 */ isNull5 = value5 == null; /* 064 */ if (isNull5) { /* 065 */ values[2] = null; /* 066 */ } else { /* 067 */ values[2] = value5; /* 068 */ } /* 069 */ /* 070 */ final org.apache.spark.sql.Row value = new org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema(values, schema); /* 071 */ if (false) { /* 072 */ mutableRow.setNullAt(0); /* 073 */ } else { /* 074 */ /* 075 */ mutableRow.update(0, value); /* 076 */ } /* 077 */ /* 078 */ return mutableRow; /* 079 */ } /* 080 */ } ``` **After** ```bash [1969-12-31 00:00:00.0,1.0,1969-12-31 16:00:00.002] [1969-12-31 00:00:00.0,4.0,1969-12-31 16:00:00.005] [1969-12-31 16:00:00.002,10.5,string] ``` ## How was this patch tested? Unit tests in `DataFrameSuite` Author: hyukjinkwon Closes #15072 from HyukjinKwon/SPARK-17123. --- .../scala/org/apache/spark/sql/Dataset.scala | 18 ++++++++++++++---- .../org/apache/spark/sql/DataFrameSuite.scala | 16 ++++++++++++++++ 2 files changed, 30 insertions(+), 4 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 073d2b1512b95..286d8549bfe27 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 @@ -556,7 +556,7 @@ class Dataset[T] private[sql]( * 1983 03 0.410516 0.442194 * 1984 04 0.450090 0.483521 * }}} - * + * * @param numRows Number of rows to show * @param truncate If set to more than 0, truncates strings to `truncate` characters and * all cells will be aligned right. @@ -1524,7 +1524,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)) @@ -1540,7 +1540,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) } @@ -1554,7 +1554,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) } @@ -2725,4 +2725,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 16cc368208485..e87baa454c8b3 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.util.Random @@ -1615,4 +1616,19 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { qe.assertAnalyzed() } } + + test("SPARK-17123: Performing set operations that combine non-scala native types") { + val dates = Seq( + (new Date(0), BigDecimal.valueOf(1), new Timestamp(2)), + (new Date(3), BigDecimal.valueOf(4), new Timestamp(5)) + ).toDF("date", "timestamp", "decimal") + + val widenTypedRows = Seq( + (new Timestamp(2), 10.5D, "string") + ).toDF("date", "timestamp", "decimal") + + dates.union(widenTypedRows).collect() + dates.except(widenTypedRows).collect() + dates.intersect(widenTypedRows).collect() + } } From 4f1dcd3dce270268b42fbe59409790364fa5c5df Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Sat, 22 Oct 2016 11:59:28 -0700 Subject: [PATCH 35/48] [SPARK-18051][SPARK CORE] fix bug of custom PartitionCoalescer causing serialization exception ## What changes were proposed in this pull request? add a require check in `CoalescedRDD` to make sure the passed in `partitionCoalescer` to be `serializable`. and update the document for api `RDD.coalesce` ## How was this patch tested? Manual.(test code in jira [SPARK-18051]) Author: WeichenXu Closes #15587 from WeichenXu123/fix_coalescer_bug. --- core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala | 4 ++++ core/src/main/scala/org/apache/spark/rdd/RDD.scala | 3 ++- 2 files changed, 6 insertions(+), 1 deletion(-) 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 9c198a61f37af..2cba1febe8759 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -80,6 +80,10 @@ private[spark] class CoalescedRDD[T: ClassTag]( require(maxPartitions > 0 || maxPartitions == prev.partitions.length, s"Number of partitions ($maxPartitions) must be positive.") + if (partitionCoalescer.isDefined) { + require(partitionCoalescer.get.isInstanceOf[Serializable], + "The partition coalescer passed in must be serializable.") + } override def getPartitions: Array[Partition] = { val pc = partitionCoalescer.getOrElse(new DefaultPartitionCoalescer()) 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 be119578d2c35..db535de9e9bb3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -432,7 +432,8 @@ abstract class RDD[T: ClassTag]( * of partitions. This is useful if you have a small number of partitions, * say 100, potentially with a few partitions being abnormally large. Calling * coalesce(1000, shuffle = true) will result in 1000 partitions with the - * data distributed using a hash partitioner. + * data distributed using a hash partitioner. The optional partition coalescer + * passed in must be serializable. */ def coalesce(numPartitions: Int, shuffle: Boolean = false, partitionCoalescer: Option[PartitionCoalescer] = Option.empty) From bc167a2a53f5a795d089e8a884569b1b3e2cd439 Mon Sep 17 00:00:00 2001 From: Sandeep Singh Date: Sat, 22 Oct 2016 12:03:37 -0700 Subject: [PATCH 36/48] [SPARK-928][CORE] Add support for Unsafe-based serializer in Kryo ## What changes were proposed in this pull request? Now since we have migrated to Kryo-3.0.0 in https://issues.apache.org/jira/browse/SPARK-11416, we can gives users option to use unsafe SerDer. It can turned by setting `spark.kryo.useUnsafe` to `true` ## How was this patch tested? Ran existing tests ``` Benchmark Kryo Unsafe vs safe Serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ basicTypes: Int unsafe:true 160 / 178 98.5 10.1 1.0X basicTypes: Long unsafe:true 210 / 218 74.9 13.4 0.8X basicTypes: Float unsafe:true 203 / 213 77.5 12.9 0.8X basicTypes: Double unsafe:true 226 / 235 69.5 14.4 0.7X Array: Int unsafe:true 1087 / 1101 14.5 69.1 0.1X Array: Long unsafe:true 2758 / 2844 5.7 175.4 0.1X Array: Float unsafe:true 1511 / 1552 10.4 96.1 0.1X Array: Double unsafe:true 2942 / 2972 5.3 187.0 0.1X Map of string->Double unsafe:true 2645 / 2739 5.9 168.2 0.1X basicTypes: Int unsafe:false 211 / 218 74.7 13.4 0.8X basicTypes: Long unsafe:false 247 / 253 63.6 15.7 0.6X basicTypes: Float unsafe:false 211 / 216 74.5 13.4 0.8X basicTypes: Double unsafe:false 227 / 233 69.2 14.4 0.7X Array: Int unsafe:false 3012 / 3032 5.2 191.5 0.1X Array: Long unsafe:false 4463 / 4515 3.5 283.8 0.0X Array: Float unsafe:false 2788 / 2868 5.6 177.2 0.1X Array: Double unsafe:false 3558 / 3752 4.4 226.2 0.0X Map of string->Double unsafe:false 2806 / 2933 5.6 178.4 0.1X ``` Author: Sandeep Singh Author: Sandeep Singh Closes #12913 from techaddict/SPARK-928. --- .../spark/serializer/KryoSerializer.scala | 36 +++-- .../spark/serializer/KryoBenchmark.scala | 139 ++++++++++++++++++ .../serializer/KryoSerializerSuite.scala | 1 + .../UnsafeKryoSerializerSuite.scala | 33 +++++ docs/configuration.md | 8 + 5 files changed, 206 insertions(+), 11 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala create mode 100644 core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 1fba552f70501..0d26281fe1076 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -27,6 +27,7 @@ import scala.reflect.ClassTag import com.esotericsoftware.kryo.{Kryo, KryoException, Serializer => KryoClassSerializer} import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} +import com.esotericsoftware.kryo.io.{UnsafeInput => KryoUnsafeInput, UnsafeOutput => KryoUnsafeOutput} import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator} import org.apache.avro.generic.{GenericData, GenericRecord} @@ -78,8 +79,15 @@ class KryoSerializer(conf: SparkConf) .filter(!_.isEmpty) private val avroSchemas = conf.getAvroSchema + // whether to use unsafe based IO for serialization + private val useUnsafe = conf.getBoolean("spark.kryo.unsafe", false) - def newKryoOutput(): KryoOutput = new KryoOutput(bufferSize, math.max(bufferSize, maxBufferSize)) + def newKryoOutput(): KryoOutput = + if (useUnsafe) { + new KryoUnsafeOutput(bufferSize, math.max(bufferSize, maxBufferSize)) + } else { + new KryoOutput(bufferSize, math.max(bufferSize, maxBufferSize)) + } def newKryo(): Kryo = { val instantiator = new EmptyScalaKryoInstantiator @@ -172,7 +180,7 @@ class KryoSerializer(conf: SparkConf) } override def newInstance(): SerializerInstance = { - new KryoSerializerInstance(this) + new KryoSerializerInstance(this, useUnsafe) } private[spark] override lazy val supportsRelocationOfSerializedObjects: Boolean = { @@ -186,9 +194,12 @@ class KryoSerializer(conf: SparkConf) private[spark] class KryoSerializationStream( serInstance: KryoSerializerInstance, - outStream: OutputStream) extends SerializationStream { + outStream: OutputStream, + useUnsafe: Boolean) extends SerializationStream { + + private[this] var output: KryoOutput = + if (useUnsafe) new KryoUnsafeOutput(outStream) else new KryoOutput(outStream) - private[this] var output: KryoOutput = new KryoOutput(outStream) private[this] var kryo: Kryo = serInstance.borrowKryo() override def writeObject[T: ClassTag](t: T): SerializationStream = { @@ -219,9 +230,12 @@ class KryoSerializationStream( private[spark] class KryoDeserializationStream( serInstance: KryoSerializerInstance, - inStream: InputStream) extends DeserializationStream { + inStream: InputStream, + useUnsafe: Boolean) extends DeserializationStream { + + private[this] var input: KryoInput = + if (useUnsafe) new KryoUnsafeInput(inStream) else new KryoInput(inStream) - private[this] var input: KryoInput = new KryoInput(inStream) private[this] var kryo: Kryo = serInstance.borrowKryo() override def readObject[T: ClassTag](): T = { @@ -248,8 +262,8 @@ class KryoDeserializationStream( } } -private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance { - +private[spark] class KryoSerializerInstance(ks: KryoSerializer, useUnsafe: Boolean) + extends SerializerInstance { /** * A re-used [[Kryo]] instance. Methods will borrow this instance by calling `borrowKryo()`, do * their work, then release the instance by calling `releaseKryo()`. Logically, this is a caching @@ -288,7 +302,7 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ // Make these lazy vals to avoid creating a buffer unless we use them. private lazy val output = ks.newKryoOutput() - private lazy val input = new KryoInput() + private lazy val input = if (useUnsafe) new KryoUnsafeInput() else new KryoInput() override def serialize[T: ClassTag](t: T): ByteBuffer = { output.clear() @@ -329,11 +343,11 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ } override def serializeStream(s: OutputStream): SerializationStream = { - new KryoSerializationStream(this, s) + new KryoSerializationStream(this, s, useUnsafe) } override def deserializeStream(s: InputStream): DeserializationStream = { - new KryoDeserializationStream(this, s) + new KryoDeserializationStream(this, s, useUnsafe) } /** diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala new file mode 100644 index 0000000000000..64be966276140 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.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.serializer + +import scala.reflect.ClassTag +import scala.util.Random + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.serializer.KryoTest._ +import org.apache.spark.util.Benchmark + +class KryoBenchmark extends SparkFunSuite { + val benchmark = new Benchmark("Benchmark Kryo Unsafe vs safe Serialization", 1024 * 1024 * 15, 10) + + ignore(s"Benchmark Kryo Unsafe vs safe Serialization") { + Seq (true, false).foreach (runBenchmark) + benchmark.run() + + // scalastyle:off + /* + Benchmark Kryo Unsafe vs safe Serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + basicTypes: Int with unsafe:true 151 / 170 104.2 9.6 1.0X + basicTypes: Long with unsafe:true 175 / 191 89.8 11.1 0.9X + basicTypes: Float with unsafe:true 177 / 184 88.8 11.3 0.9X + basicTypes: Double with unsafe:true 193 / 216 81.4 12.3 0.8X + Array: Int with unsafe:true 513 / 587 30.7 32.6 0.3X + Array: Long with unsafe:true 1211 / 1358 13.0 77.0 0.1X + Array: Float with unsafe:true 890 / 964 17.7 56.6 0.2X + Array: Double with unsafe:true 1335 / 1428 11.8 84.9 0.1X + Map of string->Double with unsafe:true 931 / 988 16.9 59.2 0.2X + basicTypes: Int with unsafe:false 197 / 217 79.9 12.5 0.8X + basicTypes: Long with unsafe:false 219 / 240 71.8 13.9 0.7X + basicTypes: Float with unsafe:false 208 / 217 75.7 13.2 0.7X + basicTypes: Double with unsafe:false 208 / 225 75.6 13.2 0.7X + Array: Int with unsafe:false 2559 / 2681 6.1 162.7 0.1X + Array: Long with unsafe:false 3425 / 3516 4.6 217.8 0.0X + Array: Float with unsafe:false 2025 / 2134 7.8 128.7 0.1X + Array: Double with unsafe:false 2241 / 2358 7.0 142.5 0.1X + Map of string->Double with unsafe:false 1044 / 1085 15.1 66.4 0.1X + */ + // scalastyle:on + } + + private def runBenchmark(useUnsafe: Boolean): Unit = { + def check[T: ClassTag](t: T, ser: SerializerInstance): Int = { + if (ser.deserialize[T](ser.serialize(t)) === t) 1 else 0 + } + + // Benchmark Primitives + val basicTypeCount = 1000000 + def basicTypes[T: ClassTag](name: String, gen: () => T): Unit = { + lazy val ser = createSerializer(useUnsafe) + val arrayOfBasicType: Array[T] = Array.fill(basicTypeCount)(gen()) + + benchmark.addCase(s"basicTypes: $name with unsafe:$useUnsafe") { _ => + var sum = 0L + var i = 0 + while (i < basicTypeCount) { + sum += check(arrayOfBasicType(i), ser) + i += 1 + } + sum + } + } + basicTypes("Int", Random.nextInt) + basicTypes("Long", Random.nextLong) + basicTypes("Float", Random.nextFloat) + basicTypes("Double", Random.nextDouble) + + // Benchmark Array of Primitives + val arrayCount = 10000 + def basicTypeArray[T: ClassTag](name: String, gen: () => T): Unit = { + lazy val ser = createSerializer(useUnsafe) + val arrayOfArrays: Array[Array[T]] = + Array.fill(arrayCount)(Array.fill[T](Random.nextInt(arrayCount))(gen())) + + benchmark.addCase(s"Array: $name with unsafe:$useUnsafe") { _ => + var sum = 0L + var i = 0 + while (i < arrayCount) { + val arr = arrayOfArrays(i) + sum += check(arr, ser) + i += 1 + } + sum + } + } + basicTypeArray("Int", Random.nextInt) + basicTypeArray("Long", Random.nextLong) + basicTypeArray("Float", Random.nextFloat) + basicTypeArray("Double", Random.nextDouble) + + // Benchmark Maps + val mapsCount = 1000 + lazy val ser = createSerializer(useUnsafe) + val arrayOfMaps: Array[Map[String, Double]] = Array.fill(mapsCount) { + Array.fill(Random.nextInt(mapsCount)) { + (Random.nextString(mapsCount / 10), Random.nextDouble()) + }.toMap + } + + benchmark.addCase(s"Map of string->Double with unsafe:$useUnsafe") { _ => + var sum = 0L + var i = 0 + while (i < mapsCount) { + val map = arrayOfMaps(i) + sum += check(map, ser) + i += 1 + } + sum + } + } + + def createSerializer(useUnsafe: Boolean): SerializerInstance = { + val conf = new SparkConf() + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName) + conf.set("spark.kryo.unsafe", useUnsafe.toString) + + new KryoSerializer(conf).newInstance() + } + +} diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index bc6e98365daef..5040841811054 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -36,6 +36,7 @@ import org.apache.spark.util.Utils class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName) + conf.set("spark.kryo.unsafe", "false") test("SPARK-7392 configuration limits") { val kryoBufferProperty = "spark.kryoserializer.buffer" diff --git a/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala new file mode 100644 index 0000000000000..d63a45ae4a6a9 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.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.serializer + +class UnsafeKryoSerializerSuite extends KryoSerializerSuite { + + // This test suite should run all tests in KryoSerializerSuite with kryo unsafe. + + override def beforeAll() { + conf.set("spark.kryo.unsafe", "true") + super.beforeAll() + } + + override def afterAll() { + conf.set("spark.kryo.unsafe", "false") + super.afterAll() + } +} diff --git a/docs/configuration.md b/docs/configuration.md index a4a99d6fa4630..b07867d99aa9d 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -799,6 +799,14 @@ Apart from these, the following properties are also available, and may be useful See the tuning guide for more details.
spark.kryo.unsafefalse + Whether to use unsafe based Kryo serializer. Can be + substantially faster by using Unsafe Based IO. +
spark.kryoserializer.buffer.max 64m