From cbcbc76e8fc8faa982913d15a166716bf76b84b4 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Wed, 13 Sep 2023 14:44:19 -0500 Subject: [PATCH 01/46] xfail fixed_length_byte_array.parquet test due to rapidsai/cudf#14104 (#9235) Signed-off-by: Jason Lowe --- integration_tests/src/main/python/parquet_testing_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration_tests/src/main/python/parquet_testing_test.py b/integration_tests/src/main/python/parquet_testing_test.py index d2d973c9021..92070214298 100644 --- a/integration_tests/src/main/python/parquet_testing_test.py +++ b/integration_tests/src/main/python/parquet_testing_test.py @@ -39,7 +39,6 @@ # When the association is a pair rather than a string, it's a way to xfail the test # by providing the error string and xfail reason. _error_files = { - "fixed_length_byte_array.parquet": "Exception", "large_string_map.brotli.parquet": "Exception", "nation.dict-malformed.parquet": ("Exception", "https://github.com/NVIDIA/spark-rapids/issues/8644"), "non_hadoop_lz4_compressed.parquet": "Exception", @@ -50,6 +49,7 @@ # xfail reason message. _xfail_files = { "byte_array_decimal.parquet": "https://github.com/NVIDIA/spark-rapids/issues/8629", + "fixed_length_byte_array.parquet": "https://github.com/rapidsai/cudf/issues/14104", "datapage_v2.snappy.parquet": "datapage v2 not supported by cudf", "delta_binary_packed.parquet": "https://github.com/rapidsai/cudf/issues/13501", "delta_byte_array.parquet": "https://github.com/rapidsai/cudf/issues/13501", From 204956fdcd672c143b9cdffdc41569f08564a812 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Wed, 13 Sep 2023 16:16:58 -0500 Subject: [PATCH 02/46] Check for null keys when creating map (#9237) Signed-off-by: Robert (Bobby) Evans --- integration_tests/src/main/python/map_test.py | 6 ++++++ .../org/apache/spark/sql/rapids/complexTypeCreator.scala | 1 + 2 files changed, 7 insertions(+) diff --git a/integration_tests/src/main/python/map_test.py b/integration_tests/src/main/python/map_test.py index 4ec9fdda86b..bf31dfebb43 100644 --- a/integration_tests/src/main/python/map_test.py +++ b/integration_tests/src/main/python/map_test.py @@ -293,6 +293,12 @@ def test_map_expr_expr_keys_dupe_exception(): conf={'spark.sql.mapKeyDedupPolicy':'EXCEPTION'}, error_message = "Duplicate map key") +def test_map_keys_null_exception(): + assert_gpu_and_cpu_error( + lambda spark: spark.sql( + "select map(x, -1) from (select explode(array(1,null)) as x)").collect(), + conf = {}, + error_message = "Cannot use null as map key") def test_map_expr_literal_keys_dupe_last_win(): data_gen = [('a', StringGen(nullable=False)), ('b', StringGen(nullable=False))] diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/complexTypeCreator.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/complexTypeCreator.scala index dfc95016363..b47de2cdac2 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/complexTypeCreator.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/complexTypeCreator.scala @@ -145,6 +145,7 @@ object GpuCreateMap { def createMapFromKeysValuesAsStructs(dataType: MapType, listsOfKeyValueStructs : ColumnView): GpuColumnVector = { + GpuMapUtils.assertNoNullKeys(listsOfKeyValueStructs) withResource(listsOfKeyValueStructs.dropListDuplicatesWithKeysValues()) { deduped => if (exceptionOnDupKeys) { // Compare child data row count before and after removing duplicates to determine From 4826b491d3763c2b33c79e3e7adff7a45315f651 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 13 Sep 2023 17:00:20 -0600 Subject: [PATCH 03/46] Enable Spark 350 builds (#9234) * Enable Spark 350 builds * signoff Signed-off-by: Andy Grove --------- Signed-off-by: Andy Grove --- pom.xml | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 02b59f0cd1a..e672d2a9e17 100644 --- a/pom.xml +++ b/pom.xml @@ -699,11 +699,10 @@ 333, 330cdh, 340, - 341 + 341, + 350 - - 321db, From 34d615dae8d7e76c49d304dbe2d5a7ca8895dfd5 Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Fri, 15 Sep 2023 09:16:09 -0500 Subject: [PATCH 04/46] Allow skipping host spill for a direct device->disk spill (#9211) * Refactor code to allow skipping host * Implement RapidsBufferChannelWritable in RapidsTable and RapidsDeviceMemoryBuffer * Add a test for device->disk skipping host * Small fixes * getMemoryUsedBytes -> memoryUsedBytes as a val Signed-off-by: Alessandro Bellina * Update copyright * Fix bug where buffer could be spilled to a lower tier while it was being spilled at a higher tier * Fix leak in aggregate when there are retries Signed-off-by: Alessandro Bellina * Ensure that 0-byte RapidsBuffers are never spillable * Remove spillBuffer from catalog * Fix test issues --------- Signed-off-by: Alessandro Bellina --- .../AbstractHostByteBufferIterator.scala | 103 ++++++++ .../spark/rapids/HostByteBufferIterator.scala | 47 ---- .../nvidia/spark/rapids/RapidsBuffer.scala | 13 +- .../spark/rapids/RapidsBufferCatalog.scala | 197 +++++---------- .../spark/rapids/RapidsBufferStore.scala | 174 ++++++++++++-- .../com/nvidia/spark/rapids/RapidsConf.scala | 12 + .../rapids/RapidsDeviceMemoryStore.scala | 86 +++++-- .../nvidia/spark/rapids/RapidsDiskStore.scala | 25 +- .../nvidia/spark/rapids/RapidsGdsStore.scala | 10 +- .../spark/rapids/RapidsHostMemoryStore.scala | 110 ++++++--- .../spark/rapids/SpillableColumnarBatch.scala | 4 +- .../shuffle/RapidsShuffleIterator.scala | 2 +- .../rapids/RapidsBufferCatalogSuite.scala | 6 +- .../rapids/RapidsDeviceMemoryStoreSuite.scala | 8 +- .../spark/rapids/RapidsDiskStoreSuite.scala | 224 +++++++++++++++++- .../spark/rapids/RapidsGdsStoreSuite.scala | 4 +- .../rapids/RapidsHostMemoryStoreSuite.scala | 12 +- .../shuffle/RapidsShuffleIteratorSuite.scala | 2 +- .../rapids/SpillableColumnarBatchSuite.scala | 2 +- 19 files changed, 747 insertions(+), 294 deletions(-) create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/AbstractHostByteBufferIterator.scala delete mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostByteBufferIterator.scala diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AbstractHostByteBufferIterator.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AbstractHostByteBufferIterator.scala new file mode 100644 index 00000000000..24f3bacf28b --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AbstractHostByteBufferIterator.scala @@ -0,0 +1,103 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 com.nvidia.spark.rapids + +import java.nio.ByteBuffer + +import ai.rapids.cudf.{Cuda, HostMemoryBuffer, MemoryBuffer} + +abstract class AbstractHostByteBufferIterator + extends Iterator[ByteBuffer] { + private[this] var nextBufferStart: Long = 0L + + val totalLength: Long + + protected val limit: Long = Integer.MAX_VALUE + + def getByteBuffer(offset: Long, length: Long): ByteBuffer + + override def hasNext: Boolean = nextBufferStart < totalLength + + override def next(): ByteBuffer = { + val offset = nextBufferStart + val length = Math.min(totalLength - nextBufferStart, limit) + nextBufferStart += length + getByteBuffer(offset, length) + } +} + +/** + * Create an iterator that will emit ByteBuffer instances sequentially + * to work around the 2GB ByteBuffer size limitation. This allows + * the entire address range of a >2GB host buffer to be covered + * by a sequence of ByteBuffer instances. + *

NOTE: It is the caller's responsibility to ensure this iterator + * does not outlive the host buffer. The iterator DOES NOT increment + * the reference count of the host buffer to ensure it remains valid. + * + * @param hostBuffer host buffer to iterate + * @return ByteBuffer iterator + */ +class HostByteBufferIterator(hostBuffer: HostMemoryBuffer) + extends AbstractHostByteBufferIterator { + override protected val limit: Long = Integer.MAX_VALUE + + override val totalLength: Long = if (hostBuffer == null) { + 0 + } else { + hostBuffer.getLength + } + + override def getByteBuffer(offset: Long, length: Long): ByteBuffer = { + hostBuffer.asByteBuffer(offset, length.toInt) + } +} + +/** + * Create an iterator that will emit ByteBuffer instances sequentially + * to work around the 2GB ByteBuffer size limitation after copying a `MemoryBuffer` + * (which is likely a `DeviceMemoryBuffer`) to a host-backed bounce buffer + * that is likely smaller than 2GB. + * @note It is the caller's responsibility to ensure this iterator + * does not outlive `memoryBuffer`. The iterator DOES NOT increment + * the reference count of `memoryBuffer` to ensure it remains valid. + * @param memoryBuffer memory buffer to copy. This is likely a DeviceMemoryBuffer + * @param bounceBuffer a host bounce buffer that will be used to stage copies onto the host + * @param stream stream to synchronize on after staging to bounceBuffer + * @return ByteBuffer iterator + */ +class MemoryBufferToHostByteBufferIterator( + memoryBuffer: MemoryBuffer, + bounceBuffer: HostMemoryBuffer, + stream: Cuda.Stream) + extends AbstractHostByteBufferIterator { + override val totalLength: Long = if (memoryBuffer == null) { + 0 + } else { + memoryBuffer.getLength + } + + override protected val limit: Long = + Math.min(bounceBuffer.getLength, Integer.MAX_VALUE) + + override def getByteBuffer(offset: Long, length: Long): ByteBuffer = { + bounceBuffer + .copyFromMemoryBufferAsync(0, memoryBuffer, offset, length, stream) + stream.sync() + bounceBuffer.asByteBuffer(0, length.toInt) + } +} \ No newline at end of file diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostByteBufferIterator.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostByteBufferIterator.scala deleted file mode 100644 index 5df9f220435..00000000000 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostByteBufferIterator.scala +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Copyright (c) 2020, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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 com.nvidia.spark.rapids - -import java.nio.ByteBuffer - -import ai.rapids.cudf.HostMemoryBuffer - -/** - * Create an iterator that will emit ByteBuffer instances sequentially - * to work around the 2GB ByteBuffer size limitation. This allows - * the entire address range of a >2GB host buffer to be covered - * by a sequence of ByteBuffer instances. - *

NOTE: It is the caller's responsibility to ensure this iterator - * does not outlive the host buffer. The iterator DOES NOT increment - * the reference count of the host buffer to ensure it remains valid. - * - * @param hostBuffer host buffer to iterate - * @return ByteBuffer iterator - */ -class HostByteBufferIterator(hostBuffer: HostMemoryBuffer) - extends Iterator[ByteBuffer] { - private[this] var nextBufferStart: Long = 0L - - override def hasNext: Boolean = hostBuffer != null && nextBufferStart < hostBuffer.getLength - - override def next(): ByteBuffer = { - val offset = nextBufferStart - val length = Math.min(hostBuffer.getLength - nextBufferStart, Integer.MAX_VALUE) - nextBufferStart += length - hostBuffer.asByteBuffer(offset, length.toInt) - } -} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala index ed0699e92f0..4bdf2493647 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala @@ -230,14 +230,14 @@ trait RapidsBuffer extends AutoCloseable { * * @note Do not use this size to allocate a target buffer to copy, always use `getPackedSize.` */ - def getMemoryUsedBytes: Long + val memoryUsedBytes: Long /** * The size of this buffer if it has already gone through contiguous_split. * * @note Use this function when allocating a target buffer for spill or shuffle purposes. */ - def getPackedSizeBytes: Long = getMemoryUsedBytes + def getPackedSizeBytes: Long = memoryUsedBytes /** * At spill time, obtain an iterator used to copy this buffer to a different tier. @@ -389,7 +389,7 @@ sealed class DegenerateRapidsBuffer( override val id: RapidsBufferId, override val meta: TableMeta) extends RapidsBuffer { - override def getMemoryUsedBytes: Long = 0L + override val memoryUsedBytes: Long = 0L override val storageTier: StorageTier = StorageTier.DEVICE @@ -451,7 +451,7 @@ trait RapidsHostBatchBuffer extends AutoCloseable { */ def getHostColumnarBatch(sparkTypes: Array[DataType]): ColumnarBatch - def getMemoryUsedBytes(): Long + val memoryUsedBytes: Long } trait RapidsBufferChannelWritable { @@ -459,7 +459,10 @@ trait RapidsBufferChannelWritable { * At spill time, write this buffer to an nio WritableByteChannel. * @param writableChannel that this buffer can just write itself to, either byte-for-byte * or via serialization if needed. + * @param stream the Cuda.Stream for the spilling thread. If the `RapidsBuffer` that + * implements this method is on the device, synchronization may be needed + * for staged copies. * @return the amount of bytes written to the channel */ - def writeToChannel(writableChannel: WritableByteChannel): Long + def writeToChannel(writableChannel: WritableByteChannel, stream: Cuda.Stream): Long } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala index 45fa981cff5..5c58e73bf55 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala @@ -19,9 +19,7 @@ package com.nvidia.spark.rapids import java.util.concurrent.ConcurrentHashMap import java.util.function.BiFunction -import scala.collection.mutable.ArrayBuffer - -import ai.rapids.cudf.{ContiguousTable, Cuda, DeviceMemoryBuffer, HostMemoryBuffer, MemoryBuffer, NvtxColor, NvtxRange, Rmm, Table} +import ai.rapids.cudf.{ContiguousTable, Cuda, DeviceMemoryBuffer, HostMemoryBuffer, MemoryBuffer, Rmm, Table} import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import com.nvidia.spark.rapids.RapidsBufferCatalog.getExistingRapidsBufferAndAcquire import com.nvidia.spark.rapids.RapidsPluginImplicits._ @@ -32,7 +30,6 @@ import com.nvidia.spark.rapids.jni.RmmSpark import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.internal.Logging import org.apache.spark.sql.rapids.{RapidsDiskBlockManager, TempSpillBufferId} -import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.vectorized.ColumnarBatch /** @@ -378,7 +375,28 @@ class RapidsBufferCatalog( table: Table, initialSpillPriority: Long, needsSync: Boolean = true): RapidsBufferHandle = { - val id = TempSpillBufferId() + addTable(TempSpillBufferId(), table, initialSpillPriority, needsSync) + } + + /** + * Adds a table to the device storage. + * + * This takes ownership of the table. The reason for this is that tables + * don't have a reference count, so we cannot cleanly capture ownership by increasing + * ref count and decreasing from the caller. + * + * @param id specific RapidsBufferId to use for this table + * @param table table that will be owned by the store + * @param initialSpillPriority starting spill priority value + * @param needsSync whether the spill framework should stream synchronize while adding + * this table (defaults to true) + * @return RapidsBufferHandle handle for this RapidsBuffer + */ + def addTable( + id: RapidsBufferId, + table: Table, + initialSpillPriority: Long, + needsSync: Boolean): RapidsBufferHandle = { val rapidsBuffer = deviceStorage.addTable( id, table, @@ -442,7 +460,7 @@ class RapidsBufferCatalog( */ def acquireBuffer(handle: RapidsBufferHandle): RapidsBuffer = { val id = handle.id - (0 until RapidsBufferCatalog.MAX_BUFFER_LOOKUP_ATTEMPTS).foreach { _ => + def lookupAndReturn: Option[RapidsBuffer] = { val buffers = bufferMap.get(id) if (buffers == null || buffers.isEmpty) { throw new NoSuchElementException( @@ -450,7 +468,27 @@ class RapidsBufferCatalog( } val buffer = buffers.head if (buffer.addReference()) { - return buffer + Some(buffer) + } else { + None + } + } + + // fast path + (0 until RapidsBufferCatalog.MAX_BUFFER_LOOKUP_ATTEMPTS).foreach { _ => + val mayBuffer = lookupAndReturn + if (mayBuffer.isDefined) { + return mayBuffer.get + } + } + + // try one last time after locking the catalog (slow path) + // if there is a lot of contention here, I would rather lock the world than + // have tasks error out with "Unable to acquire" + synchronized { + val mayBuffer = lookupAndReturn + if (mayBuffer.isDefined) { + return mayBuffer.get } } throw new IllegalStateException(s"Unable to acquire buffer for ID: $id") @@ -552,14 +590,9 @@ class RapidsBufferCatalog( store: RapidsBufferStore, targetTotalSize: Long, stream: Cuda.Stream = Cuda.DEFAULT_STREAM): Option[Long] = { - val spillStore = store.spillStore - if (spillStore == null) { + if (store.spillStore == null) { throw new OutOfMemoryError("Requested to spill without a spill store") } - - // total amount spilled in this invocation - var totalSpilled: Long = 0 - require(targetTotalSize >= 0, s"Negative spill target size: $targetTotalSize") val mySpillCount = spillCount @@ -573,124 +606,21 @@ class RapidsBufferCatalog( // None which lets the calling code know that rmm should retry allocation None } else { - // this thread win the race and should spill + // this thread wins the race and should spill spillCount += 1 - - logWarning(s"Targeting a ${store.name} size of $targetTotalSize. " + - s"Current total ${store.currentSize}. " + - s"Current spillable ${store.currentSpillableSize}") - - if (store.currentSpillableSize > targetTotalSize) { - withResource(new NvtxRange(s"${store.name} sync spill", NvtxColor.ORANGE)) { _ => - logWarning(s"${store.name} store spilling to reduce usage from " + - s"${store.currentSize} total (${store.currentSpillableSize} spillable) " + - s"to $targetTotalSize bytes") - - // If the store has 0 spillable bytes left, it has exhausted. - var exhausted = false - - val buffersToFree = new ArrayBuffer[RapidsBuffer]() - - try { - while (!exhausted && - store.currentSpillableSize > targetTotalSize) { - val nextSpillable = store.nextSpillable() - if (nextSpillable != null) { - // we have a buffer (nextSpillable) to spill - spillBuffer(nextSpillable, spillStore, stream) - .foreach(buffersToFree.append(_)) - totalSpilled += nextSpillable.getMemoryUsedBytes - } - } - if (totalSpilled <= 0) { - // we didn't spill in this iteration, exit loop - exhausted = true - logWarning("Unable to spill enough to meet request. " + - s"Total=${store.currentSize} " + - s"Spillable=${store.currentSpillableSize} " + - s"Target=$targetTotalSize") - } - } finally { - if (buffersToFree.nonEmpty) { - // This is a hack in order to completely synchronize with the GPU before we free - // a buffer. It is necessary because of non-synchronous cuDF calls that could fall - // behind where the CPU is. Freeing a rapids buffer in these cases needs to wait for - // all launched GPU work, otherwise crashes or data corruption could occur. - // A more performant implementation would be to synchronize on the thread that read - // the buffer via events. - // https://github.com/NVIDIA/spark-rapids/issues/8610 - Cuda.deviceSynchronize() - buffersToFree.safeFree() - } - } - } - } - Some(totalSpilled) - } - } - } - - /** - * Given a specific `RapidsBuffer` spill it to `spillStore` - * @return the buffer, if successfully spilled, in order for the caller to free it - * @note called with catalog lock held - */ - private def spillBuffer( - buffer: RapidsBuffer, - spillStore: RapidsBufferStore, - stream: Cuda.Stream): Option[RapidsBuffer] = { - if (buffer.addReference()) { - withResource(buffer) { _ => - logDebug(s"Spilling $buffer ${buffer.id} to ${spillStore.name}") - val bufferHasSpilled = isBufferSpilled(buffer.id, buffer.storageTier) - if (!bufferHasSpilled) { - // if the spillStore specifies a maximum size spill taking this ceiling - // into account before trying to create a buffer there - // TODO: we may need to handle what happens if we can't spill anymore - // because all host buffers are being referenced. - trySpillToMaximumSize(buffer, spillStore, stream) - - // copy the buffer to spillStore - val newBuffer = spillStore.copyBuffer(buffer, stream) - - // once spilled, we get back a new RapidsBuffer instance in this new tier - registerNewBuffer(newBuffer) - } else { - logDebug(s"Skipping spilling $buffer ${buffer.id} to ${spillStore.name} as it is " + - s"already stored in multiple tiers") - } + Some(store.synchronousSpill(targetTotalSize, this, stream)) } - // we can now remove the old tier linkage - removeBufferTier(buffer.id, buffer.storageTier) - - // return the buffer - Some(buffer) - } else { - None } } - /** - * If `spillStore` defines a maximum size, spill to make room for `buffer`. - */ - private def trySpillToMaximumSize( - buffer: RapidsBuffer, - spillStore: RapidsBufferStore, - stream: Cuda.Stream): Unit = { - val spillStoreMaxSize = spillStore.getMaxSize - if (spillStoreMaxSize.isDefined) { - // this spillStore has a maximum size requirement (host only). We need to spill from it - // in order to make room for `buffer`. - val targetTotalSize = - math.max(spillStoreMaxSize.get - buffer.getMemoryUsedBytes, 0) - val maybeAmountSpilled = synchronousSpill(spillStore, targetTotalSize, stream) - maybeAmountSpilled.foreach { amountSpilled => - if (amountSpilled != 0) { - logInfo(s"Spilled $amountSpilled bytes from the ${spillStore.name} store") - TrampolineUtil.incTaskMetricsDiskBytesSpilled(amountSpilled) - } - } - } + def updateTiers(bufferSpill: BufferSpill): Long = bufferSpill match { + case BufferSpill(spilledBuffer, maybeNewBuffer) => + logDebug(s"Spilled ${spilledBuffer.id} from tier ${spilledBuffer.storageTier}. " + + s"Removing. Registering ${maybeNewBuffer.map(_.id).getOrElse ("None")} " + + s"${maybeNewBuffer}") + maybeNewBuffer.foreach(registerNewBuffer) + removeBufferTier(spilledBuffer.id, spilledBuffer.storageTier) + spilledBuffer.memoryUsedBytes } /** @@ -707,10 +637,12 @@ class RapidsBufferCatalog( // do not create a new one, else add a reference acquireBuffer(buffer.id, StorageTier.DEVICE) match { case None => - val newBuffer = deviceStorage.copyBuffer(buffer, stream) - newBuffer.addReference() // add a reference since we are about to use it - registerNewBuffer(newBuffer) - newBuffer + val maybeNewBuffer = deviceStorage.copyBuffer(buffer, this, stream) + maybeNewBuffer.map { newBuffer => + newBuffer.addReference() // add a reference since we are about to use it + registerNewBuffer(newBuffer) + newBuffer + }.get // the GPU store has to return a buffer here for now, or throw OOM case Some(existingBuffer) => existingBuffer } } @@ -764,7 +696,6 @@ class RapidsBufferCatalog( } object RapidsBufferCatalog extends Logging { - private val MAX_BUFFER_LOOKUP_ATTEMPTS = 100 private var deviceStorage: RapidsDeviceMemoryStore = _ @@ -841,7 +772,9 @@ object RapidsBufferCatalog extends Logging { // We are going to re-initialize so make sure all of the old things were closed... closeImpl() assert(memoryEventHandler == null) - deviceStorage = new RapidsDeviceMemoryStore(rapidsConf.chunkedPackBounceBufferSize) + deviceStorage = new RapidsDeviceMemoryStore( + rapidsConf.chunkedPackBounceBufferSize, + rapidsConf.spillToDiskBounceBufferSize) diskBlockManager = new RapidsDiskBlockManager(conf) if (rapidsConf.isGdsSpillEnabled) { gdsStorage = new RapidsGdsStore(diskBlockManager, rapidsConf.gdsSpillBatchWriteBufferSize) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala index ecbd8e64259..090d1fc5f65 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala @@ -21,7 +21,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock import scala.collection.mutable -import ai.rapids.cudf.{BaseDeviceMemoryBuffer, Cuda, DeviceMemoryBuffer, HostMemoryBuffer, MemoryBuffer} +import ai.rapids.cudf.{BaseDeviceMemoryBuffer, Cuda, DeviceMemoryBuffer, HostMemoryBuffer, MemoryBuffer, NvtxColor, NvtxRange} import com.nvidia.spark.rapids.Arm._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.StorageTier.{DEVICE, StorageTier} @@ -32,6 +32,15 @@ import org.apache.spark.sql.rapids.GpuTaskMetrics import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch +/** + * A helper case class that contains the buffer we spilled from our current tier + * and likely a new buffer created in a spill store tier, but it can be set to None. + * If the buffer already exists in the target spill store, `newBuffer` will be None. + * @param spilledBuffer a `RapidsBuffer` we spilled from this store + * @param newBuffer an optional `RapidsBuffer` in the target spill store. + */ +case class BufferSpill(spilledBuffer: RapidsBuffer, newBuffer: Option[RapidsBuffer]) + /** * Base class for all buffer store types. * @@ -67,14 +76,14 @@ abstract class RapidsBufferStore(val tier: StorageTier) if (old != null) { throw new DuplicateBufferException(s"duplicate buffer registered: ${buffer.id}") } - totalBytesStored += buffer.getMemoryUsedBytes + totalBytesStored += buffer.memoryUsedBytes // device buffers "spillability" is handled via DeviceMemoryBuffer ref counting // so spillableOnAdd should be false, all other buffer tiers are spillable at // all times. - if (spillableOnAdd) { + if (spillableOnAdd && buffer.memoryUsedBytes > 0) { if (spillable.offer(buffer)) { - totalBytesSpillable += buffer.getMemoryUsedBytes + totalBytesSpillable += buffer.memoryUsedBytes } } } @@ -84,9 +93,9 @@ abstract class RapidsBufferStore(val tier: StorageTier) spilling.remove(id) val obj = buffers.remove(id) if (obj != null) { - totalBytesStored -= obj.getMemoryUsedBytes + totalBytesStored -= obj.memoryUsedBytes if (spillable.remove(obj)) { - totalBytesSpillable -= obj.getMemoryUsedBytes + totalBytesSpillable -= obj.memoryUsedBytes } } } @@ -115,19 +124,19 @@ abstract class RapidsBufferStore(val tier: StorageTier) * @param isSpillable whether the buffer should now be spillable */ def setSpillable(buffer: RapidsBufferBase, isSpillable: Boolean): Unit = synchronized { - if (isSpillable) { + if (isSpillable && buffer.memoryUsedBytes > 0) { // if this buffer is in the store and isn't currently spilling if (!spilling.contains(buffer.id) && buffers.containsKey(buffer.id)) { // try to add it to the spillable collection if (spillable.offer(buffer)) { - totalBytesSpillable += buffer.getMemoryUsedBytes + totalBytesSpillable += buffer.memoryUsedBytes logDebug(s"Buffer ${buffer.id} is spillable. " + s"total=${totalBytesStored} spillable=${totalBytesSpillable}") } // else it was already there (unlikely) } } else { if (spillable.remove(buffer)) { - totalBytesSpillable -= buffer.getMemoryUsedBytes + totalBytesSpillable -= buffer.memoryUsedBytes logDebug(s"Buffer ${buffer.id} is not spillable. " + s"total=${totalBytesStored}, spillable=${totalBytesSpillable}") } // else it was already removed @@ -139,8 +148,8 @@ abstract class RapidsBufferStore(val tier: StorageTier) if (buffer != null) { // mark the id as "spilling" (this buffer is in the middle of a spill operation) spilling.add(buffer.id) - totalBytesSpillable -= buffer.getMemoryUsedBytes - logDebug(s"Spilling buffer ${buffer.id}. size=${buffer.getMemoryUsedBytes} " + + totalBytesSpillable -= buffer.memoryUsedBytes + logDebug(s"Spilling buffer ${buffer.id}. size=${buffer.memoryUsedBytes} " + s"total=${totalBytesStored}, new spillable=${totalBytesSpillable}") } buffer @@ -196,15 +205,19 @@ abstract class RapidsBufferStore(val tier: StorageTier) * (i.e.: this method will not take ownership of the incoming buffer object). * This does not need to update the catalog, the caller is responsible for that. * @param buffer data from another store + * @param catalog RapidsBufferCatalog we may need to modify during this copy * @param stream CUDA stream to use for copy or null * @return the new buffer that was created */ def copyBuffer( buffer: RapidsBuffer, - stream: Cuda.Stream): RapidsBufferBase = { - freeOnExcept(createBuffer(buffer, stream)) { newBuffer => - addBuffer(newBuffer) - newBuffer + catalog: RapidsBufferCatalog, + stream: Cuda.Stream): Option[RapidsBufferBase] = { + createBuffer(buffer, catalog, stream).map { newBuffer => + freeOnExcept(newBuffer) { newBuffer => + addBuffer(newBuffer) + newBuffer + } } } @@ -220,12 +233,14 @@ abstract class RapidsBufferStore(val tier: StorageTier) * @note DO NOT close the buffer unless adding a reference! * @note `createBuffer` impls should synchronize against `stream` before returning, if needed. * @param buffer data from another store + * @param catalog RapidsBufferCatalog we may need to modify during this create * @param stream CUDA stream to use or null * @return the new buffer that was created. */ protected def createBuffer( buffer: RapidsBuffer, - stream: Cuda.Stream): RapidsBufferBase + catalog: RapidsBufferCatalog, + stream: Cuda.Stream): Option[RapidsBufferBase] /** Update bookkeeping for a new buffer */ protected def addBuffer(buffer: RapidsBufferBase): Unit = { @@ -255,6 +270,129 @@ abstract class RapidsBufferStore(val tier: StorageTier) buffers.nextSpillableBuffer() } + def synchronousSpill( + targetTotalSize: Long, + catalog: RapidsBufferCatalog, + stream: Cuda.Stream = Cuda.DEFAULT_STREAM): Long = { + if (currentSpillableSize > targetTotalSize) { + logWarning(s"Targeting a ${name} size of $targetTotalSize. " + + s"Current total ${currentSize}. " + + s"Current spillable ${currentSpillableSize}") + val bufferSpills = new mutable.ArrayBuffer[BufferSpill]() + withResource(new NvtxRange(s"${name} sync spill", NvtxColor.ORANGE)) { _ => + logWarning(s"${name} store spilling to reduce usage from " + + s"${currentSize} total (${currentSpillableSize} spillable) " + + s"to $targetTotalSize bytes") + + // If the store has 0 spillable bytes left, it has exhausted. + try { + var exhausted = false + var totalSpilled = 0L + while (!exhausted && + currentSpillableSize > targetTotalSize) { + val nextSpillableBuffer = nextSpillable() + if (nextSpillableBuffer != null) { + if (nextSpillableBuffer.addReference()) { + withResource(nextSpillableBuffer) { _ => + val bufferHasSpilled = + catalog.isBufferSpilled( + nextSpillableBuffer.id, + nextSpillableBuffer.storageTier) + val bufferSpill = if (!bufferHasSpilled) { + spillBuffer( + nextSpillableBuffer, this, catalog, stream) + } else { + // if `nextSpillableBuffer` already spilled, we still need to + // remove it from our tier and call free on it, but set + // `newBuffer` to None because there's nothing to register + // as it has already spilled. + BufferSpill(nextSpillableBuffer, None) + } + totalSpilled += bufferSpill.spilledBuffer.memoryUsedBytes + bufferSpills.append(bufferSpill) + catalog.updateTiers(bufferSpill) + } + } + } + } + if (totalSpilled <= 0) { + // we didn't spill in this iteration, exit loop + exhausted = true + logWarning("Unable to spill enough to meet request. " + + s"Total=${currentSize} " + + s"Spillable=${currentSpillableSize} " + + s"Target=$targetTotalSize") + } + totalSpilled + } finally { + if (bufferSpills.nonEmpty) { + // This is a hack in order to completely synchronize with the GPU before we free + // a buffer. It is necessary because of non-synchronous cuDF calls that could fall + // behind where the CPU is. Freeing a rapids buffer in these cases needs to wait for + // all launched GPU work, otherwise crashes or data corruption could occur. + // A more performant implementation would be to synchronize on the thread that read + // the buffer via events. + // https://github.com/NVIDIA/spark-rapids/issues/8610 + Cuda.deviceSynchronize() + bufferSpills.foreach(_.spilledBuffer.safeFree()) + } + } + } + } else { + 0L // nothing spilled + } + } + + /** + * Given a specific `RapidsBuffer` spill it to `spillStore` + * + * @return a `BufferSpill` instance with the target buffer in this store, and an optional + * new `RapidsBuffer` in the target spill store if this rapids buffer hadn't already + * spilled. + * @note called with catalog lock held + */ + private def spillBuffer( + buffer: RapidsBuffer, + store: RapidsBufferStore, + catalog: RapidsBufferCatalog, + stream: Cuda.Stream): BufferSpill = { + // copy the buffer to spillStore + var maybeNewBuffer: Option[RapidsBuffer] = None + var lastTier: Option[StorageTier] = None + var nextSpillStore = store.spillStore + while (maybeNewBuffer.isEmpty && nextSpillStore != null) { + lastTier = Some(nextSpillStore.tier) + // copy buffer if it fits + maybeNewBuffer = nextSpillStore.copyBuffer(buffer, catalog, stream) + + // if it didn't fit, we can try a lower tier that has more space + if (maybeNewBuffer.isEmpty) { + nextSpillStore = nextSpillStore.spillStore + } + } + if (maybeNewBuffer.isEmpty) { + throw new IllegalStateException( + s"Unable to spill buffer ${buffer.id} of size ${buffer.memoryUsedBytes} " + + s"to tier ${lastTier}") + } + // return the buffer to free and the new buffer to register + BufferSpill(buffer, maybeNewBuffer) + } + + /** + * Tries to make room for `buffer` in the host store by spilling. + * + * @param buffer buffer that will be copied to the host store if it fits + * @param stream CUDA stream to synchronize for memory operations + * @return true if the buffer fits after a potential spill + */ + protected def trySpillToMaximumSize( + buffer: RapidsBuffer, + catalog: RapidsBufferCatalog, + stream: Cuda.Stream): Boolean = { + true // default to success, HostMemoryStore overrides this + } + /** Base class for all buffers in this store. */ abstract class RapidsBufferBase( override val id: RapidsBufferId, @@ -411,7 +549,7 @@ abstract class RapidsBufferStore(val tier: StorageTier) freeBuffer() } } else { - logWarning(s"Trying to free an invalid buffer => $id, size = ${getMemoryUsedBytes}, $this") + logWarning(s"Trying to free an invalid buffer => $id, size = ${memoryUsedBytes}, $this") } } @@ -453,7 +591,7 @@ abstract class RapidsBufferStore(val tier: StorageTier) releaseResources() } - override def toString: String = s"$name buffer size=${getMemoryUsedBytes}" + override def toString: String = s"$name buffer size=${memoryUsedBytes}" } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index 9fe3793cffe..f5c60dedd5b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -1995,6 +1995,16 @@ object RapidsConf { "The chunked pack bounce buffer must be at least 1MB in size") .createWithDefault(128L * 1024 * 1024) + val SPILL_TO_DISK_BOUNCE_BUFFER_SIZE = + conf("spark.rapids.memory.host.spillToDiskBounceBufferSize") + .doc("Amount of host memory (in bytes) to set aside at startup for the " + + "bounce buffer used for gpu to disk spill that bypasses the host store.") + .internal() + .bytesConf(ByteUnit.BYTE) + .checkValue(v => v >= 1, + "The gpu to disk spill bounce buffer must have a positive size") + .createWithDefault(128L * 1024 * 1024) + val SPLIT_UNTIL_SIZE_OVERRIDE = conf("spark.rapids.sql.test.overrides.splitUntilSize") .doc("Only for tests: override the value of GpuDeviceManager.splitUntilSize") .internal() @@ -2679,6 +2689,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val chunkedPackBounceBufferSize: Long = get(CHUNKED_PACK_BOUNCE_BUFFER_SIZE) + lazy val spillToDiskBounceBufferSize: Long = get(SPILL_TO_DISK_BOUNCE_BUFFER_SIZE) + lazy val splitUntilSizeOverride: Option[Long] = get(SPLIT_UNTIL_SIZE_OVERRIDE) private val optimizerDefaults = Map( diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStore.scala index 7b0f07bf876..bc09752bfdf 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStore.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStore.scala @@ -16,16 +16,18 @@ package com.nvidia.spark.rapids +import java.nio.channels.WritableByteChannel import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable import ai.rapids.cudf.{ColumnVector, Cuda, DeviceMemoryBuffer, HostMemoryBuffer, MemoryBuffer, Table} import com.nvidia.spark.rapids.Arm._ +import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableSeq import com.nvidia.spark.rapids.StorageTier.StorageTier import com.nvidia.spark.rapids.format.TableMeta -import org.apache.spark.sql.rapids.TempSpillBufferId +import org.apache.spark.sql.rapids.storage.RapidsStorageUtils import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -35,7 +37,9 @@ import org.apache.spark.sql.vectorized.ColumnarBatch * during spill in chunked_pack. The parameter defaults to 128MB, * with a rule-of-thumb of 1MB per SM. */ -class RapidsDeviceMemoryStore(chunkedPackBounceBufferSize: Long = 128L*1024*1024) +class RapidsDeviceMemoryStore( + chunkedPackBounceBufferSize: Long = 128L*1024*1024, + hostBounceBufferSize: Long = 128L*1024*1024) extends RapidsBufferStore(StorageTier.DEVICE) { // The RapidsDeviceMemoryStore handles spillability via ref counting @@ -45,9 +49,13 @@ class RapidsDeviceMemoryStore(chunkedPackBounceBufferSize: Long = 128L*1024*1024 private var chunkedPackBounceBuffer: DeviceMemoryBuffer = DeviceMemoryBuffer.allocate(chunkedPackBounceBufferSize) + private var hostSpillBounceBuffer: HostMemoryBuffer = + HostMemoryBuffer.allocate(hostBounceBufferSize) + override protected def createBuffer( other: RapidsBuffer, - stream: Cuda.Stream): RapidsBufferBase = { + catalog: RapidsBufferCatalog, + stream: Cuda.Stream): Option[RapidsBufferBase] = { val memoryBuffer = withResource(other.getCopyIterator) { copyIterator => copyIterator.next() } @@ -64,12 +72,12 @@ class RapidsDeviceMemoryStore(chunkedPackBounceBufferSize: Long = 128L*1024*1024 case b => throw new IllegalStateException(s"Unrecognized buffer: $b") } } - new RapidsDeviceMemoryBuffer( + Some(new RapidsDeviceMemoryBuffer( other.id, deviceBuffer.getLength, other.meta, deviceBuffer, - other.getSpillPriority) + other.getSpillPriority)) } } @@ -127,7 +135,7 @@ class RapidsDeviceMemoryStore(chunkedPackBounceBufferSize: Long = 128L*1024*1024 * @return the RapidsBuffer instance that was added. */ def addTable( - id: TempSpillBufferId, + id: RapidsBufferId, table: Table, initialSpillPriority: Long, needsSync: Boolean): RapidsBuffer = { @@ -209,13 +217,14 @@ class RapidsDeviceMemoryStore(chunkedPackBounceBufferSize: Long = 128L*1024*1024 * @param spillPriority a starting spill priority */ class RapidsTable( - id: TempSpillBufferId, + id: RapidsBufferId, table: Table, spillPriority: Long) extends RapidsBufferBase( id, null, - spillPriority) { + spillPriority) + with RapidsBufferChannelWritable { /** The storage tier for this buffer */ override val storageTier: StorageTier = StorageTier.DEVICE @@ -256,7 +265,7 @@ class RapidsDeviceMemoryStore(chunkedPackBounceBufferSize: Long = 128L*1024*1024 chunkedPacker.getMeta } - override def getMemoryUsedBytes: Long = unpackedSizeInBytes + override val memoryUsedBytes: Long = unpackedSizeInBytes override def getPackedSizeBytes: Long = getChunkedPacker.getTotalContiguousSize @@ -371,6 +380,32 @@ class RapidsDeviceMemoryStore(chunkedPackBounceBufferSize: Long = 128L*1024*1024 } } } + + override def writeToChannel(outputChannel: WritableByteChannel, stream: Cuda.Stream): Long = { + var written: Long = 0L + withResource(getCopyIterator) { copyIter => + while(copyIter.hasNext) { + withResource(copyIter.next()) { slice => + val iter = + new MemoryBufferToHostByteBufferIterator( + slice, + hostSpillBounceBuffer, + stream) + iter.foreach { bb => + try { + while (bb.hasRemaining) { + written += outputChannel.write(bb) + } + } finally { + RapidsStorageUtils.dispose(bb) + } + } + } + } + written + } + } + } class RapidsDeviceMemoryBuffer( @@ -380,9 +415,10 @@ class RapidsDeviceMemoryStore(chunkedPackBounceBufferSize: Long = 128L*1024*1024 contigBuffer: DeviceMemoryBuffer, spillPriority: Long) extends RapidsBufferBase(id, meta, spillPriority) - with MemoryBuffer.EventHandler { + with MemoryBuffer.EventHandler + with RapidsBufferChannelWritable { - override def getMemoryUsedBytes(): Long = size + override val memoryUsedBytes: Long = size override val storageTier: StorageTier = StorageTier.DEVICE @@ -456,10 +492,32 @@ class RapidsDeviceMemoryStore(chunkedPackBounceBufferSize: Long = 128L*1024*1024 } super.free() } + + override def writeToChannel(outputChannel: WritableByteChannel, stream: Cuda.Stream): Long = { + var written: Long = 0L + val iter = new MemoryBufferToHostByteBufferIterator( + contigBuffer, + hostSpillBounceBuffer, + stream) + iter.foreach { bb => + try { + while (bb.hasRemaining) { + written += outputChannel.write(bb) + } + } finally { + RapidsStorageUtils.dispose(bb) + } + } + written + } } override def close(): Unit = { - super.close() - chunkedPackBounceBuffer.close() - chunkedPackBounceBuffer = null + try { + super.close() + } finally { + Seq(chunkedPackBounceBuffer, hostSpillBounceBuffer).safeClose() + chunkedPackBounceBuffer = null + hostSpillBounceBuffer = null + } } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDiskStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDiskStore.scala index ffb8960ccb1..d61f6061116 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDiskStore.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDiskStore.scala @@ -37,7 +37,8 @@ class RapidsDiskStore(diskBlockManager: RapidsDiskBlockManager) override protected def createBuffer( incoming: RapidsBuffer, - stream: Cuda.Stream): RapidsBufferBase = { + catalog: RapidsBufferCatalog, + stream: Cuda.Stream): Option[RapidsBufferBase] = { // assuming that the disk store gets contiguous buffers val id = incoming.id val path = if (id.canShareDiskPaths) { @@ -49,14 +50,14 @@ class RapidsDiskStore(diskBlockManager: RapidsDiskBlockManager) val (fileOffset, diskLength) = if (id.canShareDiskPaths) { // only one writer at a time for now when using shared files path.synchronized { - writeToFile(incoming, path, append = true) + writeToFile(incoming, path, append = true, stream) } } else { - writeToFile(incoming, path, append = false) + writeToFile(incoming, path, append = false, stream) } logDebug(s"Spilled to $path $fileOffset:$diskLength") - incoming match { + val buff = incoming match { case _: RapidsHostBatchBuffer => new RapidsDiskColumnarBatch( id, @@ -73,19 +74,26 @@ class RapidsDiskStore(diskBlockManager: RapidsDiskBlockManager) incoming.meta, incoming.getSpillPriority) } + Some(buff) } /** Copy a host buffer to a file, returning the file offset at which the data was written. */ private def writeToFile( incoming: RapidsBuffer, path: File, - append: Boolean): (Long, Long) = { + append: Boolean, + stream: Cuda.Stream): (Long, Long) = { incoming match { case fileWritable: RapidsBufferChannelWritable => withResource(new FileOutputStream(path, append)) { fos => withResource(fos.getChannel) { outputChannel => val startOffset = outputChannel.position() - val writtenBytes = fileWritable.writeToChannel(outputChannel) + val writtenBytes = fileWritable.writeToChannel(outputChannel, stream) + if (writtenBytes == 0) { + throw new IllegalStateException( + s"Buffer ${fileWritable} wrote 0 bytes disk on spill. This is not supported!" + ) + } (startOffset, writtenBytes) } } @@ -109,16 +117,17 @@ class RapidsDiskStore(diskBlockManager: RapidsDiskBlockManager) id, meta, spillPriority) { private[this] var hostBuffer: Option[HostMemoryBuffer] = None - override def getMemoryUsedBytes(): Long = size + override val memoryUsedBytes: Long = size override val storageTier: StorageTier = StorageTier.DISK override def getMemoryBuffer: MemoryBuffer = synchronized { if (hostBuffer.isEmpty) { + require(size > 0, + s"$this attempted an invalid 0-byte mmap of a file") val path = id.getDiskPath(diskBlockManager) val mappedBuffer = HostMemoryBuffer.mapFile(path, MapMode.READ_WRITE, fileOffset, size) - logDebug(s"Created mmap buffer for $path $fileOffset:$size") hostBuffer = Some(mappedBuffer) } hostBuffer.foreach(_.incRefCount()) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsGdsStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsGdsStore.scala index 1030d6e4f75..529f3fece63 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsGdsStore.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsGdsStore.scala @@ -39,13 +39,14 @@ class RapidsGdsStore( override protected def createBuffer( other: RapidsBuffer, - stream: Cuda.Stream): RapidsBufferBase = { + catalog: RapidsBufferCatalog, + stream: Cuda.Stream): Option[RapidsBufferBase] = { // assume that we get 1 buffer val otherBuffer = withResource(other.getCopyIterator) { it => it.next() } - withResource(otherBuffer) { _ => + val buff = withResource(otherBuffer) { _ => val deviceBuffer = otherBuffer match { case d: BaseDeviceMemoryBuffer => d case _ => throw new IllegalStateException("copying from buffer without device memory") @@ -56,6 +57,7 @@ class RapidsGdsStore( singleShotSpill(other, deviceBuffer) } } + Some(buff) } override def close(): Unit = { @@ -71,7 +73,7 @@ class RapidsGdsStore( extends RapidsBufferBase(id, meta, spillPriority) { override val storageTier: StorageTier = StorageTier.GDS - override def getMemoryUsedBytes(): Long = size + override val memoryUsedBytes: Long = size override def getMemoryBuffer: MemoryBuffer = getDeviceMemoryBuffer } @@ -231,7 +233,7 @@ class RapidsGdsStore( var isPending: Boolean = true) extends RapidsGdsBuffer(id, size, meta, spillPriority) { - override def getMemoryUsedBytes(): Long = size + override val memoryUsedBytes: Long = size override def materializeMemoryBuffer: MemoryBuffer = this.synchronized { closeOnExcept(DeviceMemoryBuffer.allocate(size)) { buffer => diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsHostMemoryStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsHostMemoryStore.scala index 743711cee7a..dbdbb38f13c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsHostMemoryStore.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsHostMemoryStore.scala @@ -28,6 +28,7 @@ import com.nvidia.spark.rapids.SpillPriorities.{applyPriorityOffset, HOST_MEMORY import com.nvidia.spark.rapids.StorageTier.StorageTier import com.nvidia.spark.rapids.format.TableMeta +import org.apache.spark.sql.rapids.execution.TrampolineUtil import org.apache.spark.sql.rapids.storage.RapidsStorageUtils import org.apache.spark.sql.types.DataType import org.apache.spark.sql.vectorized.ColumnarBatch @@ -41,7 +42,7 @@ class RapidsHostMemoryStore( maxSize: Long) extends RapidsBufferStore(StorageTier.HOST) { - override def spillableOnAdd: Boolean = false + override protected def spillableOnAdd: Boolean = false override def getMaxSize: Option[Long] = Some(maxSize) @@ -97,42 +98,75 @@ class RapidsHostMemoryStore( } } + override protected def trySpillToMaximumSize( + buffer: RapidsBuffer, + catalog: RapidsBufferCatalog, + stream: Cuda.Stream): Boolean = { + // this spillStore has a maximum size requirement (host only). We need to spill from it + // in order to make room for `buffer`. + val targetTotalSize = maxSize - buffer.memoryUsedBytes + if (targetTotalSize <= 0) { + // lets not spill to host when the buffer we are about + // to spill is larger than our limit + false + } else { + val amountSpilled = synchronousSpill(targetTotalSize, catalog, stream) + if (amountSpilled != 0) { + logDebug(s"Spilled $amountSpilled bytes from ${name} to make room for ${buffer.id}") + TrampolineUtil.incTaskMetricsDiskBytesSpilled(amountSpilled) + } + // if after spill we can fit the new buffer, return true + buffer.memoryUsedBytes <= (maxSize - currentSize) + } + } + override protected def createBuffer( other: RapidsBuffer, - stream: Cuda.Stream): RapidsBufferBase = { - withResource(other.getCopyIterator) { otherBufferIterator => - val isChunked = otherBufferIterator.isChunked - val totalCopySize = otherBufferIterator.getTotalCopySize - closeOnExcept(allocateHostBuffer(totalCopySize)) { hostBuffer => - withResource(new NvtxRange("spill to host", NvtxColor.BLUE)) { _ => - var hostOffset = 0L - val start = System.nanoTime() - while (otherBufferIterator.hasNext) { - val otherBuffer = otherBufferIterator.next() - withResource(otherBuffer) { _ => - otherBuffer match { - case devBuffer: DeviceMemoryBuffer => - hostBuffer.copyFromMemoryBufferAsync( - hostOffset, devBuffer, 0, otherBuffer.getLength, stream) - hostOffset += otherBuffer.getLength - case _ => - throw new IllegalStateException("copying from buffer without device memory") + catalog: RapidsBufferCatalog, + stream: Cuda.Stream): Option[RapidsBufferBase] = { + val wouldFit = trySpillToMaximumSize(other, catalog, stream) + // TODO: this is disabled for now since subsequent work will tie this into + // our host allocator apis. + if (false && !wouldFit) { + // skip host + logWarning(s"Buffer ${other} with size ${other.memoryUsedBytes} does not fit " + + s"in the host store, skipping tier.") + None + } else { + withResource(other.getCopyIterator) { otherBufferIterator => + val isChunked = otherBufferIterator.isChunked + val totalCopySize = otherBufferIterator.getTotalCopySize + closeOnExcept(allocateHostBuffer(totalCopySize)) { hostBuffer => + withResource(new NvtxRange("spill to host", NvtxColor.BLUE)) { _ => + var hostOffset = 0L + val start = System.nanoTime() + while (otherBufferIterator.hasNext) { + val otherBuffer = otherBufferIterator.next() + withResource(otherBuffer) { _ => + otherBuffer match { + case devBuffer: DeviceMemoryBuffer => + hostBuffer.copyFromMemoryBufferAsync( + hostOffset, devBuffer, 0, otherBuffer.getLength, stream) + hostOffset += otherBuffer.getLength + case _ => + throw new IllegalStateException("copying from buffer without device memory") + } } } + stream.sync() + val end = System.nanoTime() + val szMB = (totalCopySize.toDouble / 1024.0 / 1024.0).toLong + val bw = (szMB.toDouble / ((end - start).toDouble / 1000000000.0)).toLong + logDebug(s"Spill to host (chunked=$isChunked) " + + s"size=$szMB MiB bandwidth=$bw MiB/sec") } - stream.sync() - val end = System.nanoTime() - val szMB = (totalCopySize.toDouble / 1024.0 / 1024.0).toLong - val bw = (szMB.toDouble / ((end - start).toDouble / 1000000000.0)).toLong - logDebug(s"Spill to host (chunked=$isChunked) " + - s"size=$szMB MiB bandwidth=$bw MiB/sec") + Some(new RapidsHostMemoryBuffer( + other.id, + totalCopySize, + other.meta, + applyPriorityOffset(other.getSpillPriority, HOST_MEMORY_BUFFER_SPILL_OFFSET), + hostBuffer)) } - new RapidsHostMemoryBuffer( - other.id, - totalCopySize, - other.meta, - applyPriorityOffset(other.getSpillPriority, HOST_MEMORY_BUFFER_SPILL_OFFSET), - hostBuffer) } } } @@ -158,7 +192,7 @@ class RapidsHostMemoryStore( } } - override def writeToChannel(outputChannel: WritableByteChannel): Long = { + override def writeToChannel(outputChannel: WritableByteChannel, ignored: Cuda.Stream): Long = { var written: Long = 0L val iter = new HostByteBufferIterator(buffer) iter.foreach { bb => @@ -184,7 +218,7 @@ class RapidsHostMemoryStore( } /** The size of this buffer in bytes. */ - override def getMemoryUsedBytes: Long = size + override val memoryUsedBytes: Long = size // If this require triggers, we are re-adding a `HostMemoryBuffer` outside of // the catalog lock, which should not possible. The event handler is set to null @@ -304,10 +338,6 @@ class RapidsHostMemoryStore( override val storageTier: StorageTier = StorageTier.HOST - // This is the current size in batch form. It is to be used while this - // batch hasn't migrated to another store. - private val hostSizeInByes: Long = RapidsHostColumnVector.getTotalHostMemoryUsed(hostCb) - // By default all columns are NOT spillable since we are not the only owners of // the columns (the caller is holding onto a ColumnarBatch that will be closed // after instantiation, triggering onClosed callbacks) @@ -329,7 +359,9 @@ class RapidsHostMemoryStore( null } - override def getMemoryUsedBytes: Long = hostSizeInByes + // This is the current size in batch form. It is to be used while this + // batch hasn't migrated to another store. + override val memoryUsedBytes: Long = RapidsHostColumnVector.getTotalHostMemoryUsed(hostCb) /** * Mark a column as spillable @@ -376,7 +408,7 @@ class RapidsHostMemoryStore( "RapidsHostColumnarBatch does not support getCopyIterator") } - override def writeToChannel(outputChannel: WritableByteChannel): Long = { + override def writeToChannel(outputChannel: WritableByteChannel, ignored: Cuda.Stream): Long = { withResource(Channels.newOutputStream(outputChannel)) { outputStream => withResource(new DataOutputStream(outputStream)) { dos => val columns = RapidsHostColumnVector.extractBases(hostCb) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SpillableColumnarBatch.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SpillableColumnarBatch.scala index beb5db35cbd..82efa7699ef 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SpillableColumnarBatch.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SpillableColumnarBatch.scala @@ -97,7 +97,7 @@ class SpillableColumnarBatchImpl ( } override lazy val sizeInBytes: Long = - withRapidsBuffer(_.getMemoryUsedBytes) + withRapidsBuffer(_.memoryUsedBytes) /** * Set a new spill priority. @@ -164,7 +164,7 @@ class SpillableHostColumnarBatchImpl ( } override lazy val sizeInBytes: Long = { - withRapidsHostBatchBuffer(_.getMemoryUsedBytes) + withRapidsHostBatchBuffer(_.memoryUsedBytes) } /** diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala index 43399cb4825..e9d9d7f2d65 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIterator.scala @@ -361,7 +361,7 @@ class RapidsShuffleIterator( try { sb = catalog.acquireBuffer(handle) cb = sb.getColumnarBatch(sparkTypes) - metricsUpdater.update(blockedTime, 1, sb.getMemoryUsedBytes, cb.numRows()) + metricsUpdater.update(blockedTime, 1, sb.memoryUsedBytes, cb.numRows()) } finally { nvtxRangeAfterGettingBatch.close() range.close() diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsBufferCatalogSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsBufferCatalogSuite.scala index 54827e12878..3666b85458e 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsBufferCatalogSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsBufferCatalogSuite.scala @@ -243,7 +243,7 @@ class RapidsBufferCatalogSuite extends AnyFunSuite with MockitoSugar { assertResult(unspilled)(unspilledSame) } // verify that we invoked the copy function exactly once - verify(deviceStore, times(1)).copyBuffer(any(), any()) + verify(deviceStore, times(1)).copyBuffer(any(), any(), any()) unspilled } val unspilledSame = catalog.unspillBufferToDeviceStore( @@ -253,7 +253,7 @@ class RapidsBufferCatalogSuite extends AnyFunSuite with MockitoSugar { assertResult(unspilled)(unspilledSame) } // verify that we invoked the copy function exactly once - verify(deviceStore, times(1)).copyBuffer(any(), any()) + verify(deviceStore, times(1)).copyBuffer(any(), any(), any()) } } } @@ -330,7 +330,7 @@ class RapidsBufferCatalogSuite extends AnyFunSuite with MockitoSugar { var _acquireAttempts: Int = acquireAttempts var currentPriority: Long = initialPriority override val id: RapidsBufferId = bufferId - override def getMemoryUsedBytes: Long = 0 + override val memoryUsedBytes: Long = 0 override def meta: TableMeta = tableMeta override val storageTier: StorageTier = tier override def getColumnarBatch(sparkTypes: Array[DataType]): ColumnarBatch = null diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStoreSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStoreSuite.scala index 1e4bde7a65d..45d96be4cb6 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStoreSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDeviceMemoryStoreSuite.scala @@ -466,9 +466,11 @@ class RapidsDeviceMemoryStoreSuite extends AnyFunSuite with MockitoSugar { override protected def createBuffer( b: RapidsBuffer, - s: Cuda.Stream): RapidsBufferBase = { + c: RapidsBufferCatalog, + s: Cuda.Stream): Option[RapidsBufferBase] = { spilledBuffers += b.id - new MockRapidsBuffer(b.id, b.getPackedSizeBytes, b.meta, b.getSpillPriority) + Some(new MockRapidsBuffer( + b.id, b.getPackedSizeBytes, b.meta, b.getSpillPriority)) } class MockRapidsBuffer(id: RapidsBufferId, size: Long, meta: TableMeta, spillPriority: Long) @@ -481,7 +483,7 @@ class RapidsDeviceMemoryStoreSuite extends AnyFunSuite with MockitoSugar { throw new UnsupportedOperationException /** The size of this buffer in bytes. */ - override def getMemoryUsedBytes: Long = size + override val memoryUsedBytes: Long = size } } } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDiskStoreSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDiskStoreSuite.scala index e8878e2cc6c..6adcbcc1909 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDiskStoreSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDiskStoreSuite.scala @@ -19,7 +19,7 @@ package com.nvidia.spark.rapids import java.io.File import java.math.RoundingMode -import ai.rapids.cudf.{ContiguousTable, DeviceMemoryBuffer, HostMemoryBuffer, Table} +import ai.rapids.cudf.{ColumnVector, ContiguousTable, Cuda, DeviceMemoryBuffer, HostMemoryBuffer, Table} import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import org.mockito.ArgumentMatchers import org.mockito.Mockito.{spy, times, verify} @@ -31,16 +31,31 @@ import org.apache.spark.sql.types.{DataType, DecimalType, DoubleType, IntegerTyp class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { private def buildContiguousTable(): ContiguousTable = { - withResource(new Table.TestBuilder() + withResource(buildTable()) { table => + table.contiguousSplit()(0) + } + } + + private def buildTable(): Table = { + new Table.TestBuilder() .column(5, null.asInstanceOf[java.lang.Integer], 3, 1) .column("five", "two", null, null) .column(5.0, 2.0, 3.0, 1.0) .decimal64Column(-5, RoundingMode.UNNECESSARY, 0, null, -1.4, 10.123) - .build()) { table => - table.contiguousSplit()(0) + .build() + } + + private def buildEmptyTable(): Table = { + withResource(buildTable()) { tbl => + withResource(ColumnVector.fromBooleans(false, false, false, false)) { mask => + tbl.filter(mask) // filter all out + } } } + private val mockTableDataTypes: Array[DataType] = + Array(IntegerType, StringType, DoubleType, DecimalType(10, 5)) + test("spill updates catalog") { val bufferId = MockRapidsBufferId(7, canShareDiskPaths = false) val spillPriority = -7 @@ -54,7 +69,7 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { assertResult(0)(diskStore.currentSize) hostStore.setSpillStore(diskStore) val (bufferSize, handle) = - addTableToCatalog(catalog, bufferId, spillPriority) + addContiguousTableToCatalog(catalog, bufferId, spillPriority) val path = handle.id.getDiskPath(null) assert(!path.exists()) catalog.synchronousSpill(devStore, 0) @@ -68,7 +83,7 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { ArgumentMatchers.eq(handle.id), ArgumentMatchers.eq(StorageTier.DEVICE)) withResource(catalog.acquireBuffer(handle)) { buffer => assertResult(StorageTier.DISK)(buffer.storageTier) - assertResult(bufferSize)(buffer.getMemoryUsedBytes) + assertResult(bufferSize)(buffer.memoryUsedBytes) assertResult(handle.id)(buffer.id) assertResult(spillPriority)(buffer.getSpillPriority) } @@ -93,7 +108,7 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { diskStore => hostStore.setSpillStore(diskStore) - val (_, handle) = addTableToCatalog(catalog, bufferId, spillPriority) + val (_, handle) = addContiguousTableToCatalog(catalog, bufferId, spillPriority) assert(!handle.id.getDiskPath(null).exists()) val expectedTable = withResource(catalog.acquireBuffer(handle)) { buffer => assertResult(StorageTier.DEVICE)(buffer.storageTier) @@ -134,7 +149,7 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { devStore.setSpillStore(hostStore) withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { diskStore => hostStore.setSpillStore(diskStore) - val (_, handle) = addTableToCatalog(catalog, bufferId, spillPriority) + val (_, handle) = addContiguousTableToCatalog(catalog, bufferId, spillPriority) assert(!handle.id.getDiskPath(null).exists()) val expectedBuffer = withResource(catalog.acquireBuffer(handle)) { buffer => assertResult(StorageTier.DEVICE)(buffer.storageTier) @@ -162,6 +177,161 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { } } + test("skip host: spill device memory buffer to disk") { + val bufferId = MockRapidsBufferId(1, canShareDiskPaths = false) + val bufferPath = bufferId.getDiskPath(null) + assert(!bufferPath.exists) + val spillPriority = -7 + withResource(new RapidsDeviceMemoryStore) { devStore => + val catalog = new RapidsBufferCatalog(devStore) + withResource(new AlwaysFailingRapidsHostMemoryStore) { + hostStore => + devStore.setSpillStore(hostStore) + withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { diskStore => + hostStore.setSpillStore(diskStore) + val (_, handle) = addContiguousTableToCatalog(catalog, bufferId, spillPriority) + assert(!handle.id.getDiskPath(null).exists()) + val expectedBuffer = withResource(catalog.acquireBuffer(handle)) { buffer => + assertResult(StorageTier.DEVICE)(buffer.storageTier) + withResource(buffer.getMemoryBuffer) { devbuf => + closeOnExcept(HostMemoryBuffer.allocate(devbuf.getLength)) { hostbuf => + hostbuf.copyFromDeviceBuffer(devbuf.asInstanceOf[DeviceMemoryBuffer]) + hostbuf + } + } + } + withResource(expectedBuffer) { expectedBuffer => + catalog.synchronousSpill(devStore, 0) + withResource(catalog.acquireBuffer(handle)) { buffer => + assertResult(StorageTier.DISK)(buffer.storageTier) + withResource(buffer.getMemoryBuffer) { actualBuffer => + assert(actualBuffer.isInstanceOf[HostMemoryBuffer]) + val actualHostBuffer = actualBuffer.asInstanceOf[HostMemoryBuffer] + assertResult(expectedBuffer.asByteBuffer)(actualHostBuffer.asByteBuffer) + } + } + } + } + } + } + } + + test("skip host: spill table to disk") { + val bufferId = MockRapidsBufferId(1, canShareDiskPaths = false) + val bufferPath = bufferId.getDiskPath(null) + assert(!bufferPath.exists) + val spillPriority = -7 + withResource(new RapidsDeviceMemoryStore) { devStore => + val catalog = new RapidsBufferCatalog(devStore) + withResource(new AlwaysFailingRapidsHostMemoryStore) { + hostStore => + devStore.setSpillStore(hostStore) + withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { diskStore => + hostStore.setSpillStore(diskStore) + val handle = addTableToCatalog(catalog, bufferId, spillPriority) + withResource(buildTable()) { expectedTable => + withResource( + GpuColumnVector.from(expectedTable, mockTableDataTypes)) { expectedBatch => + catalog.synchronousSpill(devStore, 0) + withResource(catalog.acquireBuffer(handle)) { buffer => + assert(handle.id.getDiskPath(null).exists()) + assertResult(StorageTier.DISK)(buffer.storageTier) + withResource(buffer.getColumnarBatch(mockTableDataTypes)) { fromDiskBatch => + TestUtils.compareBatches(expectedBatch, fromDiskBatch) + } + } + } + } + } + } + } + } + + test("skip host: spill table to disk with small host bounce buffer") { + val bufferId = MockRapidsBufferId(1, canShareDiskPaths = false) + val bufferPath = bufferId.getDiskPath(null) + assert(!bufferPath.exists) + val spillPriority = -7 + withResource(new RapidsDeviceMemoryStore(1L*1024*1024, 10)) { devStore => + val catalog = new RapidsBufferCatalog(devStore) + withResource(new AlwaysFailingRapidsHostMemoryStore) { + hostStore => + devStore.setSpillStore(hostStore) + withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { diskStore => + hostStore.setSpillStore(diskStore) + val handle = addTableToCatalog(catalog, bufferId, spillPriority) + withResource(buildTable()) { expectedTable => + withResource( + GpuColumnVector.from(expectedTable, mockTableDataTypes)) { expectedBatch => + catalog.synchronousSpill(devStore, 0) + withResource(catalog.acquireBuffer(handle)) { buffer => + assert(handle.id.getDiskPath(null).exists()) + assertResult(StorageTier.DISK)(buffer.storageTier) + withResource(buffer.getColumnarBatch(mockTableDataTypes)) { fromDiskBatch => + TestUtils.compareBatches(expectedBatch, fromDiskBatch) + } + } + } + } + } + } + } + } + + + test("0-byte table is never spillable as we would fail to mmap") { + val bufferId = MockRapidsBufferId(1, canShareDiskPaths = false) + val bufferPath = bufferId.getDiskPath(null) + val bufferId2 = MockRapidsBufferId(2, canShareDiskPaths = false) + assert(!bufferPath.exists) + val spillPriority = -7 + val hostStoreMaxSize = 1L * 1024 * 1024 + withResource(new RapidsDeviceMemoryStore) { devStore => + val catalog = new RapidsBufferCatalog(devStore) + withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + devStore.setSpillStore(hostStore) + withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { diskStore => + hostStore.setSpillStore(diskStore) + val handle = addZeroRowsTableToCatalog(catalog, bufferId, spillPriority - 1) + val handle2 = addTableToCatalog(catalog, bufferId2, spillPriority) + withResource(handle2) { _ => + assert(!handle.id.getDiskPath(null).exists()) + withResource(buildTable()) { expectedTable => + withResource(buildEmptyTable()) { expectedEmptyTable => + withResource( + GpuColumnVector.from( + expectedTable, mockTableDataTypes)) { expectedCb => + withResource( + GpuColumnVector.from( + expectedEmptyTable, mockTableDataTypes)) { expectedEmptyCb => + catalog.synchronousSpill(devStore, 0) + catalog.synchronousSpill(hostStore, 0) + withResource(catalog.acquireBuffer(handle2)) { buffer => + withResource(catalog.acquireBuffer(handle)) { emptyBuffer => + // the 0-byte table never moved from device. It is not spillable + assertResult(StorageTier.DEVICE)(emptyBuffer.storageTier) + withResource(emptyBuffer.getColumnarBatch(mockTableDataTypes)) { cb => + TestUtils.compareBatches(expectedEmptyCb, cb) + } + // the second table (with rows) did spill + assertResult(StorageTier.DISK)(buffer.storageTier) + withResource(buffer.getColumnarBatch(mockTableDataTypes)) { cb => + TestUtils.compareBatches(expectedCb, cb) + } + } + } + assertResult(0)(devStore.currentSize) + assertResult(0)(hostStore.currentSize) + } + } + } + } + } + } + } + } + } + test("exclusive spill files are deleted when buffer deleted") { testBufferFileDeletion(canShareDiskPaths = false) } @@ -170,6 +340,15 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { testBufferFileDeletion(canShareDiskPaths = true) } + class AlwaysFailingRapidsHostMemoryStore extends RapidsHostMemoryStore(0L){ + override def createBuffer( + other: RapidsBuffer, + catalog: RapidsBufferCatalog, + stream: Cuda.Stream): Option[RapidsBufferBase] = { + None + } + } + private def testBufferFileDeletion(canShareDiskPaths: Boolean): Unit = { val bufferId = MockRapidsBufferId(1, canShareDiskPaths) val bufferPath = bufferId.getDiskPath(null) @@ -183,7 +362,7 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { devStore.setSpillStore(hostStore) withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { diskStore => hostStore.setSpillStore(diskStore) - val (_, handle) = addTableToCatalog(catalog, bufferId, spillPriority) + val (_, handle) = addContiguousTableToCatalog(catalog, bufferId, spillPriority) val bufferPath = handle.id.getDiskPath(null) assert(!bufferPath.exists()) catalog.synchronousSpill(devStore, 0) @@ -200,7 +379,7 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { } } - private def addTableToCatalog( + private def addContiguousTableToCatalog( catalog: RapidsBufferCatalog, bufferId: RapidsBufferId, spillPriority: Long): (Long, RapidsBufferHandle) = { @@ -216,6 +395,31 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { } } + private def addTableToCatalog( + catalog: RapidsBufferCatalog, + bufferId: RapidsBufferId, + spillPriority: Long): RapidsBufferHandle = { + // store takes ownership of the table + catalog.addTable( + bufferId, + buildTable(), + spillPriority, + false) + } + + private def addZeroRowsTableToCatalog( + catalog: RapidsBufferCatalog, + bufferId: RapidsBufferId, + spillPriority: Long): RapidsBufferHandle = { + val table = buildEmptyTable() + // store takes ownership of the table + catalog.addTable( + bufferId, + table, + spillPriority, + false) + } + case class MockRapidsBufferId( tableId: Int, override val canShareDiskPaths: Boolean) extends RapidsBufferId { diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsGdsStoreSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsGdsStoreSuite.scala index 275d1d99fb2..94fa440fd72 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsGdsStoreSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsGdsStoreSuite.scala @@ -92,7 +92,7 @@ class RapidsGdsStoreSuite extends FunSuiteWithTempDir with MockitoSugar { withResource(catalog.acquireBuffer(handle)) { buffer => assertResult(StorageTier.GDS)(buffer.storageTier) assertResult(id)(buffer.id) - assertResult(size)(buffer.getMemoryUsedBytes) + assertResult(size)(buffer.memoryUsedBytes) assertResult(spillPriority)(buffer.getSpillPriority) } } @@ -126,7 +126,7 @@ class RapidsGdsStoreSuite extends FunSuiteWithTempDir with MockitoSugar { ArgumentMatchers.eq(bufferId), ArgumentMatchers.eq(StorageTier.DEVICE)) withResource(catalog.acquireBuffer(handle)) { buffer => assertResult(StorageTier.GDS)(buffer.storageTier) - assertResult(bufferSize)(buffer.getMemoryUsedBytes) + assertResult(bufferSize)(buffer.memoryUsedBytes) assertResult(bufferId)(buffer.id) assertResult(spillPriority)(buffer.getSpillPriority) } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsHostMemoryStoreSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsHostMemoryStoreSuite.scala index 1354cc32bba..2d028f0cf7b 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsHostMemoryStoreSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsHostMemoryStoreSuite.scala @@ -116,7 +116,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { ArgumentMatchers.eq(handle.id), ArgumentMatchers.eq(StorageTier.DEVICE)) withResource(catalog.acquireBuffer(handle)) { buffer => assertResult(StorageTier.HOST)(buffer.storageTier) - assertResult(bufferSize)(buffer.getMemoryUsedBytes) + assertResult(bufferSize)(buffer.memoryUsedBytes) assertResult(handle.id)(buffer.id) assertResult(spillPriority)(buffer.getSpillPriority) } @@ -556,7 +556,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { override def getDiskPath(diskBlockManager: RapidsDiskBlockManager): File = null }) when(mockStore.getMaxSize).thenAnswer(_ => None) - when(mockStore.copyBuffer(any(), any())).thenReturn(mockBuff) + when(mockStore.copyBuffer(any(), any(), any())).thenReturn(Some(mockBuff)) when(mockStore.tier) thenReturn (StorageTier.DISK) withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => devStore.setSpillStore(hostStore) @@ -580,7 +580,9 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { } // close the bigTable so it can be spilled bigTable = null catalog.synchronousSpill(devStore, 0) - verify(mockStore, never()).copyBuffer(ArgumentMatchers.any[RapidsBuffer], + verify(mockStore, never()).copyBuffer( + ArgumentMatchers.any[RapidsBuffer], + ArgumentMatchers.any[RapidsBufferCatalog], ArgumentMatchers.any[Cuda.Stream]) withResource(catalog.acquireBuffer(bigHandle)) { buffer => assertResult(StorageTier.HOST)(buffer.storageTier) @@ -598,7 +600,9 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { catalog.synchronousSpill(devStore, 0) val rapidsBufferCaptor: ArgumentCaptor[RapidsBuffer] = ArgumentCaptor.forClass(classOf[RapidsBuffer]) - verify(mockStore).copyBuffer(rapidsBufferCaptor.capture(), + verify(mockStore).copyBuffer( + rapidsBufferCaptor.capture(), + ArgumentMatchers.any[RapidsBufferCatalog], ArgumentMatchers.any[Cuda.Stream]) assertResult(bigHandle.id)(rapidsBufferCaptor.getValue.id) } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIteratorSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIteratorSuite.scala index afc17081afb..4e0325f9048 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIteratorSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleIteratorSuite.scala @@ -227,7 +227,7 @@ class RapidsShuffleIteratorSuite extends RapidsShuffleTestHelper { assert(cl.hasNext) assertResult(cb)(cl.next()) assertResult(1)(testMetricsUpdater.totalRemoteBlocksFetched) - assertResult(mockBuffer.getMemoryUsedBytes)(testMetricsUpdater.totalRemoteBytesRead) + assertResult(mockBuffer.memoryUsedBytes)(testMetricsUpdater.totalRemoteBytesRead) assertResult(10)(testMetricsUpdater.totalRowsFetched) } finally { RmmSpark.taskDone(taskId) diff --git a/tests/src/test/scala/org/apache/spark/sql/rapids/SpillableColumnarBatchSuite.scala b/tests/src/test/scala/org/apache/spark/sql/rapids/SpillableColumnarBatchSuite.scala index a5209e9bd0e..c4a531a8d7d 100644 --- a/tests/src/test/scala/org/apache/spark/sql/rapids/SpillableColumnarBatchSuite.scala +++ b/tests/src/test/scala/org/apache/spark/sql/rapids/SpillableColumnarBatchSuite.scala @@ -47,7 +47,7 @@ class SpillableColumnarBatchSuite extends AnyFunSuite { } class MockBuffer(override val id: RapidsBufferId) extends RapidsBuffer { - override def getMemoryUsedBytes: Long = 123 + override val memoryUsedBytes: Long = 123 override def meta: TableMeta = null override val storageTier: StorageTier = StorageTier.DEVICE override def getMemoryBuffer: MemoryBuffer = null From 293852920c97cc4074e99defccc2b3038fddae59 Mon Sep 17 00:00:00 2001 From: liyuan <84758614+nvliyuan@users.noreply.github.com> Date: Mon, 18 Sep 2023 10:00:34 +0800 Subject: [PATCH 05/46] [DOC] update the turning guide document issues [skip ci] (#9053) * update docs Signed-off-by: liyuan * fix type Signed-off-by: liyuan * Update docs/tuning-guide.md Co-authored-by: Jason Lowe * add the description about the default value Signed-off-by: liyuan * add link Signed-off-by: liyuan * refine the word Signed-off-by: liyuan --------- Signed-off-by: liyuan Co-authored-by: Jason Lowe --- docs/get-started/best-practices.md | 4 ++-- docs/tuning-guide.md | 21 ++++++++++++++------- 2 files changed, 16 insertions(+), 9 deletions(-) diff --git a/docs/get-started/best-practices.md b/docs/get-started/best-practices.md index 8d534944ced..7ae131d28ec 100644 --- a/docs/get-started/best-practices.md +++ b/docs/get-started/best-practices.md @@ -105,9 +105,9 @@ spark.rapids.sql.batchSizeBytes spark.rapids.sql.reader.batchSizeBytes ``` -### 6. File an issue or ask a question on the Github repo +### 6. File an issue or ask a question on the GitHub repo If you are still getting an OOM exception, please get the Spark eventlog and stack trace from the executor (the whole executor log ideally) and send to spark-rapids-support@nvidia.com , or file a -Github issue on [spark-rapids GitHub repo](https://github.com/NVIDIA/spark-rapids/issues) if it is +GitHub issue on [spark-rapids GitHub repo](https://github.com/NVIDIA/spark-rapids/issues) if it is not sensitive. Or, open a [discussion thread](https://github.com/NVIDIA/spark-rapids/discussions). \ No newline at end of file diff --git a/docs/tuning-guide.md b/docs/tuning-guide.md index 4b1f239d33d..19935364124 100644 --- a/docs/tuning-guide.md +++ b/docs/tuning-guide.md @@ -52,13 +52,14 @@ Default value: `true` Configuration key: [`spark.rapids.memory.gpu.allocFraction`](configs.md#memory.gpu.allocFraction) -Default value: `0.9` +Default value: `1.0` Allocating memory on a GPU can be an expensive operation. RAPIDS uses a pooling allocator called [RMM](https://github.com/rapidsai/rmm) to mitigate this overhead. By default, on startup -the plugin will allocate `90%` (`0.9`) of the _available_ memory on the GPU and keep it as a pool -that can be allocated from. If the pool is exhausted more memory will be allocated and added to -the pool. +the plugin will allocate almost `100%` (`1.0`) of the _available_ memory on the GPU and keep it as a pool +that can be allocated from. We reserve 640 MiB by default for system use such as memory needed for kernels +and kernel launches. If the pool is exhausted more memory will be allocated and added to the pool. + Most of the time this is a huge win, but if you need to share the GPU with other [libraries](additional-functionality/ml-integration.md) that are not aware of RMM this can lead to memory issues, and you may need to disable pooling. @@ -101,9 +102,13 @@ the spill storage, so it can use pinned memory too. Configuration key: [`spark.rapids.memory.host.spillStorageSize`](configs.md#memory.host.spillStorageSize) -Default value: `1g` +Default value: `-1` This is the amount of host memory that is used to cache spilled data before it is flushed to disk. +The default value is '-1', it is the combined size of [pinned](configs.md#memory.pinnedPool.size) +and [pageable](additional-functionality/advanced_configs.md#memory.host.pageablePool.size) memory pools. +If there is no spilling, the default value for the spill storage is fine. But it is recommended to +use a few gigabytes pinned memory in both spilling and no-spilling cases. The GPU Accelerator employs different algorithms that allow it to process more data than can fit in the GPU's memory. We do not support this for all operations, and are constantly trying to add more. The way that this can work is by spilling parts of the data to host memory or to disk, and then @@ -205,7 +210,9 @@ provides work for all GPUs. GPUs process data much more efficiently when they have a large amount of data to process in parallel. Loading data from fewer, large input files will perform better than loading data from many small input files. Ideally input files should be on the order of a few gigabytes -rather than megabytes or smaller. +rather than megabytes or smaller. The `spark.sql.files.openCostInBytes` config can be tuned to +a larger value than the default (4 MB) to reduce the number of tasks in a data scan stage +to improve performance if there are many small files in a table. Note that the GPU can encode Parquet and ORC data much faster than the CPU, so the costs of writing large files can be significantly lower. @@ -267,7 +274,7 @@ for details. ## Columnar Batch Size Configuration key: [`spark.rapids.sql.batchSizeBytes`](configs.md#sql.batchSizeBytes) -Default value: `2147483647` (just under 2 GiB) +Default value: `1073741824` (just under 1 GiB) The RAPIDS Accelerator plugin processes data on the GPU in a columnar format. Data is processed in a series of columnar batches. During processing multiple batches may be concatenated From aedda6d10cefbfd3163f611bfda40457cd86da24 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Sun, 17 Sep 2023 21:46:13 -0500 Subject: [PATCH 06/46] Fix flaky ORC filecache test (#9240) Signed-off-by: Jason Lowe --- .../filecache/FileCacheIntegrationSuite.scala | 20 ++++++------------- 1 file changed, 6 insertions(+), 14 deletions(-) diff --git a/tests/src/test/scala/org/apache/spark/sql/rapids/filecache/FileCacheIntegrationSuite.scala b/tests/src/test/scala/org/apache/spark/sql/rapids/filecache/FileCacheIntegrationSuite.scala index fa23e2f5056..931e6d95628 100644 --- a/tests/src/test/scala/org/apache/spark/sql/rapids/filecache/FileCacheIntegrationSuite.scala +++ b/tests/src/test/scala/org/apache/spark/sql/rapids/filecache/FileCacheIntegrationSuite.scala @@ -187,29 +187,21 @@ class FileCacheIntegrationSuite extends SparkQueryCompareTestSuite with BeforeAn } private def checkMetricsFullHit(func: () => Option[SparkPlan]): Unit = { - var expectedFooterMisses = 0L - var expectedDataRangeMisses = 0L var metrics: Map[String, SQLMetric] = func().get.metrics var attempts = 0 - while (attempts < 10 && metrics(FILECACHE_DATA_RANGE_MISSES).value > expectedDataRangeMisses) { - expectedDataRangeMisses = metrics(FILECACHE_DATA_RANGE_MISSES).value - expectedFooterMisses = metrics(FILECACHE_FOOTER_MISSES).value - Thread.sleep(100) + while (attempts < 10 && metrics(FILECACHE_DATA_RANGE_MISSES).value > 0) { + Thread.sleep(1000) metrics = func().get.metrics attempts += 1 } assert(metrics(FILECACHE_FOOTER_HITS).value > 0) assert(metrics(FILECACHE_FOOTER_HITS_SIZE).value > 0) - assertResult(expectedFooterMisses)(metrics(FILECACHE_FOOTER_MISSES).value) - if (expectedFooterMisses == 0) { - assertResult(0)(metrics(FILECACHE_FOOTER_MISSES_SIZE).value) - } + assertResult(0)(metrics(FILECACHE_FOOTER_MISSES).value) + assertResult(0)(metrics(FILECACHE_FOOTER_MISSES_SIZE).value) assert(metrics(FILECACHE_DATA_RANGE_HITS).value > 0) assert(metrics(FILECACHE_DATA_RANGE_HITS_SIZE).value > 0) - assertResult(expectedDataRangeMisses)(metrics(FILECACHE_DATA_RANGE_MISSES).value) - if (expectedDataRangeMisses == 0) { - assertResult(0)(metrics(FILECACHE_DATA_RANGE_MISSES_SIZE).value) - } + assertResult(0)(metrics(FILECACHE_DATA_RANGE_MISSES).value) + assertResult(0)(metrics(FILECACHE_DATA_RANGE_MISSES_SIZE).value) assert(metrics.contains(FILECACHE_FOOTER_READ_TIME)) assert(metrics.contains(FILECACHE_DATA_RANGE_READ_TIME)) } From 92f308ce07c8884bd0e4dc7c971131cc0fd27e66 Mon Sep 17 00:00:00 2001 From: Chong Gao Date: Mon, 18 Sep 2023 13:25:23 +0800 Subject: [PATCH 07/46] Add test cases for Parquet statistics [databricks] (#9090) * Add test case for Parquet statistics Signed-off-by: Chong Gao * Add tests for nested types * Update test cases Signed-off-by: Chong Gao * Fix compile error * Add config * Update * Update for timestamp type for Spark 31x; update for String type * Disable tests and mark these case as scale test, will be moved to scale test project/repo * Remove the value range * Enable test cases to test premerge * Revert "Enable test cases to test premerge" This reverts commit f50f42d67abf93e1fae3804fc7e43b68124f9f53. --------- Signed-off-by: Chong Gao Co-authored-by: Chong Gao --- .../spark/rapids/ParquetScaleTestSuite.scala | 538 ++++++++++++++++++ .../rapids/SparkQueryCompareTestSuite.scala | 2 - 2 files changed, 538 insertions(+), 2 deletions(-) create mode 100644 tests/src/test/scala/com/nvidia/spark/rapids/ParquetScaleTestSuite.scala diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScaleTestSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScaleTestSuite.scala new file mode 100644 index 00000000000..987fff1fb31 --- /dev/null +++ b/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScaleTestSuite.scala @@ -0,0 +1,538 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 com.nvidia.spark.rapids + +import java.io.File + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.parquet.format.converter.ParquetMetadataConverter +import org.apache.parquet.hadoop.ParquetFileReader +import org.apache.parquet.schema.PrimitiveType + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.apache.spark.sql.tests.datagen.{DBGen, TableGen} +import org.apache.spark.sql.types._ + +class ParquetScaleTestSuite extends SparkQueryCompareTestSuite with Logging { + private val sparkConf = new SparkConf() + // for date time + .set("spark.sql.legacy.parquet.datetimeRebaseModeInWrite", "CORRECTED") // for Spark 31x + .set("spark.sql.parquet.datetimeRebaseModeInWrite", "CORRECTED") // for Spark 32x, 33x and ... + .set("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "CORRECTED") // for Spark 31x + .set("spark.sql.parquet.datetimeRebaseModeInRead", "CORRECTED") // for Spark 32x, 33x and ... + + // for int96 + .set("spark.sql.legacy.parquet.int96RebaseModeInWrite", "CORRECTED") // for Spark 31x + .set("spark.sql.parquet.int96RebaseModeInWrite", "CORRECTED") // for Spark 32x, 33x and ... + .set("spark.sql.legacy.parquet.int96RebaseModeInRead", "CORRECTED") // for Spark 31x + .set("spark.sql.parquet.int96RebaseModeInRead", "CORRECTED") // for Spark 32x, 33x and ... + .set("spark.rapids.sql.explain", "ALL") + + /** + * By default cuDF splits row groups by 1,000,000 rows, we expect one row group. + * Refer to cuDF parquet.hpp + * default_row_group_size_bytes = 128 * 1024 * 1024; ///< 128MB per row group + * default_row_group_size_rows = 1000000; ///< 1 million rows per row group + */ + private val rowsNum: Long = 1000L * 1000L + + case class ParquetStat( + schema: Seq[String], + rowGroupStats: Seq[RowGroupStat]) + + case class RowGroupStat(rowCount: Long, columnStats: Seq[ColumnChunkStat]) + + case class ColumnChunkStat( + primitiveType: PrimitiveType, + min: Comparable[_], + max: Comparable[_], + hasNonNullValue: Boolean, + isNumNullsSet: Boolean, + numNulls: Long + ) + + def writeScaleTestDataOnCpu(testDataPath: File, gen: SparkSession => DataFrame): Unit = { + withCpuSparkSession( + spark => { + // define table + val path = testDataPath.getAbsolutePath + // write to a file on CPU + gen(spark).coalesce(1).write.mode("overwrite").parquet(path) + }, + sparkConf) + } + + // write test data on CPU or GPU, then read the stats + def getStats(filePath: File): SparkSession => ParquetStat = { spark => + withTempPath { writePath => + // Read from the testing Parquet file and then write to a Parquet file + spark.read.parquet(filePath.getAbsolutePath).coalesce(1) + .write.mode("overwrite").parquet(writePath.getAbsolutePath) + + // get Stats + getStatsFromFile(writePath) + } + } + + /** + * Find a parquet file in parquetDir and get the stats. It's similar to output of + * `Parquet-cli meta file`. Parquet-cli:https://github + * .com/apache/parquet-mr/tree/master/parquet-cli + * + * @param parquetDir parquet file directory + * @return Parquet statistics + */ + @scala.annotation.nowarn("msg=method readFooter in class ParquetFileReader is deprecated") + private def getStatsFromFile(parquetDir: File): ParquetStat = { + val parquetFile = parquetDir.listFiles(f => f.getName.endsWith(".parquet"))(0) + val p = new Path(parquetFile.getCanonicalPath) + val footer = ParquetFileReader.readFooter(new Configuration(), p, + ParquetMetadataConverter.NO_FILTER) + val columnTypes = footer.getFileMetaData.getSchema.getColumns.asScala.toArray + .map(c => c.toString) + val groupStats = footer.getBlocks.asScala.toArray.map { g => + val rowCount = g.getRowCount + val columnChunkStats = g.getColumns.asScala.toArray.map { col => + ColumnChunkStat( + col.getPrimitiveType, + col.getStatistics.genericGetMin(), + col.getStatistics.genericGetMax(), + col.getStatistics.hasNonNullValue, + col.getStatistics.isNumNullsSet, + col.getStatistics.getNumNulls) + } + RowGroupStat(rowCount, columnChunkStats) + } + + ParquetStat(columnTypes, groupStats) + } + + private def checkStats( + genDf: SparkSession => DataFrame, + skipCheckSchema: Boolean = false): (ParquetStat, ParquetStat) = { + withTempPath { testDataFile => + // Write test data to a file on CPU + writeScaleTestDataOnCpu(testDataFile, genDf) + + // write data and get stats on CPU + val cpuStats = withCpuSparkSession(getStats(testDataFile), sparkConf) + val cpuFileSize = testDataFile.listFiles(f => f.getName.endsWith(".parquet"))(0).length() + + // write data and get stats on GPU + val gpuStats = withGpuSparkSession(getStats(testDataFile), sparkConf) + val gpuFileSize = testDataFile.listFiles(f => f.getName.endsWith(".parquet"))(0).length() + + // compare schema + if (!skipCheckSchema) { + assertResult(cpuStats.schema)(gpuStats.schema) + } + + // compare stats + assert(cpuStats.rowGroupStats.length == gpuStats.rowGroupStats.length) + cpuStats.rowGroupStats.zip(gpuStats.rowGroupStats).foreach { + case (cpuRowGroup, gpuRowGroup) => { + assert(cpuRowGroup.rowCount == gpuRowGroup.rowCount) + assert(cpuRowGroup.columnStats.length == gpuRowGroup.columnStats.length) + cpuRowGroup.columnStats.zip(gpuRowGroup.columnStats).foreach { + case (cpuColumnStat, gpuColumnStat) => { + assert(cpuColumnStat.hasNonNullValue == gpuColumnStat.hasNonNullValue) + if (cpuColumnStat.hasNonNullValue) { + // compare all the attributes + assertResult(cpuColumnStat)(gpuColumnStat) + } else { + // hasNonNullValue is false, which means stats are invalid, no need to compare + // other attributes. + /** + * hasNonNullValue means: + * + * Returns whether there have been non-null values added to this statistics + * + * @return true if the values contained at least one non-null value + * + * Refer to link: https://github.com/apache/parquet-mr/blob/apache-parquet-1.10.1 + * /parquet-column/src/main/java/org/apache/parquet/column/statistics + * /Statistics.java#L504-L506 + * + * e.g.: Spark 31x, for timestamp type + * CPU: hasNonNullValue: false, isNumNullsSet: false, getNumNulls: -1 + * GPU: hasNonNullValue: false, isNumNullsSet: true, getNumNulls: 0 + * + * Above are expected differences. + */ + assertResult(cpuColumnStat.primitiveType)(gpuColumnStat.primitiveType) + } + } + } + } + } + + // Check the Gpu file size is not too large. + assert(gpuFileSize < 2 * cpuFileSize) + + (cpuStats, gpuStats) + } + } + + private val basicTypes = Seq( + // "float", "Blocked by https://github.com/rapidsai/cudf/issues/13948" + // "double", "Blocked by https://github.com/rapidsai/cudf/issues/13948" + "boolean", + "byte", + "short", + "int", + "long", + "decimal", + "string", + "date", + "timestamp") + + test("Statistics tests for Parquet files written by GPU, float/double") { + assume(false, "Blocked by https://github.com/rapidsai/cudf/issues/13948") + assume(false, "Move to scale test") + + val schema = StructType(Seq( + StructField("c01", FloatType), + StructField("c02", DoubleType) + )) + // 2 rows with NaN + val data = Seq(Row(1.1f, Double.NaN), Row(Float.NaN, 2.2d)) + + def genDf(schema: StructType, data: Seq[Row]): SparkSession => DataFrame = spark => + spark.createDataFrame(spark.sparkContext.parallelize(data), schema) + + checkStats(genDf(schema, data)) + + // After https://github.com/rapidsai/cudf/issues/13948 is fixed + // Change this test to the following code: + // val schemaStr = + // """ + // struct< + // c01: float, + // c02: double + // > + // """ + // val gen = DBGen() + // val tab = gen.addTable("tab", schemaStr, rowsNum) + // tab("c01").setNullProbability(0.5) + // checkStats(tab) + } + + test("Statistics tests for Parquet files written by GPU, basic types") { + assume(false, "Move to scale test") + val schema = + """ + struct< + c01: boolean, + c02: byte, + c03: short, + c04: int, + c05: long, + c06: decimal, + c07: string, + c08: date, + c09: timestamp + > + """ + // "float", "Blocked by https://github.com/rapidsai/cudf/issues/13948" + // "double", "Blocked by https://github.com/rapidsai/cudf/issues/13948" + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c01").setNullProbability(0.5) + tab("c06").setNullProbability(0.5) + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + checkStats(genDf(tab)) + } + + test("Statistics tests for Parquet files written by GPU, array") { + assume(false, "Move to scale test") + basicTypes.foreach { t => + val schema = s"struct>" + val nullProbabilities = Seq(0d, 0.5d) + nullProbabilities.foreach { nullProbability => + try { + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c01").setNullProbability(nullProbability) + tab("c01").setLength(5) // avoid row group exceeds 128M, we expect one row group + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + checkStats(genDf(tab)) + } catch { + case e: Exception => + logError(s"check $schema, $nullProbability failed", e) + throw e + } + } + } + } + + test("Statistics tests for Parquet files written by GPU, map") { + assume(false, "Move to scale test") + basicTypes.foreach { t => + val nullProbabilities = Seq(0d, 0.5d) + nullProbabilities.foreach { nullProbability => + val schema = s"struct>" + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c01").setNullProbability(nullProbability) + tab("c01").setLength(3) // avoid row group exceeds 128M, we expect one row group + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + checkStats(genDf(tab)) + } + } + } + + test("Statistics tests for Parquet files written by GPU, struct") { + assume(false, "Move to scale test") + val schema = basicTypes.zipWithIndex.map { case (t, index) => + s"c0$index: $t" + }.mkString("struct<\nc1: struct<", ", \n", ">>") + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c1").setNullProbability(0.5) + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + checkStats(genDf(tab)) + } + + test("Statistics tests for Parquet files written by GPU, struct(array, map, struct)") { + assume(false, "Move to scale test") + val schema = + """ + struct< + c1: struct< + c01: array + >, + c2: struct< + c01: map + >, + c3: struct< + c301: struct< + c30101: int, + c30102: byte + > + > + > + """ + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c1").setNullProbability(0.9) + tab("c2").setNullProbability(0.9) + tab("c3").setNullProbability(0.9) + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + checkStats(genDf(tab)) + } + + test("Statistics tests for Parquet files written by GPU, array(struct, array)") { + assume(false, "Move to scale test") + val schema = + """ + struct< + c1: array< + struct< + c201: long, + c202: int + > + >, + c2: array> + > + """ + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c1").setNullProbability(0.8) + tab("c2").setNullProbability(0.8) + tab("c1").setLength(3) + tab("c2").setLength(3) + tab("c2")("child").setLength(3) + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + checkStats(genDf(tab)) + } + + test("Statistics tests for Parquet files written by GPU, map(array)") { + assume(false, "Move to scale test") + val schema = + """ + struct< + c1: map< + array, + array + > + > + """ + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c1").setNullProbability(0.8) + tab("c1").setLength(3) + tab("c1")("key").setLength(3) + tab("c1")("value").setLength(3) + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + checkStats(genDf(tab)) + } + + test("Statistics tests for Parquet files written by GPU, map(struct)") { + assume(false, "Move to scale test") + val schema = + """ + struct< + c1: map< + struct< + c101: long, + c102: int + >, + struct< + c101: long, + c102: string + > + > + > + """ + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c1").setNullProbability(0.8) + tab("c1").setLength(3) + tab("c1")("key").setLength(3) + tab("c1")("value").setLength(3) + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + checkStats(genDf(tab)) + } + + test("Statistics tests for Parquet files written by GPU, array(map)") { + assume(false, "Move to scale test") + val schema = + """ + struct< + c1: array> + > + """ + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c1").setNullProbability(0.8) + tab("c1")("child").setLength(3) + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + /** + * Note: There are discrepancies between CPU and GPU file schemas, + * but the Spark can read both them correctly, so it's not an issue. + * + * Details: + * + * CPU Parquet file schema is: + * message spark_schema { + * optional group c1 (LIST) { + * repeated group list { + * optional group element (MAP) { + * repeated group key_value { + * required binary key (STRING); + * optional int64 value; + * } + * } + * } + * } + * } + * + * GPU Parquet file schema is: + * message schema { + * optional group c1 (LIST) { + * repeated group list { + * optional group c1 (MAP) { + * repeated group key_value { + * required binary key (STRING); + * optional int64 value; + * } + * } + * } + * } + * } + * + * Spark reads both of them as: + * + * df.printSchema() + * root + * |-- c1: array (nullable = true) + * | |-- element: map (containsNull = true) + * | | |-- key: string + * | | |-- value: long (valueContainsNull = true) + * + */ + // skip check the schema + val (cpuStat, gpuStat) = checkStats(genDf(tab), skipCheckSchema = true) + + val expectedCpuSchemaForSpark31x = Seq( + "[c1, list, element, key_value, key] required binary key (UTF8)", + "[c1, list, element, key_value, value] optional int64 value") + + val expectedCpuSchemaForSpark320Plus = Seq( + "[c1, list, element, key_value, key] required binary key (STRING)", + "[c1, list, element, key_value, value] optional int64 value") + + val expectedGpuSchemaForSpark31x = Seq( + "[c1, list, c1, key_value, key] required binary key (UTF8)", + "[c1, list, c1, key_value, value] optional int64 value") + + val expectedGpuSchemaForSpark320Plus = Seq( + "[c1, list, c1, key_value, key] required binary key (STRING)", + "[c1, list, c1, key_value, value] optional int64 value") + + assert(cpuStat.schema == expectedCpuSchemaForSpark31x || + cpuStat.schema == expectedCpuSchemaForSpark320Plus) + + assert(gpuStat.schema == expectedGpuSchemaForSpark31x || + gpuStat.schema == expectedGpuSchemaForSpark320Plus) + } + + test("Statistics tests for Parquet files written by GPU, map(map)") { + assume(false, "Move to scale test") + val schema = + """ + struct< + c1: map< + map, + map + > + > + """ + val gen = DBGen() + val tab = gen.addTable("tab", schema, rowsNum) + tab("c1").setNullProbability(0.8) + tab("c1").setLength(3) + tab("c1")("key").setLength(3) + tab("c1")("value").setLength(3) + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + checkStats(genDf(tab)) + } +} diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/SparkQueryCompareTestSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/SparkQueryCompareTestSuite.scala index fb8b8e11dcb..798384ae85e 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/SparkQueryCompareTestSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/SparkQueryCompareTestSuite.scala @@ -202,8 +202,6 @@ trait SparkQueryCompareTestSuite extends AnyFunSuite with BeforeAndAfterAll { def withCpuSparkSession[U](f: SparkSession => U, conf: SparkConf = new SparkConf()): U = { val c = conf.clone() .set(RapidsConf.SQL_ENABLED.key, "false") // Just to be sure - // temp work around to unsupported timestamp type - .set("spark.sql.parquet.outputTimestampType", "TIMESTAMP_MICROS") withSparkSession(c, f) } From 318807ed708916c18970c1b0b269ea7d979c399c Mon Sep 17 00:00:00 2001 From: Navin Kumar <97137715+NVnavkumar@users.noreply.github.com> Date: Mon, 18 Sep 2023 13:46:40 -0700 Subject: [PATCH 08/46] Handle escaping the dangling right ] and right } in the regexp transpiler (#9239) * Handle escaping the dangling right ] and right } automatically in the transpiler to ensure compatibility with cudf Signed-off-by: Navin Kumar * Fix a syntax error in pytest that snuck in Signed-off-by: Navin Kumar * fix scalatest failures Signed-off-by: Navin Kumar --------- Signed-off-by: Navin Kumar --- .../src/main/python/regexp_test.py | 24 +++++++++++++++++++ .../com/nvidia/spark/rapids/RegexParser.scala | 6 ++++- .../rapids/RegularExpressionParserSuite.scala | 4 ++-- 3 files changed, 31 insertions(+), 3 deletions(-) diff --git a/integration_tests/src/main/python/regexp_test.py b/integration_tests/src/main/python/regexp_test.py index 3d3ce6ce4af..63a96d0fd37 100644 --- a/integration_tests/src/main/python/regexp_test.py +++ b/integration_tests/src/main/python/regexp_test.py @@ -122,6 +122,25 @@ def test_split_re_no_limit(): 'split(a, "^[o]")'), conf=_regexp_conf) +def test_split_with_dangling_brackets(): + data_gen = mk_str_gen('([bf]o{0,2}[.?+\\^$|{}]{1,2}){1,7}') \ + .with_special_case('boo.and.foo') \ + .with_special_case('boo?and?foo') \ + .with_special_case('boo+and+foo') \ + .with_special_case('boo^and^foo') \ + .with_special_case('boo$and$foo') \ + .with_special_case('boo|and|foo') \ + .with_special_case('boo{and}foo') \ + .with_special_case('boo$|and$|foo') + assert_gpu_and_cpu_are_equal_collect( + lambda spark : unary_op_df(spark, data_gen).selectExpr( + 'split(a, "[a-z]]")', + 'split(a, "[boo]]]")', + 'split(a, "[foo]}")', + 'split(a, "[foo]}}")'), + conf=_regexp_conf) + + def test_split_optimized_no_re(): data_gen = mk_str_gen('([bf]o{0,2}[.?+\\^$|{}]{1,2}){1,7}') \ .with_special_case('boo.and.foo') \ @@ -134,6 +153,11 @@ def test_split_optimized_no_re(): .with_special_case('boo$|and$|foo') assert_gpu_and_cpu_are_equal_collect( lambda spark : unary_op_df(spark, data_gen).selectExpr( + 'split(a, "]")', + 'split(a, "]]")', + 'split(a, "}")', + 'split(a, "}}")', + 'split(a, ",")', 'split(a, "\\\\.")', 'split(a, "\\\\?")', 'split(a, "\\\\+")', diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala index 985c4efda40..acaa90b0f12 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala @@ -147,6 +147,10 @@ class RegexParser(pattern: String) { parseGroup() case '[' => parseCharacterClass() + case ']' => + RegexEscaped(']') + case '}' => + RegexEscaped('}') case '\\' => parseEscapedCharacter() case '\u0000' => @@ -1857,7 +1861,7 @@ sealed case class RegexChar(ch: Char) extends RegexCharacterClassComponent { override def toRegexString: String = ch.toString } -sealed case class RegexEscaped(a: Char) extends RegexCharacterClassComponent{ +sealed case class RegexEscaped(a: Char) extends RegexCharacterClassComponent { def this(a: Char, position: Int) { this(a) this.position = Some(position) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionParserSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionParserSuite.scala index 3c4f91d0816..00465308e6f 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionParserSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionParserSuite.scala @@ -52,7 +52,7 @@ class RegularExpressionParserSuite extends AnyFunSuite { test("not a quantifier") { assert(parse("{1}") === RegexSequence(ListBuffer( - RegexChar('{'), RegexChar('1'),RegexChar('}')))) + RegexChar('{'), RegexChar('1'),RegexEscaped('}')))) } test("nested repetition") { @@ -109,7 +109,7 @@ class RegularExpressionParserSuite extends AnyFunSuite { assert(parse("[a]]") === RegexSequence(ListBuffer( RegexCharacterClass(negated = false, - ListBuffer(RegexChar('a'))), RegexChar(']')))) + ListBuffer(RegexChar('a'))), RegexEscaped(']')))) } test("escaped brackets") { From 0f8dc74430933cd81ed6e66d35e5376a7328e6fa Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Mon, 18 Sep 2023 15:55:04 -0500 Subject: [PATCH 09/46] Enable a couple of retry asserts in internal row to cudf row iterator suite (#9251) * Enable a couple of retry asserts in internal row to cudf row iterator suite Signed-off-by: Alessandro Bellina * Clarify usage of taskId=1 --------- Signed-off-by: Alessandro Bellina --- ...ternalRowToCudfRowIteratorRetrySuite.scala | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GeneratedInternalRowToCudfRowIteratorRetrySuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GeneratedInternalRowToCudfRowIteratorRetrySuite.scala index 2f26d7efa50..ea8ca9bc5cf 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GeneratedInternalRowToCudfRowIteratorRetrySuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GeneratedInternalRowToCudfRowIteratorRetrySuite.scala @@ -41,6 +41,11 @@ class GeneratedInternalRowToCudfRowIteratorRetrySuite } } + private def getAndResetNumRetryThrowCurrentTask: Int = { + // taskId 1 was associated with the current thread in RmmSparkRetrySuiteBase + RmmSpark.getAndResetNumRetryThrow(/*taskId*/ 1) + } + test("a retry when copying to device is handled") { val batch = buildBatch() val ctriter = new ColumnarToRowIterator( @@ -85,16 +90,13 @@ class GeneratedInternalRowToCudfRowIteratorRetrySuite ctriter, schema, TargetSize(Int.MaxValue), NoopMetric, NoopMetric, NoopMetric, NoopMetric, NoopMetric)) RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId, 2) - assertResult(0)(RmmSpark.getAndResetNumRetryThrow(RmmSpark.getCurrentThreadId)) + assertResult(0)(getAndResetNumRetryThrowCurrentTask) withResource(myIter.next()) { devBatch => withResource(buildBatch()) { expected => TestUtils.compareBatches(expected, devBatch) } } - // TODO: enable this assert, for some reason this is returning 0, but I verified - // via the debugger and printfs that we are retrying 2 times total in the first block, - // and 3 times in the second block that I have added retries to. - // assertResult(5)(RmmSpark.getAndResetNumRetryThrow(RmmSpark.getCurrentThreadId)) + assertResult(5)(getAndResetNumRetryThrowCurrentTask) assert(!myIter.hasNext) assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) // This is my wrap around of checking that we did retry the last part @@ -131,16 +133,13 @@ class GeneratedInternalRowToCudfRowIteratorRetrySuite ctriter, schema, TargetSize(Int.MaxValue), NoopMetric, NoopMetric, NoopMetric, NoopMetric, NoopMetric)) RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId, 2) - assertResult(0)(RmmSpark.getAndResetNumRetryThrow(RmmSpark.getCurrentThreadId)) + assertResult(0)(getAndResetNumRetryThrowCurrentTask) withResource(myIter.next()) { devBatch => withResource(buildBatch()) { expected => TestUtils.compareBatches(expected, devBatch) } } - // TODO: enable this assert, for some reason this is returning 0, but I verified - // via the debugger and printfs that we are retrying 2 times total in the first block, - // and 3 times in the second block that I have added retries to. - // assertResult(5)(RmmSpark.getAndResetNumRetryThrow(RmmSpark.getCurrentThreadId)) + assertResult(5)(getAndResetNumRetryThrowCurrentTask) assert(!myIter.hasNext) assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) // This is my wrap around of checking that we did retry the last part From 8dfe2a184f3817c075bfd68a2bb58cb3752b6e54 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Tue, 19 Sep 2023 07:59:14 -0600 Subject: [PATCH 10/46] Add Spark 3.5.0 to list of supported Spark versions [skip ci] (#9252) * Add Spark 3.5.0 to list of supported Spark versions * signoff Signed-off-by: Andy Grove * Update rapids-shuffle documentation --------- Signed-off-by: Andy Grove --- docs/additional-functionality/rapids-shuffle.md | 1 + docs/download.md | 1 + 2 files changed, 2 insertions(+) diff --git a/docs/additional-functionality/rapids-shuffle.md b/docs/additional-functionality/rapids-shuffle.md index 3aa2f7580e3..7c2ef5a3020 100644 --- a/docs/additional-functionality/rapids-shuffle.md +++ b/docs/additional-functionality/rapids-shuffle.md @@ -31,6 +31,7 @@ in our plugin: | 3.3.3 | com.nvidia.spark.rapids.spark333.RapidsShuffleManager | | 3.4.0 | com.nvidia.spark.rapids.spark340.RapidsShuffleManager | | 3.4.1 | com.nvidia.spark.rapids.spark341.RapidsShuffleManager | +| 3.5.0 | com.nvidia.spark.rapids.spark350.RapidsShuffleManager | | Databricks 10.4 | com.nvidia.spark.rapids.spark321db.RapidsShuffleManager | | Databricks 11.3 | com.nvidia.spark.rapids.spark330db.RapidsShuffleManager | | Databricks 12.2 | com.nvidia.spark.rapids.spark332db.RapidsShuffleManager | diff --git a/docs/download.md b/docs/download.md index e2a1e96c09e..c1ed44b8c63 100644 --- a/docs/download.md +++ b/docs/download.md @@ -44,6 +44,7 @@ The plugin is tested on the following architectures: Apache Spark 3.2.0, 3.2.1, 3.2.2, 3.2.3, 3.2.4 Apache Spark 3.3.0, 3.3.1, 3.3.2 Apache Spark 3.4.0, 3.4.1 + Apache Spark 3.5.0 Supported Databricks runtime versions for Azure and AWS: Databricks 10.4 ML LTS (GPU, Scala 2.12, Spark 3.2.1) From d7f58eacc42fe0154ee6eaaf1429ae5d22d21842 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Tue, 19 Sep 2023 10:42:06 -0500 Subject: [PATCH 11/46] Add in unbounded to unbounded optimization for min/max (#9228) Signed-off-by: Robert (Bobby) Evans --- .../src/main/python/window_function_test.py | 36 ++++++++++++++ .../spark/rapids/GpuWindowExpression.scala | 49 +++++++++++++++++++ .../spark/sql/rapids/AggregateFunctions.scala | 10 ++++ 3 files changed, 95 insertions(+) diff --git a/integration_tests/src/main/python/window_function_test.py b/integration_tests/src/main/python/window_function_test.py index 07ee61cd1a3..6c81fcb804e 100644 --- a/integration_tests/src/main/python/window_function_test.py +++ b/integration_tests/src/main/python/window_function_test.py @@ -444,6 +444,42 @@ def test_range_windows_with_string_order_by_column(data_gen, batch_size): ' FROM window_agg_table ', conf={'spark.rapids.sql.batchSizeBytes': batch_size}) +# This is for aggregations that work with the optimized unbounded to unbounded window optimization. +# They don't need to be batched specially, but it only works if all of the aggregations can support this. +# the order returned should be consistent because the data ends up in a single task (no partitioning) +@pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) # set the batch size so we can test multiple stream batches +@pytest.mark.parametrize('b_gen', all_basic_gens + [decimal_gen_32bit, decimal_gen_128bit], ids=meta_idfn('data:')) +def test_window_batched_unbounded_no_part(b_gen, batch_size): + conf = {'spark.rapids.sql.batchSizeBytes': batch_size, + 'spark.rapids.sql.castFloatToDecimal.enabled': True} + query_parts = ['min(b) over (order by a rows between UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) as min_col', + 'max(b) over (order by a rows between UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) as max_col'] + + assert_gpu_and_cpu_are_equal_sql( + lambda spark : two_col_df(spark, UniqueLongGen(), b_gen, length=1024 * 14), + "window_agg_table", + 'select ' + + ', '.join(query_parts) + + ' from window_agg_table ', + validate_execs_in_gpu_plan = ['GpuCachedDoublePassWindowExec'], + conf = conf) + +@pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) # set the batch size so we can test multiple stream batches +@pytest.mark.parametrize('b_gen', all_basic_gens + [decimal_gen_32bit, decimal_gen_128bit], ids=meta_idfn('data:')) +def test_window_batched_unbounded(b_gen, batch_size): + conf = {'spark.rapids.sql.batchSizeBytes': batch_size, + 'spark.rapids.sql.castFloatToDecimal.enabled': True} + query_parts = ['min(b) over (order by a rows between UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) as min_col', + 'max(b) over (partition by a % 2 order by a rows between UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) as max_col'] + + assert_gpu_and_cpu_are_equal_sql( + lambda spark : two_col_df(spark, UniqueLongGen(), b_gen, length=1024 * 14), + "window_agg_table", + 'select ' + + ', '.join(query_parts) + + ' from window_agg_table ', + validate_execs_in_gpu_plan = ['GpuCachedDoublePassWindowExec'], + conf = conf) # This is for aggregations that work with a running window optimization. They don't need to be batched # specially, but it only works if all of the aggregations can support this. diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala index 98e47d93f42..974647d1046 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala @@ -994,6 +994,55 @@ class CountUnboundedToUnboundedFixer(errorOnOverflow: Boolean) } } +class BatchedUnboundedToUnboundedBinaryFixer(val binOp: BinaryOp, val dataType: DataType) + extends BatchedUnboundedToUnboundedWindowFixer { + private var previousResult: Option[Scalar] = None + + override def updateState(scalar: Scalar): Unit = previousResult match { + case None => + previousResult = Some(scalar.incRefCount()) + case Some(prev) => + // This is ugly, but for now it is simple to make it work + val result = withResource(ColumnVector.fromScalar(prev, 1)) { p1 => + withResource(p1.binaryOp(binOp, scalar, prev.getType)) { result1 => + result1.getScalarElement(0) + } + } + closeOnExcept(result) { _ => + previousResult.foreach(_.close) + previousResult = Some(result) + } + } + + override def fixUp(samePartitionMask: Either[ColumnVector, Boolean], + column: ColumnVector): ColumnVector = { + val scalar = previousResult match { + case Some(value) => + value.incRefCount() + case None => + GpuScalar.from(null, dataType) + } + + withResource(scalar) { scalar => + samePartitionMask match { + case scala.Left(cv) => + cv.ifElse(scalar, column) + case scala.Right(true) => + ColumnVector.fromScalar(scalar, column.getRowCount.toInt) + case _ => + column.incRefCount() + } + } + } + + override def close(): Unit = reset() + + override def reset(): Unit = { + previousResult.foreach(_.close()) + previousResult = None + } +} + /** * This class fixes up batched running windows by performing a binary op on the previous value and * those in the the same partition by key group. It does not deal with nulls, so it works for things diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala index bc3a5b3ea9e..78811fef08e 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala @@ -573,6 +573,7 @@ object GpuMin{ abstract class GpuMin(child: Expression) extends GpuAggregateFunction with GpuBatchedRunningWindowWithFixer + with GpuUnboundToUnboundWindowWithFixer with GpuAggregateWindowFunction with GpuRunningWindowFunction with Serializable { @@ -602,6 +603,10 @@ abstract class GpuMin(child: Expression) extends GpuAggregateFunction override def newFixer(): BatchedRunningWindowFixer = new BatchedRunningWindowBinaryFixer(BinaryOp.NULL_MIN, "min") + // UNBOUNDED TO UNBOUNDED WINDOW + override def newUnboundedToUnboundedFixer: BatchedUnboundedToUnboundedWindowFixer = + new BatchedUnboundedToUnboundedBinaryFixer(BinaryOp.NULL_MIN, dataType) + override def groupByScanInputProjection(isRunningBatched: Boolean): Seq[Expression] = inputProjection @@ -752,6 +757,7 @@ object GpuMax { abstract class GpuMax(child: Expression) extends GpuAggregateFunction with GpuBatchedRunningWindowWithFixer + with GpuUnboundToUnboundWindowWithFixer with GpuAggregateWindowFunction with GpuRunningWindowFunction with Serializable { @@ -781,6 +787,10 @@ abstract class GpuMax(child: Expression) extends GpuAggregateFunction override def newFixer(): BatchedRunningWindowFixer = new BatchedRunningWindowBinaryFixer(BinaryOp.NULL_MAX, "max") + // UNBOUNDED TO UNBOUNDED WINDOW + override def newUnboundedToUnboundedFixer: BatchedUnboundedToUnboundedWindowFixer = + new BatchedUnboundedToUnboundedBinaryFixer(BinaryOp.NULL_MAX, dataType) + override def groupByScanInputProjection(isRunningBatched: Boolean): Seq[Expression] = inputProjection From 3f8bb238d64acbd28b9aa3379c30fbe98dec5216 Mon Sep 17 00:00:00 2001 From: liyuan <84758614+nvliyuan@users.noreply.github.com> Date: Wed, 20 Sep 2023 08:58:00 +0800 Subject: [PATCH 12/46] update getting started doc for emr 613 release (#9254) Signed-off-by: liyuan --- docs/get-started/getting-started-aws-emr.md | 7 ++++--- docs/img/AWS-EMR/name-and-applications.png | Bin 120619 -> 121761 bytes 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/docs/get-started/getting-started-aws-emr.md b/docs/get-started/getting-started-aws-emr.md index c227126cc56..4d5dceacab0 100644 --- a/docs/get-started/getting-started-aws-emr.md +++ b/docs/get-started/getting-started-aws-emr.md @@ -13,6 +13,7 @@ Different versions of EMR ship with different versions of Spark, RAPIDS Accelera | EMR | Spark | RAPIDS Accelerator jar | cuDF jar | xgboost4j-spark jar | --- | --- | --- | ---| --- | +| 6.13 | 3.4.1 | rapids-4-spark_2.12-23.06.0-amzn-1.jar | Bundled with rapids-4-spark | xgboost4j-spark_3.0-1.4.2-0.3.0.jar | | 6.12 | 3.4.0 | rapids-4-spark_2.12-23.06.0-amzn-0.jar | Bundled with rapids-4-spark | xgboost4j-spark_3.0-1.4.2-0.3.0.jar | | 6.11 | 3.3.2 | rapids-4-spark_2.12-23.02.0-amzn-0.jar | Bundled with rapids-4-spark | xgboost4j-spark_3.0-1.4.2-0.3.0.jar | | 6.10 | 3.3.1 | rapids-4-spark_2.12-22.12.0.jar | Bundled with rapids-4-spark | xgboost4j-spark_3.0-1.4.2-0.3.0.jar | @@ -88,8 +89,8 @@ top right corner. Click `Create cluster`, which will bring up a detailed cluster Enter a custom "Cluster name" for your cluster. -Select **emr-6.12.0** for the release and pick "Custom" for the "Application bundle". Uncheck all the -software options, and then check **Hadoop 3.3.3**, **Spark 3.4.0**, **Livy 0.7.1** and +Select **emr-6.13.0** for the release and pick "Custom" for the "Application bundle". Uncheck all the +software options, and then check **Hadoop 3.3.3**, **Spark 3.4.1**, **Livy 0.7.1** and **JupyterEnterpriseGateway 2.6.0**. Optionally pick Amazon Linux Release or configure a "Custom AMI". @@ -262,7 +263,7 @@ You will need: ```bash aws emr create-cluster \ ---release-label emr-6.12.0 \ +--release-label emr-6.13.0 \ --applications Name=Hadoop Name=Spark Name=Livy Name=JupyterEnterpriseGateway \ --service-role DemoServiceRole \ --ec2-attributes KeyName=demo-key-pair,SubnetId=demo-subnet,InstanceProfile=DemoInstanceProfile \ diff --git a/docs/img/AWS-EMR/name-and-applications.png b/docs/img/AWS-EMR/name-and-applications.png index 9b28457b3744d56fb665475e891021bc878ffe18..fc9860ef5aa035ac8929fa64e7cb671b44657ba1 100644 GIT binary patch literal 121761 zcmeFZbyQSs7d}j*(jp}d(%oHB0@B?m4xK{`9intdBOMab(jC%6NlOl$(lNjgzwvqB zS9!kwet&-J`&ct;owYe}pMCbd?`vQCIuoX*B9Hat`4a>L1T2MDG8za7$e8f2<|8z? z1-UR)9{~aJt+lkYnu4@6l^W2|!rBgifWR2*_Ni5=%M7# zyY*<;T3QR>Zy!oIK>fs#A_sKW(9<`@_u<%YrB`@t(+RT2f>BVI90p&p z?$xyT-uhGt96K2bnR=An?K_5*M;d4p`KiefSfm6nyk@mei9E%2ZkJNRqjZ{n?lUB` zxLoZu`j%V8N199x2n@aq6HIUzucaT^18mr$x3!wGvE~<{F`;vQw0^Pu_?6WA&_{AR zgf90Uxo)atvYd9`JA`@H&EiDlo&;ehdQhM;5=({Z$)u=m)Zh6^>kb+tnkS!W|Cp+R z1ptu6n#=hcP|rpXhzRIOPeuV^b@d5Rp;!lX0nyTZo`z^X(voGnf`>_KlhFYe@+Wgy zcr})y`LV&j;7%6==qXqzDMt(*G>TnAymGwl#nM zb2E|S8v=qPf`W{cmIvamMO33_!=&g(A>7|c36RgV@dya;d&*~K4jg_R`0Zc93=VBU z@V__kPej?Hdrvfwadh)qhse>ofZsB|g(axnJLL4&(63payBncTpQsQJA4wu0;RYa} z{ND5-5_G#mYW$)sJw`^>Pssc~`+s-Ad`xA=`66iJ-(~~iBZ~hX|G)h(vOjiKVog6d zvm_N5gUT%Q#1RpGFMqBl*-?prgx1Bchx}j|fs7mh0o34jRJe~G46Y055i#bx8P-1o z{qH$)TM&?f8Se=Z{xkePmxmA2E{N~pFhft^hv(iHA|O1pal{9g?;wrX=Eqgks= z&i;zWWjg(p5SpgD*Y271x!_D0B-`O+wMXg1VJfLur$j+qoMCbUkL%mp9%klyOW9qC zWydF&n318zZz=x_VY!}&)4StGts#zLMz8E$jl+aEiWv+$W25Qi$B2)5A*eErkhc7_ zKHv4x#b&6Vew?LhR!QI0fWO6C~MQq2-a zWpTd@Rd>DnE}JUiC~n*ar28p6^uYBWR6BklcGWu~?#9|J)O?O67bzb(?JgYcWby3O zdE2{-Dk_E%TV2f~B-~q|NN~>$yIN>;NdqU8QTYopjups7J@21w@f2g!?2Vk$P6__| z03HEB3dPYdX=)me~zcfMPzl_$p+IS+aAK}@4^aQPJM3v}vM%24Vu=WyQqt?; zLXmGEh@U~N4OWJTf2PA?6Pqp((=Ubdy_tL{}SltN1Mr z6WlBe5nhMMPjL)5!pSaMBNLa89rZ$MHKq3frnTYxHwvn=R4mQH5yJv31)-PFTmki z7>Cm}ZhP*t_8$*rT&^t_P@W6;<)%;HPPG}6?vAyV7#X#^BWd!x4R#>RJN3cb8qL%> zbU!5yuicvt`5D|i_G`8_Dpxjw@GUvVryvivrVd6DBDNGI+3;ud6h~=1$stvGhkHrG zq>O2{I*rjRUufl`r_*AHfIa^G%Yj}ql%lRFyOUVE@AJ~Albc1#44XCBm+I`%`5b2z z1tNq;w_3fobiygbgg**6Ld08(5Tp;0l4B4FgSB-oP3#>LYAIb!KTXS>Hn5xx??alh1TX<}O z7Xhn1HHA}b${5Uv`^QiIxcfV^)9wu_P$4ZiS4i;FY#%2hdY9D8n1jW2R>*B8zi)6A zbGNOHdS|gt?UVDp`H0Pt&kD5in3|_{b1kwN(<)k}dV#?>q%VzDw4S1)52x|vaOKIX zU7zi&zs1&zCYN9@j3E;%K*MNR3XXvZ9hA6944;js+yb{Wz_56Y^F0 zAYguiA#Ou+WLx*5PK~jy;HU*2<1Yy(wf(!i=0*Q_`hM9fnVx8LT^Huv4OwLLv-k-P z(>4v3b$D`59CWurdQUBlHjor!)VMG;9hD~FIN)>b=R8Ah&&=sI{GNm>w@FrqN>&^k=CWTO5S#Cua@N%y?~4t0q&z z_@RDk6n{X#4+gWbfKDmiH#cIHlzi6H8VBC%{an}4Qtb29Aj*9Li3M~qg}4{4#*wz1 zbUoZZOw$cX6W-a9PJYUb6kFanXgpyqnbP$Ir6wkdhJt#)iEq<;6ud5v@bXnM4D@9Y z10^ys^Vug~zfdHuii0k9BJ-i6v#luD_1tcI{*pQ)9^3iqsMZrV_~hArWPYiBb#aO3 zLg=?=NiEmlmAgAdX6>pE+$5`fEr)@)sMds56`s+*9Cn{mICdZ+Ghkb}A*0dJ#sn=x zpVn(Jg#{r4X0Ojl!~6xe*QQC$*US?3-ApSk+w!hmR5VfR->pLjvUJkDWIo3UkN-jw zvYOAgtXcL@%O1BG@C1ev1=ZD$%9D=5#j!SzZ!UbcuiUy|F9RlL~U_(s%-?tvGI-qG6q9@+2qoGxCC|WttHrwJ2WA z={gXo#W|FOR4r&6BJT6W;4?&5iHxiG-w=D%Q@KfV;j!qj&b5Px&UXYPqx)9eMA?Je zgWj$M*RIEjxAJ=zO&m)2#}nKqQ?_ZO(O5H@&Y{#up`b+gQpCxpUMBFrdEMBSZmR)K zdBJb&brW4YEU2KcCX?4maEFW7gKWe0{_z$H*VlKO%Vml=2V^9Fkgg;n5+P|<_F~1j za%i&k;pB-Zq)``5)MX~4)!Y3?qSBV~!Qdz$C3QR$u}yLAn>~+ln+N;Pm;?OCgqGxP z{Nx0)z{3|=!p;fo=_2{j$Z97aH~5U3YHThh3P+143RH$td31-t=3M0_;v9!gB^c75 zmz#GLfo^Wv^yh8CbQNt#8W9WzOgjPF76obFVHC@S>K~KF^OZ|18l8@Fg@?qdW`lH}BC2SgaSX&wcoC&h@Tu&zNs9ODCc)7ifzF(p+`46L&sI z@VF`6IYhL#9YW)}Ovy^k-#_VwO%xb40*kA8EQSg&R*!2%HvzC{$~BpqzCsIr?oODf z)6pj%$h-beQrI-QRghMY>1Rx6NtwqzsS*U~eN)=3|LC?E(HWOc66rkQ-rva3L-jjT zdc2qPW@uEkdu3~TuV8&3uH?3}kp=`EZQccOZExPrUZuWu;=|F}6xmdSxY&QjXSk8J z9>1~rs5T9l^s#k<_#yW|xk?iu_t~X5qbwl9l{Njgw~3gd!k`_(g1472>vNoCB`Y3CRf`}8rCM0KgXSw9!yj6i3yCmY;q?0I#C$S67jy(V&o>I1*1aL zA-Oq=$@s@*GwbGXhW&}Rq)6s_blTGxGR8JH%}QTp9v^5HrRktK$>omv#$9PqXB<#a06+H6nn3d)uWRJBA7ISE>z5F8@ljD9Q&EelKnrqg8@3ZL zF|hQt9$41oLC|#kV#9QY9J=owL^eB}?JQ~Rn|TXb~Z^|@7E`Q2T-oju8qICFPildoci9Orl&^==yX zz@$7|O5pH7YIKg{J&#Usl?sdx6I>I;o%3$ob2w3$Lb-HBu|CT`+kKGz#}3SIGA{u` z8%$BxAz!FB^zwfx7qJz|;(nFC)tk$i8pL*dzp1)#Xk<9c>wL-ySYwxk%0goub16k7 zdP3VtglfW{gG`%%=f#+0362`r6DnBm?yk>Ed`>jL2`b&HNl%nk543I`=~LWyBE~w5 z`@MONE=KlA)a&e(pZPcUWe!K8q!zGWn=g$1pt4=F9Z_H3VLP$i?K7qHAml8aRbS(L zJ|*#{pbg!FN?0N{30a-)QVYy^2A)YDjG@kC?1_1~VYX`d)KtLlSw%bND;9%-1dedn zXYQ^lHvgRYHG!;+<*gL#YvirUgpig$;6@VfhoAHO=u}F=)8VsJ9iKEK8M?lo-==i1 zG@%0onj40x%8R`TK4dRGKo_pKGT(S6Fw`TYoG5quEEKG7bg-MWcod`;40TC;u5*(F zNo$f{{)|k_Gh`n<6_;~6+>GKcGX$SF zYH(xG|G($ou=~I9(hvowR+&zGcp{_n?`o;8!v8+)Wg@dKMja;4pDFu0J9SjBnOy{| z?AGqN?u-MTbGyn^^7xQo=mwI5~Yu=;qfJDBFfhwZgO#8!mfZ7 z>sA`CJvGv&v|WD3o0}r}QSmBPsFCMa$9BZCoe2gvG5Uvd3n8!)L+^#z(FybG-;{%c znAh0WXqGgYwxnc$ojs#*@e6p}nl@gjB#(@SUC^hR+22CMVUpDOy?oKT$R-iIow{wJW;v}2wz|-m^!FsY$GsKK^c6N3QzI03idLjrC*^pruh(>ZX(*3 z)h+3L9DkPH-%p?F7ZM=`aU6rHoIlQUNn~`a0>#9;C(iE{jIHkPg5VqL-ee(K;hW^Z=^Ax+bN09zIz@tlcJo)YD71=K8hmNMo zMZx8iN#MPB*x5j2c@_#p?25}DwB9J5(((Ap>fN6u?wcDBfG~J~lK@yTh#!o?u<%o^d@lbq1#i4Jw`R5}A!N zZKlhK>=q+GR~Ka$>V{^miHmohym>G)JG@Ybu9Zip2XWhID~$5tsc=3GPJ|P;A<=l2 zxb5+)4eHnz!8!R8(MzXt)Ak&Sse$;yqBm<6ziDT$s}q9OWAD83wJ@WDgY{)xW%jk{ zus2=u;GGv?@%_3t4`x`UU*IFhHyo$H zKsT2{lUCSx2o9yyczgU;gQ&S~V~YgW%YJqJ-h8FxL|JN+tC7neZ)n#SUD_~4Hg=TA zh23+$-AC%`F1Go~C+)Afykr;gIKr*9nl+%nhxkIO+(#Qx=fQXo94u^Dj9#}!7~u@8OrO6%>QoEv(ff5 z%s9bUz1;5c{E4Mf5x!SFl+jc3Vh+HIb&6`CV#boxfIMj*w zlV2J|RU8!h69+SvT8?Q+NlBe&#Bvts;Y7yzL1uj13zgrUr#x-|%a5}8`mg38b?d2o zEa$(bcKh~6J)CP-|!=vjdh|i$X z>>KO+;V`ME3IKek5Q0sa^K2k~0^as`2P*!andy3cMq*Iw^-p^RUbhrbsHO`>cT4n+ z&s5w^*7?dCG}#HkS@^gri{ZDdBGV5tpx$T6FK!CE6U~K1TJ`Zp942jFb|x&x!`4Co zbg3-6CAt;)_zbEVaIU+5Y%G@Fe(5E=qeJEI@2}rjrduDqjh%OYbaf!An?lNGCz&UY zg#)GYWQvY}GniqDo*8p|L5cCq`SDg_DXRXr+EV**-M+U%5)i;vxJwt>x%F*uRE_tj zi5u+oXYTPh*;6L%Vy#Z_#7L=eJ^LiEr6v3a92Trw_}|2aHORi{sp_Hm$;_I(KijN| z^0=?2XZ+}m^g7qK`MX>6^(Ov(;eKBx5*F>72>N@0*kAkS^UPNXGvp$mr|3@UPdo|p z4;Gu$X4o^{spYSCvi!}00q0LrbXaOSu4d=%tQKCSGsZ3;LmIc92{@2lfy?lW8to@5 z#fQ_{)wJ&9e|+Y#3}QZ>D()A(y~ar}PF|0^>;AtvkSOKMd!Kr@InzbD^1UT(5QUlZQ{{Bn#>4^GOk(JoQ@%$+~EAE~@P znw&fxfv|I&*H@MJX6XJdCz6C)Enm@}WFB+x!J1ksvn*eBOC-L$J5>y(S_$3GYqy7C z1DzFAg{qndXxp6E=#8J5lO$y6=B7=+pp@%PGLUAh+o7*z#;xy#LAz9jg5k-m(x*hM zyn5-}Bi~g}q_J1A#qOOdHEW8JJMGWN{XM{34amqWaiIb4H3N~ZnDnc@B#3%)J7#P=Sh$y`sf zqtDfrOx465y#2Q+U9JO$4_x%be9WR04{EHa7+YDtv_bg=>A!-+tj0!geP$x z-YndDd>X&_*#m$v*7NxBU;MKS#SJ|QkeVMgq1Niy5Eul|Iq20Yx6?1yEF?V6(=x2FlkJP4h>JnlD8$k(BIc!k-ySci zPyuNvz@wF)7*3Ni$8VTg_BWRsm=a}du#73^wK>zf3e}?cHD6|HIm=e4xmAAhST3DQ z`MKqDao%m>1faC{hSbnPYu_Z$j?wB@NO@KjoOcPqA$={nywL1sNLM1*9t&IuZd!(Y zXyd+A+A9d_+yy7k9D79Jb%lTtltA8pnR>)WdW8W|+Zv~Ns>7<8(p^dW)^}$L68hnH z!;})9FNGGxw%h#~aAT}F`C18Mo^3oct$D;4mQ~YDV3z-MCt6+2&gpRX8tAOuI`J=29W3Z$V{MbHgq3U}~tn%6~))5GG z%`fWd)7q<(tB5nMv-M(J(l-yP{Q!LwR&{g<=C+O*Sy2SW)FM(j4-9ENZ4cbjDchNy z_5sbFiGZh?aHQU)^?_eajepYj9iuLAqEM-@6#Ezq*k%dg*)5@Rxw`p~pQXIX%uury zN+}_Jtm=X9UxEDeLF67X9Sx#m&XhwwK{SSpy$eDQ(L)JXe2{_uqk|#U4+IjNnfMu}Z3nKrsLtth zhBq20nBc*J4KGBl;E;0I%}I&AI{HnKcI(rq48e~}ZQhmx=5qf6%mD$o%VvRl%So>- zZYj`|C>X&Hq!Pc;r<^2A4AWvi?ip7|OzT5jiGYMh2iL*me)&fEF94nM8&NExi<#a$0YYgWxa}_5aYIOWW@aI9je_&daPzCB94dzXJ@&n z7(}=Hsq=OFpSHTU@^6nF7IiY$`qj}EV=14kee)w-r| zs^t+X3QCcupT+7@t9N(b*%38TaF%m--JiguV#fGsxVv6BnxY`!xRkrYW1${&ea>2& z_J|D{1%X2^c>nv$yMef`FqOC$-_reVF2jr@jt;47TwsJ%)zzA(uQT{F*o~WBhvB1d zY!K*H7#H$dr+;UD>4tc*sW9drJ>?fd-`+k=Lf~(9*F=# zigr~O97x{fH!>^Vw3{4-cx4%^`h#=Y#n|B4hl-)l(w&BKf1MQ_Etl z#^-DXH6MiCcNO6sS#a|&5zs*{ol^3k$Wgh&!DI=@^!jKeXyKrL%)A@4=Gmy!%R2eR z8l%biw7>}3{%!4@r%}O)X1$5Do4+@7KN)c14ghOsCzj}x?ES8xewPc%Y4t@L^=A&! z^^Oqr)9th(jmD>Z4wI6%SJgg3d};8~91~NgLcfIF`)2)d)UBncQ2whNc1?Qj``EDE znGz-CG~V3DYke9fI5^Icwh2?6CbRGURu50@XPctnz*X*EIg{Cta(t_f5HYvr`>1>VsK_zqc5>?3EE}vOEg&3@V-vfeAucEWO~$mi@m`%%B;W`x)cIdGXb%ilpq!e0@ zaY-m?+*2D>T64at*|?{?|K!K9iAk&*@lTFF^ZwEo1!*{42IHmKHRfkzhSxW`Nuuwz zorE}=7WzHMY;aR}J&QZwK=-Tnfq>a3quxEiR?jttEm?$|%`p|%-32~Q+q(2QS)kW) zb|{kF6CdOKMBQ|SK3aEK?FNLKDe0HB)!9wSrB<;On)k&Fayu?HWc^7L6DFb z4qNHAg!~H-MUWz*#i){TAV5A&?TWIh^cz|z8b!;o5gR4`gJeRD8l5gka(Q-j+A-s} zgp%ru*_9~<>t3{UW@vV0T0M~804L=r+^ZD$!Yb669dr6=VxcfPl-5oCJI$;^hLkHb!VRSzC+u12|V)djVfBxOt#ak&m{bO zgd80mRWmbDca}T5fF@T2v~rk$xbXr7X?W48wRNr>c?)4wOk_@MVm7F;+T~R}68s1c z#xaRC`u?@mX;g9O+Nwi~LlfvwyiC7YWog5*p06ZHv-3TwYxUPtri}DS-w4T91}?Vp z_ksR{qV>9t(Q%x@!c#Cfagq946kazo)_YtC(pW)uvOXeG@(nOVa<~oNJNtW2s25+5+@+cOYKRv}{+Dn{KBw+cm5(Kk4&b{0M zt-l(NooVra6!i}xpr%Ex(_a$U07gO(s&<~*3q`mVMmukbryeMwESZ+kQZ1Rhbq7=X{_cTgPV41Q@9myflwgVXEl2`_c5rZU3AFxlB_Y@rm_*Hc+!WbakZ`^WNt zJ|F8oN;gL}7T|~J#1jk>#ve`;b{gnlCT4L|plcUj5@AttDWZsL_go2*fQ#8xPj17M zRG2h>1l;*fZ3NS6~m=Ab3QvN zbgR=8Ob4-@i4q2(LKQZ&TGNCzqBB&LIbXfE!Zi#GOsaZSuq7{oVR1$|La`K9XF*2% zKltdz9m&G}jb8hkcoZ9UG-4tS9j#mIrIs+Gwm0TKt}B`6=4U4`#;|regV0Y71a$2Z zd0j7G34`uLYi(vuB@=o6F%w;XVbc+2z0NnZmxDz(-U;i z{A}>hUGkL@69&=4`r1xFvPbbTmygvgM>BK{-yT5xsnA_qJ}0xACSBd$-dgv7wYe!x z(YZ{d*z2q7^PUiZ2ietv&~Oq}yH@4<*iFC1Wr}&TpUPZRJpCNsFHKx?{N%PCO2{Va zZaT6>VEmw^jezo!2p$&562|>KbgeDb6^m~Cy1G}s$b-ohaRgTT!kIb;7c3QEul(G_ zd?oqa=~kkR3r4Cw^mXut$^tZNVvJTQ=4ic;-8%ko4W`woA-n2~DH{BC465k5Z@?GU zjNv@KnC0$mKC$S1hFvG@-_}y&uwQI8_&Vbo9_S#kdyS?EwD)r1{dkQt?QwUJD402@ z0vQI4@6>9)3kvlYOnkaVw>w>??BFmP8WV*~=kU15sF5a$QXr{2+nHqB$;1yl30HeE z$-4_&CD8fZ>@9enGFgt00Wpz43abZ(HH^+OabNml90rPN z^-4)Uq1)j9+AsO>n|~7_Z8g_lCVuWobRtC`RFOtA6{S zYb+TPfE5!qU44^m*bQp*_4NhllGAM{ES0aiAGv!LW}(}= z3YckVE(9@h-Sqtqf054|fOI24EB$GE&Ak)WUTKePZ8gvT zhC2gG>CGP5gyLfb|S!G9ej^BxYmUrF|{KP>*t*nXFU8;UD}53RV5f0u-aJAq0AB)D;D?tiJyOW2-qFazFmops~ znv`jKRdO?tgU1(PIIIl}xp3-vJmvU47BXKCl$dc`&?xkG?>m(6$%cc|dpjb{HKf3frA){B%RHr?}u6lWCuDg=8syLg^GV|TEgQI36 zdJ(q0Q@xc#F;+8+VYNE>IcGDX6w`?Mkt3?((&|RlxJl_N>Tvtj`j;V4(MApbyKw@q z`@1|$vygj7q_X)MHlvaDa4|>53Zd>ffmfIlZ7z&9T{KAo7*x7fW z*!Na^yGQ?sCg?GKrls>>l3X3=M{bYpgdJ)&-7izaIUR0H!}}$DA`Vk!xW2XE%W9vV z&3x5kUzo*JbAE4h=g+1xkqfor1g3tzQ_9||9b*u53E5`4lHJpj@JI8K28O+z-S(lM z7^C`ol$E)j*;69x`Yd8W6m@32E$tuwy*kFqD2 zv;;Hkn9Ls~1lkWV1nj*LS4u#E54D2~$pVMCf%GXvw&K!~MA+$)c>ia0Rw{oH{;FQ% zs@wYIW5onc#0()&tQTmH3N9^gijU3^lx2#e%im38Q5*R+`_gXE7^lt-BC9Ji7-Ty= zq*Y;|tr#mxa(jBn+vIV@?0b8yoiv~0=AxL~``KZkN&1wKgSNcrG{<zEzg)|?bt{P}D8lBVcQhvW-jPO(z&PCl!oD-EF*>r(C>ObHLy@^G zvbD$WkUvua`J@@XUUkw#@WxjT{={=_XBlSX7jpY&AJ^1cF|PelRumJu8)$#S9~L@&tn-U9z2aI!pOzrvP8pt zu))M~3vVnIWOzdd_it~OG(H0Ji<~-ryjtPY6rMlZUm*H|?G1{F7yl5r1%Fy$&Ba4y~IBvPmfmCa%(MXT|pKC^Ldk=IWp-SR{! z5LCr}qpHn6<`aM)_fXr<|BON~GS%na$2maUW$CRjySRkiVwD2&PM$Bvj)*0;m*BS^ zy=TmK!HC<+=1_<&)d%F5DK=_AI5uEgoW{t&IM%2?|Go--Y1 zzUGtNrLR~$5v00?rRkd@x zf`EdXv_xSy#nnACOUuG)KD_!1C(aQ8V>^5MFVeGdJ^5XwR}nqR9}s{}m4)Wwh3niK z&sk89kFlSN2d7R}D1$A5Bj59X^Cc$^bCP$%<4{nz!9oaRb-!wu|2@ERis)U z-3FESu^PU<(qDrPArVis{h0M@ILUDd z#Qgp5aYL>y7De>cHwmKlrs4>$tgE*)B|8M33!FFtM_1@sB$`P5u1-GoDyK5DP~-Pu z#)DR7?B|%PkMDvn4rN;w#<=*)W-Ddcv)bPY(S;Jl6sTTjm1YB-oX@Y7R~2omgWaqq zm;I`)ZY~FSnws;zR&P!K#>k5^K$^SLIav8XV6Lrz!}x;iV3FO|x4e=IQ!RANFRC{= zV&_jhooAib1i@gX@^pt9pDAHrIu9{<(!;xNfj7Jhiu|4HsT`t!Z{=6$FT33Gf9 zkM9<}!1WzVlhw{w-IC`~Ti^Ik z00d~w_i6yAXK>jH0Jm>~45gH4F}^rFz(Fm`ApsuEV+PrfL8TgIr=C<~74faOp72ly z70T^dsBO0tGbl_UiT5!U`A2Lr-@}Xe37xs9S$& zoXer`iZ0O>mGt6yE?LTE4bnV0!C0GnzqhsO2Ku%8CScY|j8*|XOHkYK&ZTB|J z;y6*=Td^#r*S*QxFwqwZ>7jFVc8d46Jns$atyqzp`@Vn1M=!qdTZT@XjWxNb)(8SS z3OSr*8)Vn6C*=}433RaIgudbV^Itu$by(87xb;83cB`bhk+mh$O==I_>yY4(MOa}e zSB^)k4a$fScl4c^{wahy|IMQJC!}B}H-Z{cxjj`YRtt5osMHP*Ld71R4G2QS))4x% z)@`obFsPj*!99+9Y%*qIzQMWL7ol|A0}%!c5uwwQ`7~GzygNHg%4+zQZN$SK0dXV? z0IuGi@VO`?Ctcp=q@A@gxSk$)AGKckODXvHiu>HWs3<#tnL^Gv&+0=5*PuUh^7_J> z1y7<#U;GpKob7O3TJV1J1-Q9=A%nj9GA4@Ub)RbbwD zye29)G4MWUC9v4&@sEOd%aK9utkC-x{C11fL9rOgUFv^lPO*#6H2kYEdMB#<5|OT~-EWWRxvr&bRSuj{} zLC|^KbGuH5_+&*S#AhkJVRd-IXlIPxoFcv6tWEU0_}2z{f3V0efo^hVmEP$AOfNs@ z5*g|-;RFb+fb;VAjjC#K4*nIYFlblbFaeMKYV$XrjK(S2e5;(V+GTNeJHnuqxFL(} zYXE_9)6WgaPBj2vP+#&KLbti+a&o4B>=F$OgK7qN;J5(piW+FsHA}augz^c>tbN#T z?p8Y#q19cUXBj*kTUu3CcvrI&6pn|n z^TvLD4YqH;qZ|VOT88D^!&~4i1_$l+q^_<$m459Tzg6f@74_Inzx!X#$G~m#)U*Et zD1VKMMzlW6=9eRWy|`!iG)o)1kMFo};j}}sip!_&LxnrsBDO3THii038ZfHy z{3En)Cc@D9N-PaRt6Jn=Kbf)p*=hJB#7A|{U5fWC^M~e#CD{TK9BKcEgqK2UjD}Sy zKKIOnNB?=bZWEu=Hu;5viiF!m%MDI3v}5Ij@k#<()_kGXc|#!=W8m5J_+EZE(OKmt zi(YrUChTw>2qZMt47FSL$8{z%A8c+VZ#qwudIAZY&~axDJ5cu_N;)*SAQjbEXe?1R zy=@0ONG;UT6}2STd^BOlZEqmSif>iPPf8Z~d7r&z7N8lh&7Pgs0BOnT1{GnESLfJ! zG}gWL-RcdDjBDr#LEi)t;gl0IZG9=#HG!RPzc?7b3O&$;S*<0J^ZKY!#z!XX*H*ka z9&;cf#F;-VzquP%?wz(i-KAit;F(e~#8zb~vjQJ8E-EKLOZeL_u8+Nax*@(B#v700 zE!D)w+?40@-w&<;P&^~a{x-EF&HWdHfetB8zs+8YWOLd8BX0dm%<4S3>*D9FPm^ABeq4@JN6DI_#to zhNqiS5KSqoDLX@WXU%XeyQ;yuY-OvHWs6g_)#jCGJlDGL;`(QLiY`k~p-dFDx5M4vKr-hDqFwCxr| zYOgP9e&5{vzuy9|cO-iD&dJ{a+v*^+Z8bPrA7m(fdOf0?25lie>;w_-%@nIR=vVhA zA@6%-Zkn5c+NuY9lRWQIJ#LD)Xb))bWm25zOrcCtE9Y-C%_)qi*ZdBAw$ht(dY-b$ z>`dgx7MfdS+(P|U7e2~Dx!M;DI*r-N!;gs<+=2{iBR!pynJM-<4w12=Yneu&%_s0x ztpNTOG&nm#0%L*3jP-cyeRIX}C5c0#Fo=3x)2|_-7yjtqfxi5DYbQeiCCe=Vt5gwe zB}tDGQ<}pjKqI{;wlgHxqUp^Z3TW#Kqv?H^9|gXNc{}}ZS9e8TALU{Yx464;zIVMV zJ?XGm{3cg$F!8DWpq9^F<=2R@M+3oBLc6k$oln<$h;2RmbqYTtZvP8p8H^lJXXXj?{HyPWh_-CG6~lW<|)icGDj6d_W{mQxh>^qU6>5T0ijm4 zjjX4!C#OK+b#3v0NH_Ry{pGO|Ec2)T_MxM5 zsgqx*N<{%hYNzhkaR=gwqSqshu8%nYwLVi6rj-P5@6PL z;#N`i&1H2IO(Io5VYQq7)_#?c_O6-BV_omLpj~8(g+htXLWgn`w1m6O{CId|%YFPA5DeilEyoo)LeZ#*f{Td!6R$n73>i(Ltt-B}?>p zuU(-0cMDYi@W}z5D0bb(GoR`khcw+A6jv0HajMqXAI-HLgBL~f)!MwzZ+cjcv)<5n zCada?fU+9%C3i-$S+|;(jkr`84W7Xlr)Q_el`q2ZKNeY8ogPmFllA;Ww+Hl3+ShIz z@q)OPMdHE!m%meui1cYNJk?~OlK_)n0J7%Xz`T$*>x%Cklp=CY*KG-Kaf!bhFpd6j z2YX3ZT#Gh5?ND`I4czl-_2_8O$lBFo>*`!pcUEw4?VJfF0um8ia+4ylVg ztgPQzh!C!^VebpIJ|(?2suGKVt#)X^Kq}2;->es*C?YNz;)OTRkN0=oc>eOW1911x z6AQcBd)}OJ?px|NV^93k^)PGd7>h52PidLxUI9M|D7T1E>8uYYekjAVhT^;F_8Sk&nPaje1G* zJ=aq>x!K-~Q15r7S)drf#r25L4_Xp+8U)v`;Bp~Zy$L0@{Du05NQY-39olUBH$Noa zj@zi9qW}E?fZrF=(?CEBqH|OK*K1OxMsQ=IsU(^BUmg+xg_Q{|@rY|;B>mT^+0CX4 zU+hcxBqWs?F|=dNf{mTvGVl&Z<_-w5T8S4O^V42OwL4`YCxibqg96o1hn{mfRsN`z zwjK9Fb|{h$_(u^yqqYp!)bP5mK7*^Nt)h+p6TNO2DYZ)D^PodqRy`;TQ647{ViMqJ zbp&Dl55$LyWnRP4R)^Ufdo0Goap6`H!UY@Cz*$*%{W?n$WN|^j(Sb- zThL7^Az?Gqh7agZNE|O1m4QnZYFxJ1gG|V@SoIo06%*N2`D|w}k{o-7^tFCWuyl_) zQ)Zc6C6Hii(7#T90o_UGM<%#nRzb%m)6!WZczAE^=>huf4g1L=A{ofnCmR^rE9 zqjA>9Li+VOe8sxW2LFe>zY5E%S++po;2t12L4&)yyC%51g%Akt?ryh9{QF-8w!sDuC8M4fU7qSBq6PWM4bAEf}$ zMQ#NkPH=x=QUnx;(1FURW~(z5KBs+fYbbg)P)wc#@-c-#J#xCq2SX0FOTw}Tfaq0 zXLX{fl*qYV@80!=Y)Y<_ElvW6Utg2o={#8HQ5_uu4#zV}1h0S+1r-2i?z%W129V{V zm5iaSTJ_NbLm(j8htDtk8var4a%Xr$44}byM|!84C&)tt0#)EcivW%P33qm#nv}vk0^vX0br1!T-cU?59a&)H)6(CZW z?AQUikuWr(yXos*)#gQz0%w$;pPw!vAlF0rR!{z13h3WZ*kBPOM=j+*5ofPF;Zgc0 zw0m9`IO20p0x~rF<6UiWU43GwuUfkWKq-0`xmt8KxhV?oy*PDzPr%k{6{G zcJ)^fj>U5SwYlh8G{qdVlBwg-5|tm+sNP@=rNilr7y%6o#KP6lydBp^4&Ad&859A2 z097iNiUCZR-r_`V^`*T!k(dlxc^VH!F|#%V|2x8INdPE(50HS}O2*@|jk8>Crz^(e%UzJn;cQtx0R5brGy$3Yg*t|!p->%8 zvemuamvonnjLdE@hNJ-)bHRq-y!<{b_`jIs2-XMC~GlOhWhviZ8N& z7i0w@9e`Buc#lz{>;c4lJv&W4P&?uSZ`O$&$B;t)fcfuYHZ2j#`f5+K4ljQkp((e? zfjyu$gumS4q{mBVCU87gH3eRP)jF8M=1Wkt&}8n!m0+-6jZtg3P5q{kfk*j9;_&3? zPOjPYwfd%Sc^+fJYI@?nAp~ldAnHc-#_!eT0$H}hjrsf7J_ ztI3-1#qo!hQjsFrwxFp8MPazvczI1-71{9|^f9-`6z{0PWUtM1yND^*V9MngHsE8v z?E>pRv4cAOstE-=Wx4E(kyZWMpY3jQ%%)6-eCE@V$BcTi=1Cyj`FydUM6t_ zWvTt04gspz9cVf&T`9FTqKWzZmVBwjD|4eigro(|475!z4Io1%=;0kwU^Ffp;V8~w z)nS6xn3CEr?s#xA?jB<74yHG#jGI_% zwl#|?s`(>c3mPCAVp6$lOUY%@r>}Sfv8k^KhkL-KwO<(ag_>Bux+23P{G)m(1&+!e zNE|bY*79$2{~I^iO$^{y@xIML{%1~ln^zUY0J@cO(AbxWQU3H(^EgbdtaQ`bpgTRs8&)vBIH|iaEd{GotvC z-Eg{+T<^5?1{6l=0>`8MKT*XYRzq^V%)fsAFACVx1nd}S2ZE9c_eoCnzhC!_kqL|4 zZ5c@SuL$nnrbID$3nP+cBGCS}gWw#%5A$C})cUto{&fY>&|5*5IT4rQ?-&i>xDx5Z zw&2tQAN0RB7K}p`>_jo4XZwQww;c_zfL2Eqi~nkB0h16V$(wSX_V+Z(|6UKU9O;49 zqpx$|e_yX44p=;{X_`~v?+^3W0b1uqDxvDo(&Tw9T$Uj1{|NjlEZqjMJH*WBz80Px(>G^s8 zJeK01m5X8p;EDmjekRkO-!=3)9zd6zj!P0cn5byxv*iT<&)~Pta+94JK~Jtk)F;E) zIhg;Nz&jr9KlokREinX4dS;&A**y~9whEcPH}R%VE&EvlWGQ-tv0hLj!~RjzWfyyy zW*Irg1ud>ca?{r*Ad7d&Nzet>a{(x&tUN(ou%0ZpP(-b-i$EgoG?!EU7he7J1^gy~ zI~kbhCr7jzzCDC%OlNuW$sCI(JeuCyzQt9xw$kTHj~Wf0ByV(POWlsrgsJ+^G(dH_ zjXMhNAALav7FNrc#$~4pjo|Tk1(X@fr|uE3l&1hbxKiHGpI|K_TsAXtAX-(<*MextL?nZ6>`uV>rp_9f;@f zB6hJjUU14W9m@b+4>j2&5{~xJ(LS-bjq*CUd7!D4YsU}djOB<7(6ioTo*#T&3hnj; z1{aVvjD)S90o8KFPHU%i?sVoB_GB-fK0~l^W7C$VCz8t66Z}xvK zRw$vj5zCzF3rpPhG8&lptyEB?+s3{1I+FQ-e3$y(+!wq~km-Ey+^Ff~aJItM=8b_z z#tp##l$&hD9%!jw(l{$6Gk9@qO27ygUm#%2c82F|bxM<%s2q>xY!{aKD7FVr#M#-| z?EzY)3!ZJj-Jj;qJSl8$vBX38{&60qNIAdpd8!VUe-=MQ4=Bh4b?jD)A3y`3{#;~W+zeA~co!@quP6_mRR185d&+$ig z%lw}tfeI8hw7_;I2IEi&;*mzfw@tUh#t(=!$si)Zp8YiGma0u+!N8DvZ_WluE}ae+ zk^$J#rvesZ>+G*1ZTm~}cE8fJDuw&P`NPmCzTgg5w0OL@yOe$`=gr+vLiAe6n@nQD-y;?Y}a zJviWXvHiXOk;AG8CW}8f?C$7X+}yynOhA*%82WG4Ztw?if$!U2xtH@m z380uzbUM`1UZ9?>SQOKhNQKjQh*AKITdFYO*8VW8M9by(@X*4g)3na7{Dm}fIsR~L zGJfRu3OVU5_PKyc#8wXpWm!xW5-1Ird-(u3_P5yR=xB4v7H<^t8W?C^)azmVO*)4e;r5b2nbMi2=Zzc< zs1|588)pyqK$H}GNkuJ{B)GDm1e{l~oZI685rZ~b@5?1N*q0%0u1mCzD~(o}#a^AhIQ?FA13-!~rsQy@=yNi&32Oqz;s37L?;rtAHxQ*( zXON2NW!9gov_Vs;P*tJdJvChu>NlI@oVq)Y-Kqu@9TXeRQ9aJ?e~SS?*Ky{F?_C)8 zfaK`Vwao_P^7?9>8!`r_5q>R|MC1{mW)a}d{BOcu2lkQszX*Fr$4C1Cdx5=1SOmeR z_-kw8sgwDiNlcr3(@CX148`&kw)~8tXu;ka9sogvD{*rvl&UkC9W|a;gCeuf;fCt_ zbt5#4f2SqSl&umCIJIJKzQ?53H7qSeISj>OAcuhVOi=d}LRyuC z^9&}F1PO&bY)~oSVXG9X2XJHq+XaxClGk}`VmSbK)KcU;Tk{#7??cOqD`0GDG9e*g z9eG3Lz&t4z0nxrdr`DO^ zzu-neqVMoPCZkgb+{q@S#APy?p3kc(rM+vS`JNH^?{pokx++n33#rD9{r}^yi=gm@*Jy>VY2yb2%FazQL?+1~D%S zRtXBcK$V^CkRfrb6bvFWoy}9fLN7q#%oN7aJqQh8(7k`B|*@*Ezh| zPQ+jV4bh$ie8dFrS7gM=B!_^iF&_;lW1nu_FSRSurBIx-s>a>#XYkXqWn%hPa4E6> z>#y&?X8>L9u7?WZ(SPWVV4?vgWYR=O{JS#_-|YH4i(>i@yN12l6`KzJzjj^6a=tji zIY<5g`AG}dc^K<~9TEQh$PPdsJW%g$V{QDexs?cI{td7C2(SP8*V2G;7XX<3sT&I> z2>P#A@fpMfHcInq?;eBx+EjHi0LtI%SnZsgtp5m^>w(7&wu8?J{5SXq_RIk16|v*2 zuPgiCsg8fW3sE)jIMEx7&X>Pk>I?yJle++h_wSMZ{*Iq?XIfB*fFSE+?m!gX*9QP5 z*ld6N1&}DJp5yl%@C9;QU-a7D@A2i16Zr}{I|V17*ZufB9>U#`y`JtA2p;RqXFtCT zBNB3@%#~#w5AcpJ1f42H zsb_5PP0XTPn~)x#$akkv@2ijsawX%gV)U{ z1b^s8*k*_F2kE^wj-M-TYJ$q>qGFJcKdE6Q(a_`QiKwY5gDIu4guvV$WfC!6Pl@~q zz7r823v@IMS2Zo=P0;^#BRp|*S{QYAYHDpb9XYb{QV%7ArVr*=j#w#T` z4_Qv_v6h}KB41}=HeIHPgCUJ6k@(&zADQ;Wk(eF|roe)cs`Pe+sY}6(%NdA{S zkvJ8Baq&8;K?!o}XW2c@9-aFBfq{lc zNeVVYQo@sP(0fvnd<=k);gvx|$+SPD3*#CO*| zj$;c~KI?>_n|*lJK2k0j7-cS{-Y$(<7fo}Iy7`_p5q z#Qy=p77TPMMeGP%cEMz3cCr1bv3#TPwd}n`D{s$*kphCSb}fqz*;6k@9NA4YD$Csw zE&Z{>mE1)Buh@#45~=$FM>n!{myf4jz5@PBp-bdv$2q5+@`+rl!;@a>G-~Dknr!A- zW>e(ND49g(&EqtpRw#VPS05C7|DZuIKVLVjc2NXY+??c2{cbd?p|E$+jRLr}d-nIX zpRn1Iw~IwYd_R=}^|6~66V6EaHfo;Q5k zNvyGV6%_?~*SPrQIA_4HyH~2w{l0osCQMI**>tAk`A>;j&tlK&G%TRFp0TxxVcL5C zYB-%_)BO2Gd;&Y0Q90hr$a(p0|Mo#pyV+i9H*cHa<>t)F7T}k$1p?CUdq(-kOHDGD zkNdlN(>-}J51ucFr?2Z_4DHh9^qTK@@VK2(rlzfCSjGgR;Nz*Y zH6w7pT@`)?kc#&vv>I1sZHG1Oci>j7D{_RHJZ_jhE7<uh`p6 z)Uzld*ubm46>;47AbAe$7Zt^!Yq1Dj)1DEL$lc}(%xQhLW*OWL@IXi>Qm)N)K0hrd zU#v`ZwZEwM3Pv0WlmOO7h^_SRt4|d=Joyl@ub^e!Z_AfkuFucWUbEv}KkXfRJpLXC zjV~EL-&X6S(2U}nVWppEP3z`Vwxv@!ATB`P8PW5eI6$^y$z$|jG|RE2(e1YTp*tPA zU#7LLuRZ`uZ#ke?jrhpc3y^cz4g--6jZV@k>MpKM{6oGhCiE^s-W)B5(P`BmjP0|S zTAHfg+)#=fcf5W%h?9|?&@MVLe?bMPvj~B$Oem{f9q3-K_&lYXzaDHX%Y>6#J>17z z)Z}E$t6LnGJMNTl-tE*(Nx@=OfQnU0n4T{g&Mp}`ay(kF*f{%LE`?xT$*jI{wFyRg|U9M0?wDGm>h5H?mCP`n6|;!J7B1P+vm@ zGp?vM+L{$oPv19fK%ogW4?`>P;bf^!MooIJfwH}WxO|M73)#05P+wKvl6W8IJlMov zspN9}4I7)wjvMyz_V`bb5`|aru~xl4bwqGOV6ObnPlL%c90zO`g_oJ0`E1+{+2ttS zeF#|klP#O$jZ$dgsvRM$3p^#OGttq9u*k@28U1v3$Ge&1&7VKE-jBkT>a@Y<1|d)N zgZz;xjCLcB?<~9Ar;z5kiKD57WBS7hLI7Ts(bL`S_(Kzxe37DL3ubupz*{U_B5-I5+~-IJ>rdNd4-01J&8!}13R_q8=*W#wZ9S85r@>A{BU zcDP;jqKw|rBqlTu-NS%}z;*z^aL5iuUBEi74Y%uaMm05v%3khrx!pHKr)6}VQdd}- z!7u;S>p6ZIC096($w>U__%g*-%`Hc1Ah8y#s}jOgZbT=Yqb%>TsNm;I#VwSG!3UB| z4|XVbMHwi^4L8EI@%5gY@2vE9UZvkAKIwDDc64;`$vI|a-aOvl&nqx44JsrQhitx< zs)y8=j+v2g0SU2YNkDDC>rj;V5lfdd(#J1w&xk>e0e@rn__&O(kXyln?#U4~!?QTx zh0X}~Hj&{KXSvytNo;aD0onF$ZSNuvw7?@WMlN9dS?cIClxBrzI@X5SR1znkLkN`c zT3C^4;Q2W~v$f;})5a2B#8^xlZLZY@zq+`iB`$s_*b!Z=L+tz+$))w)C~{=*9D>wZ zbsvIjV*+O5Fr@%q&gffBVj{-@`!{`tg)9Z5R)H6tbk(Yuk>K65Kwxm{4>G3e{-m(d zt@SErdr@(PYQv4Qthf>Gks$c+JNBi^>0XH(N0{zrrt&x|P zcYNo57p2jKuRVhe5OT^;q&Hk7)Euxe1tlQePe-5^X0$4Da;22Nw%#M;s+enOQ@^Tz zta8PW#f=E)4nQE&1Y^0L9I6+5dE(4&WiH=}4qYx+tt;kh(Vk~^eVr~Ma>GS}AW16( zRVKyc4P~i^1l3jPUl<>zyxYA><9p70A)QgLtnYtSQ{7HPuD6I4o-W^ae(u;4!ER zcQ}{ow}2xfio>?7&sphBBh$EhsrI&*TEJz+)LRiEjZdF#4rj`0tAG`b!ZutkINruRXQTTz>+7*a*v(HH0Hsf z5c)4r@OZvX(Nup$W5{ID{VgSf0S!T6IVq?=h@vyOO7}uQ_vbre+Nxl-Ft<|%Jewk# z4TY?`TmcgSKP{{5>E*y~V2X7ka8!^-+PGT!Lfu)JYm8*W-r{Na^g0CGA1RspF}}xe zRcgKcNqnIGFlkftTE6OikI??7G(})WHprM(r&&9wMK<{&xnD_!567My>0q`Gs?PCp zTCQN<91k7c|Mj`1TSGxg#@pLVwh?hRz`EN)F<_;MFoCuk5URWWC1#426T9$Llg3ND z@9=^nH;En>K6lA3Mo0SIB1(U zt52qV){dC-{w52)z+QXd&r0x7?#$q(MV3^L z@-pYdB9l8=sv4z$af%XDd<|S`W08@OB0EzqF_o{BC+1VQe-sO=G`*#_Sg90}v;WE} z(GGqeKj09NF;IEo$wA+RXl57u9jps$bJr~*oPNLCgV8{?T5pj>D!(+^Heuf ztYUt2-kB@Wmg}{5Z?xL3qs0f8Vq#{@39pkazSosC9x)W3)c#-#U5E}-2*)8ZE$fN; z5ll}{?>Wv6I6#7XsoUB`lCENC8+Jc?9j%F=;$}u8EmJT-`yo1%vhSeSGKW_%{CGzj z+LO}ZaJru);^V%@y)BV`lPSKfyhxv&om7vh^4;=`;39GjoB`8qLqX8tF6`55v|d0O zomMk;6q}V#B(-i@e@#@%yYzfAf}Ke1mDDS+yX>=ugBGh@ztBCjC=RtS4tGCIhl9h` zo$o3rBsIIkQ&pF|JHFsb1@8WU;%~Oq>TK=A$SN`XD}%%Sq{bNPB%IkE=pl8T)56Y5 z3)FktvJuMRb!$dsO@Leh(KALam+?!jVS|?z{?%*b;Q|2zcjw{tr_t+VcU3`3kWp`C z$hDTyy}%ahs<@m?H(0OmKAkOpY8wlgLZevpyqzqH-&vrtg4ks`6G0YD%;4*){N>|e z;VK4wt;M;0i8Gjoc#E*Rd4O>fsqSdF9F<@{Ux#`okGXQ}{mk=gvdWEcID={Xy_S@& z13%sNBNjYW1G<$wiB%CCV=6n;#+ZO7SkGIbaD$QqvFuDC7EVbqIE-edd=e?@u`MDz zyqD`bGrrvI(7`}p@OfyPMjC9SKs8Hx&T$)Za;jj=+W=cRfuOe0@R<5p=yulCGw=`x zIgO1k@3;kM;v)->LbKk{tFBY!81LEf z5I=X>_@EO4blRj{+L;w*-(WhATH0akTsvul_P~k9pVbw)4P`m1S)wlkXvxvQOci3m zuF-dbamd0nWJ!=mx{7eV*M#1cS=)^DdB808;a$8_J5N_J@=~jf2m{P#zpnF!h~{OS z>z2e0!KGEcmsJyJ@?i^G`1O#vrgmF- zry3u+th!8#mTB9-vHUu1w)XwGly?}GvW*z_Pd5>7n}u)~NAj%<^qVwqt=ey>rk|xW zKfrNP=Sr2*_ni`hQKyai&xS+$u>3A^}SgN;}|u3RWD6ol9` zlxR;(1nKXMf&wlq2n!1a{Gp8|c?}I=v?KP9R^Sgw0HKb5owe+HsbG1^1A=r`Csqj zFPK6E+$W`Rm|61w+=v%=Flkik^oM^w*UT5VkJ$z@h$!Ii5AzqiczZCfrCGhIy?yOe zRg1Kf=3ES??KJk&!{TWI`aH(p_pzn`?%p`{$NI9}VzD0I;kvUtzn!ZwwK5Eq#`*i@d|(6x5d&e~@ztd>1Br+_B&mK9$iF@dMMMx0x(VZi z+6Op&C`ap7O`+c1WNTClf{o=H@`A=ZqQK_dk6wgr5H6S-l&lcPV z)8@e&WHPy*7jNcToHF@d*Rn;v&9>)`8a+qxXZQvO!={h90^$i`zFj-yh-C6rAGCR?TRuZUt;d0it(9~;jtUv|v_ z;y~xuHn_n!DiT22*0O@f>IFQUfaU#23j4%01C-vsHistP)N9SA-WQCeO*m-rKcn}{ zIV097b)&dH3<0n`5hmFBLDI~f5AyOXNtDaY01O4 z902vx)6)@lsje@eY@Yji=;1&R5%ixpepBG}LCZ?u{!Ro?P69*1ogNs-bpLK;y7CkN zNLs*)M>=a2Y>y;j$u(F+{4t;DUR@;=Dv&d}*t(r4G=Si}l$EiVZ$`4)r4s}+PVMjE zw?RgCl^oYV=c*_)TZonFwqep~@tX()9sQ3d{@a^KBfOolWBO&JX~4rUM|)1vwT?4w zj3gGDM8(7mfNTX2xKKz&VRQ9oIGwE#^IpkI{h*ygqvHvBk1t+Vta3~RlvySo3%)=R zOL|ZOm6bnQo)e9n)@xbMbIq}ITI^e(R4gB@dd*l9y}t(tQ3zzOzT_uHDgps8`k&nQ zxpHUa%%9#LtwsP9-bEU8%4j{BI|BoQ8`i@w*}_mNFJNwyKXWCnJtL3if9iNpN<_|0 zHhRh9AOs*_ZCK*1hkm}2R4&q{`jSEV&(F&MhEIqo85x?;TlA9V4@57UK(&9am1|tL zM71o%HPa4x4+qd>VxPb94>{ex0#XKxmnIVb_{e=oF?0SW zktLc!&eDU|YIK(Rlx@0XRh^K-djpCg2wU5%X00Z-6pmI~(1e5vP@}ot0XeKll@9fZoTT%6 zQ)j)h@vYO-g3)8N|DV_RvR2QYaMi3R0^3L$&ErH5?4h>sBcRL;aKR({y?+;gbv3;| zml71rzpEyb`-ctT-W&y-mg=A#l3KaY8vtnB9%o(2{`EgpaX=qaU3E(}7@<`LiuR5E zB>Bhm#Y0R~Lb?pcS48votVIN=$OXoPt#}WWunq_Ie=mUUNgt1G{>MZkOcalX;zt~> zr;n5I_Yn6+X+#5N*QNjO0pR~HH_?idqQ(I{7;$tp9B{-Nbjj_4v)nyw6+fC_Iu5kV&SzHl&lRo5h7-1o|)3sV6rK0efcp7l%E^`H-Q$s zA`DCZvSdA}-640v;=PuAB%D!;~}!3Lu6?FuZV z^nfX)aR8E(bpQRkDS1ZQAK4?$u;0wxKABIN&_FJQblxdIFg6;|A-d*eoXj-;*nNjg zSi0DXTK7>|WVw64cH$i}@-)p(m}}qE{_z5d1mjD3!cdTd?hm6Z^TiPCA_(Zshki?;ND)tCG`-%{DB z^!Sx8pFUBWN-I~CYqysoLAjEP0)~-n0w$=&9`nR<_s`=cf_`1NDWaQQ2R@7iQfij< z^YLXUjTMlAkxBl4Iz~l~2g3F)19hKpbcs1ole-8r+FcZKwb2uX@dFFB3Yc8FZ=Fvw zKdLJ+;`$wag~{PXb)>`HM_B0vRxAgPFX5P|zz{nIvS63w?hU#F$^D?{W7{A|7*Fm! z){7lb2`C^SfJ|K%9#{g`+TYe5<_e@XXw~dQAK{)ZY-HYiAp?H`XjI~}ELiz2FS{jH z*^Ku0{BExSuf?ltEhc#@4;H$C_HIr+hkN9AX?#)8V%3GI90)T6NFrNE0-}X->V*_J zs6K_Mvw$xM>4TBO%T);BA^Ua}Eg zv3?SvUsqLT7u6jL4o2uq(YrA?o{SIXcKRTKD{grF8>RN;gmZ@Ay_zm8a;X4LBscPQ zrJQ$r34Uz9zaTtmrC}d@I4&FYmoW*Q;7B9J%JFV=OGf{4*iaVL0sq+7=OAD)Nv*16 zhQ$h%$QJDKjqKY}hlTY)7Znta`QTDDtPSaktS$AD_h@92ysc_RtBw?^9-uf5rZc3V zpdeXbsLsdNHzO;QIJ%-zZY?U;Gn8&BPcOlD_m$Vze!jjqI2SKCGE%sXHy-6!dOiBkhY3w{?=ov_{hrDdVBP_u!R%A-a)A0X7o7V?9(Tujx`CJUWt|{@k2PnOLGG}NAWS*#9f%^&)f3kBO@!a z-hGz15toB1XtT|hRn^Ib_%%`_4O&qWJToJm(-U!J1*S@qDul~Ud!YV!mT5b7RiRPx zCpt6yem=3FpN}01kTH+&+;lB+t3~OK#Cq4Ai@N88c|ByQBG&L}vVlK`nL|8Nk$6Er zsr3-CIFUMF2Wo7?+L@qKrv(#@T()yOTjcXpwSx%wNGiu{+Xuh(7i~PW6ythc8j;w& zA9n@O6u5b}H$&P4<(Up)kma~L@IsptqZw{?IE1sPcDsqiPtU0Mwz->w#+9D zc!W}cjAHxkH>KA2Bi^+={4|kL$#3~!T0+`HefVXc6`ej!Aj(v0RziNUvZ!*R)<5rGW&x< zP%^r9m0~TrBZ)l5`%<`{iKdX)!?-O8?f+Ovk&UWw=X^{&U91f?C70g2W#uJ0O-86Z zXN^5ovn;&a>_BJwQ^$$q?4;$iSu0V|!)s52aRtiNl!_G_JEOPE;lovR6$WFu43kbE zK~NkKKR^GOJ8uQ3+;(1NRA-IUG*ifAT7*n5(S%F{7SeL%14zD5vkm^MY-W*w@{Jjy z{Jd$3$~W4*D{P~lttz}Jji`Q+sB(7v=;KM0iR)=Du^0)WK4)wTEN&F721aA>LhZcj zYI-ZJwBZ8C;?(dN$^a9}{&$QfNneMoSFsV6oX|)4D(QbplXz#|?T{szsLo@yAyJvU z2m^asXe31gYJ=UiLpLRjE0gi+ZHqKcp-31ZBVs%65u|j6Oyp3k+rD2?2;(_o?BRr& zSXOQK`9C5~u!DDyO@|Gt*`IRFU(Tw1#Hn?Y2vKJtu^o+9`rQv_=q$9`z|E(l_7cYg z!gyy>-(|EYP;;|PV)T57i208efGS#M6x61$I;JsgG}sFUpMr;jIHb7Crm8dho$2|2 zM2VU~CtBGA%N!O?{>?xcK1|ihB{sVbmoH(E=XNt16{9Io!o1Nj(R;$XI-f`yFrH&?FP&xJ-Cqi3z_F@e{)R4eyxOpq0l z9B#gFtRs~lF_}NbIntOKL1bmrfDOf`Pl|Lct_p5gImM>pf;+>JV!&)W2RCl6e~riJ z7YL5w;Yv_=r!-%Hs*ebpuwLZ=^4L)d59;1?V`)#ypV~egh5H4I41w{Y=GGdTtVems z#3=98XoU#omu-GqUWmJyVarO{>{!#(6rQbrQ2kUpCQ$Pc%K}tuuwbH&s>Yl1vvbS~ zf2Awh>!zQpgKDhoT(*}(YC-cw=-94$rO{@&(tckN$b<^xz{W?j6i{f^4TS|p+~Y#AjLpw0;Q3TV`X!?^MC!C7s>5!2;O489(QydK$wZPB zvk@F_(RfjHbGIjsyiiOI5fT!)Hv(NjzH?Ke|d3%zdJq#n%d> zJ033}x+)s$Nls7vs_NsJJX*Q}XF*Eme(px3t>Hy6Mxkalx$aOAFXB+{*-9w`NMQ2w#r6r&LoAIVHuAGsUrh@xbhpilgFosY@J}8SJS_q=+O8K*$HlX zE7pBiDVs|*+Wb+uJ>Lr7ob^!cN?7_h+(7UVqWf2Jnc08^S5q?KO+f{-E+wmc`1rg1 zU)_A=9y}P@-`Vn^5rqQY#)T=(J|CpkQ0@n|R!iAUP|JnD# zv`N5{@4a#z&ctV@Z1yXPB+^ptxODt6Fjl3tQDw-uUo$n3e<>qokS+cB8?tW{rWst&7$3Ycd zs-l6J!mdl5gr?;=E`gsqxW`Y-s6a*Tejm5yV|hNX#8z z54ZjuEKZBx7?006A_xhhhHV~k@Y|4(rXWb@&@%5Mp-0Ubv!!>Mo2G86;dwTmAcrEN zQPR}bO8~OSc*(8_9|{yd_0Ws4RJxaLTNtqpx4^a;?D)V186+N@{Q7LRlyn30LsDIM za;`q_`+mkO({3i^kn8XP-$3!G&qGRGjCo1*pEK$_=H+<7ZaJ_Gmt^ri@PVn?l5MU$E1xZ*6a(H5CT5vaF)XI`i;@%e)9H%(Yy zBJHx{9@TbpMiNQ3<@yBLfP&VQm|wT@d3Hmcv>h-*(b?3Sb-PX&@#MUc}UHVk2uzZPm8 z{`%#Ci*Oo==La7kont0Q<@||dhHx6U)#ciqGYG>d!oRxT}Ld z2=SAqBDc2#`J|%skJ4gG?rBsftKG_@sPh*s=ZHZ&fH#Ba1-2PbgZ z`q_(~$ZYw=TkbVtvO*+^_93Ui(viafI z?@8nAZQT62xJg$iS;4h&;8{%PgcD@(T}kbe6o=EGV@DnFK+I7FGLb(%8g*`^vNxLO zmEQA<$@tMwaeZ;DZ$F;bT*besF&T?Q=GP91;y)-ygnr3Jk~ffeEf#o};ktTt-v2pI zsz2C)9wPQjI=n=DadKtfK)t_FLg;l@6bsct*EMq*OunA@NCsM(%9NSzkbWW8e4TAg z_G_r~<-DZ(ZstTs6EpTzk>V0oL;#ajRA50Og(fs>|8^ya6B|FUl{jMkCkTQ+`N^7} z5%R^QdcFHM=);sQ6}gIGHzM6&9j zIio&-VGuwLPj4aoQ!cTIWFlbEUXPZm;v8!gpt!KvdCT)l`8U2hT$lOmgagWeTc5XN_wUQ+_y-Gku)SuL9fqgH~cQ>;@;yOezCF6a9hP8gJ`*FE=V@&r2Bp=MsOjFHdvutUv-K{HD- zhZVtlLVj9kKNe0Z#u01;AL*(F=|9r$-(0l98wA}*;x?>2uq}2tmgw=_qWZxE`Z?FR z`t*ZGD61qatr)HOec*&PcmvHc<-#0ltg}?}mbtC@RML?7XBSW?a3!TQ`(+S&V)5Bfe1((SLim` z+}n~qJ@YZSUQwONEl=b&$X5ZFRyf2yIRU8LixTA@$`4C$Yg4W-=LG}a(l39~&9Bis z`L`S3Gw(it7Z`({KX$IIe+Y{GPd+c?{`lE{7LF+4 zU%*UiYaGO}%jUn*TW5o)K67!T24_uh9 zA6O%hjJ`F7O>|0if6ZZxcYcXf4|p{#(yk6yU@o%970Wcy7^BSTpQ~)wj`$&;^fesG zoc~fivx7LFAs=tdY!qZ$?yn0@{W5cr&#;toMtYe6jf;XPXKUJ2yrG=<+K316*ChHv zG1v&1ZPBYg^0~4O#nY8kYkzEsy#Bzw!(%L}2m{7)>#Yl3(#x^iN!vA)r$E&o&A5M# z15O?V4KemZIK6Z?;rR}(v7;+1V`EB5AKghJx2w2Tiw}Or-W9F5-=761w^K7AINYYw zon?PMP3a@Vkb0I-XFX-9)q?F#a^;9iB@S}4??*2H&4&(3gfTeNUnHc#MIe#LP_8OM zEz~7}2y9Q5yM3AdZGCm(fP1`X+THQ7yjTF1CYsHhUABb{36u~awv#lTO}@`i^l*Au z;EG6CX$>j5EY?*AeX&?phMSY^><+<^H4uMZT zDB`elBW7hq;GJ&4x&l8)%&nq6k{EtDS5p41foKA+Dzj+N;23bIZovZcoobzzH5f(_Zusg27!+ERyhN-<1Gkf$NA?n6GQgFg0j>9iu^rK;c*pmwf; zy$EpZFX579Y>&L!)tIEXsCRlK6o8v!OeL_5G>=`L_gsE7c?*sUx zY|r{0Q~g6pO+8;fmiVUPA1!tu+wG42=?BVhdZH6SgyKn>I%AUF6%4?Uxj7(OjcQ1- zF!VomA7$Ge%&}P_v=Kk*YV9Zt4q7|Oa+s)(PH#Jy;(OxtrA5s~Z_kwv`jP0PJG#Pt zBgH1^JaGutfD7R>)-YF(DKf9O=!stkHU>QTKa`@(K7~-MrQhWsmy_<&V+UlkS$}=U zNqyWeBbc{B`NH~%GSb8$C8ZOuuoZQ*x7K8Y<;SG8b%GV$;pijrY{lky_E;PY@l6?{ ztufg05Bgw()_WOt$PyDJK1R1My67~j$R?nKDHCk!ts;06YBdQ$)8TJ!RH(>}$}|CT zw{{#Z3a3+?jD~R@)aU&~jG;VUYY|-rLt2LyL95v4vK-ef0+uX9viNrIjpUQ;4`&w2 z+4#tgru6SFl93v5@B<`~*u<|9IEi=8zFjN8r1lNJbA1_rIWs>e0&tPCNb=2?VO4)b*2dmiA9NvnZ>M^cUW zrGc4>DrM9Ko2YyI^q5IC1M0MqKX)R(BEYs$5qE<&6iz557R%3=Kd6su=+fX(CGpXP zmXL!uoRkq_S{KdN2_$Q~p3!pE^*zUBjM}G_G3rV-=yX9uH+A7AfpS+6s-=#eq5ru3 z(?E*#owLPc>2~R3ewqJ}#v$j)lsY5BHx`$f%S$|q%#47*p>LJtX~sh>pai=Q-QAY! zojwo`Z9H^~t==Wye}65h>TG8&vneq}!NG`ty4)o;El+bbtgW%JJ_@Y)&9MTXe|ekU z{|UsEpW56nUzuQlkBcSCV#vn*S!>4=rKd~2`BiI_E9-+5y35VlWpd`e)Htwz8Y{g- zPqIXWGf#@4I-P?CahEq64-K&!fLGP^`hx^2Kd9#V3~}!gtM?^OS|XVmYHXrb&agbr zq>kV^iu!d@tBQ2ZAZzKBUYBhzd#nn0xZ-i|`KgZGt#9n0Ni` zf>vEZ!E}GkgZ+JaFqq~5&SAE^g9tcE^d|c}h)?cX;`%(~{xo~luD(E6cJRaH)oSBo zQPS!2_wY7)3L&Klj&5}UZ-YG`lO0lT#1P9R#_Ku=@E~Xm=;?y7W>#YCt_bK_ z)d~QP_m0fqchNR9kerGf58F#9VJ&u9VaL=_wcbReK8pO%da|l6H990;)i5e$t$bC6 zB){)813lK`O1>4mUHzx%M_KW%LK5TvR?UQ#;5LwV>6d3DfGW=r_3Ngl6=5*R(vb63>Z2NElA8q8xOxKk~5zot*o5GLE% znlG5HF3qd2Ie(*ldKoyKYqFyVjKV2Fg2-=)p~&~TEK@UX;Y69C!gb_o6o7$pK*QAjTo)f(>g6$->K%U{E7aIBayf2kJnVZ z%c<6-!F8#~iHoEf6;xAwlD4zIU&ISdb7oucrP1@*ifz4#6gYTHY!q)SNAjl_Ha{)l zXnI&@R@Z=n5)Pu*i-Q;rnr>BKFQL&UT-N`?-djc0(X3sg!58kH;10ndxVyVM!QI_8 zXo9;F+}$leg1bAx-5vhUmiK%2{>C|DoQre!UoBX@x~jUWx@6XT<`fU$@07_0;HlU* zW_*vd#%ZaJ)Vdt_AKtp0^{b}=j9K7TuQl0SlMk$L_YQP@EvOXH+I4nI)p!_VW8;jJ zx7>xln!gz*))E#*g`_v5xuVLl+z8EuwQts}Lxj}wMYYZYj@yO}uK8$kT4Q>1Fx+zG z$^^j@!6ABK$+3>a)C_^yRoLLjwn}FvKf_&?)pLBKEzg%6z&p&Tei+laC(}i3f4a&Q z8<<_FzIbnmNn5Uum#jo7fN!D;5&5vRaHx5dJM2@|vq3}lC9z^hhkchM$fY4nXos8T zffdV~)qa)~4?X6A$F!3PZG_r=B=&+e#o28kZT%at#?8W^EYMJLV9owP6T#E9eHO^6 z@9k5&>ww0hKOzPkg`iqb;~>n{p0@h!pA*~AB{1mg;lnRC+V$PqEKih74?DCzkABiW zx@KqS3`ZA_J@mFa`(1*^3XO)Qr(AZPvIgOF@^ev>n@-$#dIf%t{qx%^nnXFtS*ecF z^x4_I()eQBUDrKdbT+Yw_k5tH99w76y(tvA)%=JTUX-KVrMbk9d92~o*6&D7YE{tD za5fsnj1+i?2&v33^>dQ2Pk^zwdxlqSeHc`e_Z(IIZk`W*Dz@aZlOXx9AZb<&#t53dd>abMWbTK+yiDj9 z18hn|#H;a#_MbekvS? zD(@S|+F``L#8zef%AYzbN8HusMW)*uX|`vz9Qka*LfC`z&N@~skO@+T^E*D%!}m%_ zx8#q0`c%(vCsHZmh2zgyEi`*Ht20fz=~tYMzqaoYpv|OnnMav2Pq@8GZBK%@Pm(_> zV*z|(;Se~TaoWlW|6-LIv}U%;uch1D2Np>2?c}7sTk*k{9vxIMVj*%W`t8uvzwB9F zl5D16@3WSQX&)dL)jq?OtWh3B7hK@kS}|%+eso&DDB+JCh5W21EEmciAMY{3gUFTyQe0@V(<}-9S^;+WHkH++I+ia;zk7*Zroy5vO0TLhl>Tur@?#@JhpgY z9Ej*Qs?hQn93PkO&tn)6kEhIcJ)t=q9k0lU8JLZ2Eh989ieA|kLB>N+#XGDMp+f#y z>M?zKfqSNZwV7;3ZBl@bMW7cxa^PsN)~UEZxrO+dTSrsg5oy#3FDy{VKi=a)MPY) zh|G-2+GIItHY*c+^M4$n5D)7>TkTc2R6NZwo1O|9Eisu?+E%W2cDl0R zg@R`)Qxy$?2g2C4*{FFC(QOG{rJR)}p6#QWqgHCJ=0<`a$gv+DE8LP(^eEc$hyX*e5_`4I=PI=-n&?ZM&jsK;*RGV=NY!J}tM@*0&Ey3Z*8nOj9tRt!PVxs&#sBN5lk?)^X| z3%(+gnFE_RFUduP+Z5s+E6cA&`04w^E;7X$~@fk+Djk(&JgXV@%;fw#3gK+!wr| zZpgk2`XnW zs-wN3$uty?G06-8*{rk^k9pybyoa%sy^oWW9uX*aq8;%LO*poZ7)eoJV_*DO zc+H>dYx`!$vR1+iBc4BhkjI<-A(+;S{_utWHMMGwMrA}9?88ul>JzFZCK;QD!<*Cx z-yf>c8DC;OQJ2L7XcfEkbb&Le1-)P?A+^XjgYodi2i9kVP}!^kz2CQAjg#IN0eWli zJSKEqRAwq(TChE=*Hzmprz(85+96p!^`!9uCsaqdYiD21<`U4kE zdy09^xH@ok9;DGlJxe#3>ENh^?JWl^RBpQv7_YBaz)0D=%vQz$#K6x6r9^^s^I1f+k?8aA@P{gE2})lHjNYS5cO|4y}`PImach z9@RWHpQXjj(R}6hI7FE}qdT{)=t_70Iq5#Gw-=|%pi^ks9rd-^6*l|x0|R&@=FAlk z*w5j^0y!Oco8z0ra~QAQb#7B<*`+wx=>C1Z>~P12|4YpVhy{wEEpqcK_+;TDy%RH& zI`8A_a){-xW2*LvPF`EFVA`%iEfv8BhS-zkFev!lp!j@3zpXDK7dd;OSq3z>rs>5I zU+8|^;#P_=i9w7TQ*fu(i?CDnpOS##!90InU=_tjv|h603EnG9rGr|xCDZ7QHUB!H zvC%^1w#+yg{r!@mi5;i>_(Pv}Ir`U%ozME90&;M@gN}&`a|n?#a-}* z)#9Wg44w`K!>i6N`>kvJVzp9Og7GUfHJyd3-4iOe@_wg{2<$z|6(0P+*uL?+Y<*vh z;S^j3uf=v4!6A3|pYpse?H*9HrVKwan4=vIqLCRZcbQL2>}S6hz=Nqn7Bi+*6RVq$ zz4V}MSc>DSEYO%VX|cOp^g`Xv#=D-blpsRk!F#tX8;=r5JDZiXr~ew1ya2VqQEqj7 z4t`?AE8jC9m`2yq9Ihu`_LA!q2okmcC#c7PY!3EkjE5r~`C-g&eYWG2uxbb{D2O2yMR}{ zj-Avi0yI+ZUc1V+#a7QlJK3gF?ySK{6a!6P;MCV>%p}I-X#@5L^Luj;4g`_ukyQ-( zVJS13P6#3Syw$e}>S-b02sRv@5SYS{ND5;N@X$YWgNw1pWJOwLY@xiogr?k@h!1?R zER=x?BwV2Rmxz)(7M-5x;jR+c$>fdntO21Zy~^98@vbcc6qg`46h{+K%Y@q3o`XYyS^k$KH8jYo6rlPbYSa1>z5|;h`V}GT05KfP+Df| zTHmh0MQW@|Lm_FS?6GA1;ZC3tw-)vT2ZA|__(7ee9}nRdVeJeCcuAzdqZV?X-Hm3j z$9=xh_hWaPL8E$_S@6aDu`L}Dtj{qEkyFi4yoNmsn9dj#kpo?q$)uT=Jx%r%-in{O zoTr$i3FSMR!7{D%dk2OWtBaT7ark}d*1VmXf#`C8BPB3I`ISYTo;Mun1J}lLi<2c= zfmywHk4%i*18D+kGdvi}JQ0~KYh0T%)97^{%V?AeG!3K@HJB%E=ZH6|W?rS?UE;Q9 z!m!*+Ii=8EI8Tr_0G$248TL;zF{;9>VnMs#8j{#+IvMa9Y$9b8=i*&V)vk5 z&%brR7&;2tMb;M{@DAB@c+mm>DAfqCBuhfxMZ<*Be5v`WRaqP*4C0_9f(~+`j{`Es z0Mp0By7pL5Y|pA9GflP$wAn-6&X0uIK=t8{Nr~X^fRe-YwPr4dCu!ZlAzB^oCja#}HoxZ_~a(e0MKPJpq`b$URbZRpnI zN=Wr%pyu?F@Vs8z8Ohu!2bR2a9Qj`)(gIe1kWx@jOYwpUP7gXSwBWW(njjXY>orld z;7MU5!Jb9Sl#%cAyBW^W>+rk>j%ZbLl*uZy&wFd7ZD#g7=(rVO`t~x#7IwQv!*%)RR~SANh;z4^hEJPLJ6$oYhK} zb{`*wy-9HeDq+(ZO%JRCSADTobX=waZ(g3u-`n(BUqAmlSs@~FG=L#QuE%yzhY}v< zuV?AD+i+I#)&|{s}^#6{hRyg%Izg9 z&b^^m%g2LT@Y??gJScGgQ~USQR1hfOl9cvMiBD~-dQ;+LZ2zdjKg`_#FHs_o3)s8T zPg9kDmHuBaZ~I2ssD&{0BCc1Kw<2wj)*$|s@=bWNdA!^aKkQR^X$&ovA^-l zyTyp)$1(ziK+RBK7mUsQCST?LcJbxJTcKUT@c!{0AiPbDm?Q`MPaph81s42alpTx|g#KsYC0PEn7)!Di zFn;`1+kY~Mll#LqXNkxAcP9V5N;HZBq%*2xiCg?%32^}o?6@%*%m1vk09GKPB$U;u z#3bjx5<2*kF&9e-@qgw!g#e^8HBy-Le;?rg{}yPE*X#GO6cG{YT^=6x;Y#?K<=u5m z8NAZ`#nncuj~42d|65%B{(P8^v~WRGlt@z>r8kfk~2(3-kO(x0m+2 zRv%aQ(N@HTy0dpu$uwUcU947bn3^r^l{eI^S39L0RF3gJxf{IL&+b%)dGC&+?F`ZFpy;F`wd6)4!`#uK946MLS633x^ly+ zB5?U&>HD(iW7Fz4veKwhB`aY5awDofF4Ax45g<>7hDbaiOnS?cW-C|6Ut9atRgt+0 z<5uSrmb-xbwpxu#t5yYdcJ^e!2x+?-L+JMap-Q7V0o@mL81^CfZvi#7;zSQ=sw82| z*C$L1Emn5zh07i1bW2S>>}$*;Jic4}*SPmsB#CALaUZR)J`bg!0#@J37K&USTl8tw zD*CTe*K-_#T5+yD&wjJV8+5dLIOfhO%M-EEVIcFXF!x+45gCHBm8#BVg@8X?t$wTN zX5nl@J~e%UjTMvn+dI_V0a|ei%Jp7khQ&kRb~{s4L1V`ei^?pB4q-;-4_538Bl7Wl z(_l68<;4a-2f)HTda~67K7+>vFMqJBYIr2&F<{NNtXEU5LNhEUhlNV1nJCNVCc(Fq z`)rw<%kQKM-(h0{m1nXYR;$7N{oot-7kR!XaB+Zr5gUt4lfV!(@98RM_ctZo- z#Rdl`B*Sz-@xNYjSC*XPoHn_)<1Xx$C=33q!dN7zY7f5L8QK8!`wVbEI}vbYnk}Z- zNpvrggxXzMUc1z+UiGLk1pEQi)23U+m%iuEh)45+v6Y!y!A6KUk1|fcCP+Vn#fCTzXL2iSt!|cXQtCFiM}JrJR2@COKqL2! zrsv|UbbWH%8Oa??kRueQgI zV(9cC+Yu@*W?^)_dRs?%hiE!*oP@1EI*G7&$Ln=RYCM3Dqxj%nCtguhdj^xq8R-Lq zv^jIYVxB)B1&fHNoR#5m=(~_N*nKqDHb~3id|y2LdhrOfq!b1%UXf2IwGY=eE#_Dq zy5XTt2pp!uohWt!ot{kg+sP!l{q2ee+bwpD`7fpS+^>XiSU&-^cCI{~)G!xZGyO0m zLcuCuyN%Bi7QcUG!b3cEmLd$JwSH`rOug)BlSrn|WVY`bNV4I+xQ>rKc(OU|?LS<5 z#UAH*32q&1G@L)}D9vEdQ^~cH=b00k@33dW*ISkp>4$Cy7T59P}#q?y!Dpx zwWG_+%Rljf>%a^pD*RpBnXjV~@d5MLHz@Sr%1k0#Z`7+fK;QQbezsIaX!KCkNcC4L zFZV5N;RC?+vIyF|+@UpShTSp5#O^X#MvBV3&+U*qFC4!2T^#pZsp^`8;4$SVAt5<7 zDb`UI60#*?K02wHqRNN-JXFv$S9OCA(cpMo7KJ}HzQOh+g3H5dRNh{_?OLJ#c8?#D z*!wbr9=ln}oVS2Xi+*c!c+Jp&D##F+ui#O2OCBH^uC&j1( z`SSM=X3-0=UF_U8>q-3VQBc^?Qy}#3Z~24H3kiu2hKXXrM?n|)G7TKsq2;)nDQhYO zt%RK~(1wOh$1?87ib%AiNhV{NO)p&>6^EB8u;f z?c0W-ZZv^Er@n7!cG8G&a?Huib69JG)qXp)i5XJWz@njqpbxu zXI#F*)oKgra(Wq@t43P}z=4{p!*%pfYw~Emt&Q)y!7S)IzMk!o-zIm{?j50|^pPpoFIBt9}#-Mxz} z^_L^T{BKQ4_>f#I-}Iwu_NK4Rv>U>egR;JHvhdFu>o=M=sxco}EHu3tl-Cl<;=%9Y zpQ`+>*xK6*6Lm5*S}Q+TlJ4R0ghwNVnQ{4kA#yNP@}X0oQMyUFKtb@wJpWh_(@V`7 zOw8U&MLWSYj=~xNmjW(qHgJR&_2_}w_OQ+-WNRah&`p5IFg zeXG$8m%1vM?q6|65+;zREj;x8w$SG`rR0x3+0My7Yq}o)Y^sm5PEexhs-l*M5w4_qLS9NR^g+2&t?|~J zfEV2PI`m+Q87JgX{)^PpOUOf?D*AAK_KJ_d&Q#hAFI$RI)Le#%=jUSZ2{6l9bs`u( zJP+GB<6tg#R%SVChj?>vroQB!r-_uDlaccAtpNc!9Y`;wYK3`ndS?a&skJfMe7}k& z`g$Hn5CGE9m%)ph@mZf7{5tPctMMc4GBH)33L15~D(Q(jgG99ci2bLWl(d{o99RWI zp?1OuEc)P;7RN)Hf&BQCN4j`%U5J>b6fL$PPzPEDgot66MeyaCv4)bj(IF#7$=g>q zM)#CaZXtPuzFCpK4Ri7vSz%ckaERFPRp>^Y#m{UW1-X|}&k;B2;3u0BI-UtaA+bXFfpKeZw<#kY&I2EO z9Xk8%{9fgojCzZ=tfqcUOi$F{A2w5^zdFhdNj22fZ-;+Oc9$6MLv z`7+60`He#nS|uE#HLadZ*VotCZ9{cd5n+*=Led@RWsLeg?KI8=@tk*;$QwUWQ~tH^ zE|3Awx+`0b$(8(qD9Ze@t!#vTdUdm6cS1}&Qp9W>-O*LBb*BrP4PsC*#MITZt%`iM z>$mJE?3FbfX+lM3=u)qU9cLtzE7%cHlj>~ukg4Hbx|BSba2qfme@7|B|J`UbI>m|c zM=-JCu5oXM3}>Cq+RQGSMx{_H2h!Yd8hPZ{SSD@Q+UZi8yxrX?zd?EoGUaAHQloHvm^B@!eSZZbXkHmO1{D{wmD`j40 z2>1_`dOk&NZ^T?Y;SmUljSe(XVd!R`>=wUMP}qU1Cv4I%spX0rj;}#bm0(z~D6)#j zShq^_JDS|Uu^_zR-0y-ILoHO(8Ai5^TnoqhjzT;N_Sl;Y=eB0 zw>etGX6M?ZF6goBmj2&|jJqYrxLr>Ko7P@tr8@|w%Jvv%f}KF`3jvo(&#}N%4y)d8 z8mA5oQe(;b<9+LvO4m%7mM(M@JcaI)BtHRDGSfjv6^a%NTcy4PjpQNr>_g>muUBW7 zo`KC}uAX;iQHE~9rFM;`Dua8fa$Vmneh zE(?g;Z(i59;k7ggQM5{kz8&g;FHWw6DHB9KjyI|;Mw~a!tMcn%nX^7UUO%rdz6=tu z^~W{r(1^22#b$F@Wb^mgQ)Z3-JfA%4ouB`dG--Xgwd@E*8A|$1dXM1cnnpdet3*8C zsQ<9RvlE03a@VW)<3{4Xqsn%B>B;m?E+y-6(_LA%@qe0&hT>0ig%W4;P!4Js_za9V z8POyk7Nk(NBls9Dpd=Hs5r5LbELIz}Y|VHW%2h+#no37VHRRj-fN;DlZRS)U0}3RrNl)!6DTX}_@P zdVbICAQYy1qas~=`K%)GrnZKXba8uI%Z_W69U6do6g-z?d1+He5$%j+{EQ2(qCvgZ z7rKz(=;`k+v*``)Nr7PYVQ%|#d297fr19CzE7Yp~?Txz0aSHoPLZZf2{w@QKNrUI? z(EiFrzsAFRr}IKu&y|(4^R265twr|afGf0OfvZRTW_PBsc-v4$z)JZ$Kmg}^;jc*2 zyScrUto9o_g&4GwvYXdnvq-tpEcV7X4nQw&ajs->K!|pvGd@pJk}i&er;Moc(aWaO zWIj62rcD1dJL}#V*^_)636=7i%Km8$nK_mKlNX<@|Xp zDS^!=rEIORdZUf-x({z7Jil|Jf)nk;p^3m+j{^*#mdN=*$3Y&flFiGIT_{F6bpPlhB(S@&P?}xqkrejNn z@aF!FJzeqyQ?jzlTxZE3?C9K^x=hjJNTF|?*=T7rji=RQZT zl%e4ddiG1=d>=k#VVFTRM@(eGRmpnq89aI8oklg>th6VwT4%ssp)q4=#M*(!YF^B& z$rwE*E3C^Hd++tGV&Sn@CU|(_pN2MDuGgQK(=ld3H?oi}&wQ)+3-0;K1zVsIMC_s$ z^E>u14lZ}gQW{~_;!Vg3L*0(j_0xq)MD>?`H_W{4Dz#jr)MI)CC^!80>%Xfhlw=A- zAWBM|U{i2z8}FlKw_lNx{52WlQdA#fZh>Gs!37wpfy-7f%u*Y{HJaqqu$S*WABb#=)Pv?RiP zTQuQ}iB|tZZv_RMI_cedZX&gCsMp+EnJ9YmD-j{(~EekmCN&H~)N;@E)SO{}71z^AG(W030p)^WUI>e`1=@ zK`RjdokH^;s8X6W?!U0LV4z|gk^eh|jW~e_R-Cy?-LPMTK1PHpfX1m@Hm5OvXFB(M zrS9Mkt@^2H{1?k3!27G&mkz&3E>q}_oK1T$+S$MY8dsQc=e9e62;uuDvcbQVw>cT|G_~ja=97~`k*)AZ zD)(9Co!?2TACuu0nOECNl}Zbj4t3#eAnY+D42Jy8suKcqh_t(Q&jyJmZD zIov-y5bJlgyE$i>4EyyiIBYxIZ|He(cX-{1NJxag-1{CwLPBng=Gx_WHk96NJ0l;z z#~ex?^nZO(s1u8ZnSL%UrwsBJJ?I(y43y#78^^2`qq=Q(JJfabX#;nMX@e86`iLH@a_-CR3v*d(LtpEwKgiyGyg6$1qZ|B6 zbnA1M#oLO|g_HNgceYqn{qCfJuWI}G0iJ6;$_B6iXhq)C%<3<(95duR3H-1XDx zHlyWxi-vaNRZke<;`Dyv=FLLAMV(w^M8wp?4L)DqG6}YLuGvOOk+xMR4|WS)i}L{* zG$KLip!n-Ly_R+DmUUIJLY2#5tF96Orj*gZ!NSk}$R3>6ThNowGkBA==+i9&2?;Z}T^EZKZSt4^OeSB&Rc~LPi)+=I50lBE+4;?l zXqlQkDV-_(JH$x92qZ`k;?^cn3BzpcS!!PL>?{@mmwEqXD3LbTDCF||{JeXu*MTf?bOKmL9jTC>s*EFB!}!y0TRM0b|90vlAVq3FFC2HT;`oYldt>nDuXxIfZTT*P-g^gk-5YX_ zdo;P?qDNBkcAJPFO9VX5An#VWpzX{i8@)&XD76UkN3Q|QO}-B!<1G*d!5~gNkxOCB z%U8J-7cByC;S@oXS++%C%MVK)RG2n5+u2-oy2s?h!LGNZtmC&F9z1M%o$kG9zFBA` zk~Bxn-_!C%xk`l*_YN&KYr;*ox?j2N1icvLTDTku_SURe5)fN z;liHzby~im`zfAc|JM8mGo9VY{8Q^9 zn@unClL5}k!3w#=Mh6g*6F9VtpNk16HG?=WbA(Q;jszgp@*xzo4G~x-4QUYRF@<( zS$No%T;w{lVN=<;rz>pl$R!v2=j@KMckx>qy&>d+NbsjfJ+R0^_Moo<_F)^Rr>AIv zzz0eOa70>7Bp3+8uY=4laqjanKbgL`y4beb##_~DXJ@*!mtAsNd0L96Ri^UY;OdNthzf!dUqtXB7TcQDhKq@@rKu-(8n|G?C_;s0lbzXT0V&`6_>C| z;fF|*WK5tx5fKOk{*Mnx=zzDR3yAn-dgK|-oZiWLNm803R~%NGh3ao=rL0 zF!C1l88sZ!$Si?IRVs!HP<6+AZAZ!O>&7qGzVFTUV@j+L$zJ0v5q@f%DfggF<8Xju zvs~!Teh(&ID3=+5F8Hpt%dBptv$NB*$9f&iB_9Hs)buIXVyZwbbfZXfhbql58FWxT z;jFQUVeWl=1+&+%=oPs4<;+=tcT6|11N3wDNiu^9wMhOObBn4;G{o=h|9NZ>_$Ch6 zotfh6%JG0Kzk_uo)+m+oK-65q7j#Oa0K@6s>|)dC!}myi8aTK`vgx4|vS~wxs--Hm zy8*sC4t1l@m2Y zyK;D?UWXKBSEayYzNotA0WxeO{3qKrqs@U7Y)ihJ=yZ1Ftsh?w{2pFrqMg%Lh9+=N zLUNs@3R}@=)nxGa9ng-ht(M_4=Z@W99uc`*pE-aamqdflFKQ(n0{$SHM20tVuMj+D zXGnaf{OE3Yo8E{gz;*8a361ghE15PsdrD|2Lg!ue=S9c#!(5@zGm?#E*lw)x5eZCM zwH#nrM?EanZG#q##EX^#=0M1rI7lx<1AgU}8QHHyeB_cDA|7Oqd0$BEk9Lg?DSfbg| zc_YJok2(sKGE>CQg|#MK8<$`2);he{2MQGI0NXbLa+wtW4W^=%WK8ps6o;Y6Np3(O zilvNghvA$PpQK1?r-^wyq|^^{$WQbk@$0*$BE$qrqVga3h^o1!%x+^cS-<4cZNo=*8=c}zV}Os#`7@iq{H%mACc zlIY7VmDi)8dbyKBf1nEbgp`KWZ0=hFF1bQ zF8Az*bS>-MO<60xO3si6U_+3+puM2coLo0$UKLYadm z+Wag0*xNf3KO32&W1c3oKwq^sEz`4K$vX*F$~SzyrjVr$e0M(gTFICSK9wu6yq!TX zNI7Uy3iyoXpt3G7v*T7+*uS@f9%{IeB1T+v*j=hKZf&);Vl#Vx1V6={SR(5Eeb};G zF;dT046%}1+6zXh{_&LYS3UyTrc@%CT{c19ozvB}LsC2H47qc|=Q7>$L+=4nqBfsB zqz;dlP}OhXY3_JeTklUE#b~8 zxJOXX&GH~@)8IYOr($33k2UY(Hd+9i>79D-Fa$va``fG>4R0$Fz{=~DZ`!}~?#%c2 zLR8azfYDVhnZ2Ux%v|6`s{QY;cWyv8*p3!|{#56Kz(A>XJlyvjoOZ(5E7#W{vtp}Gi#v6ReX zW`9u`F~RY#wEvZbAdxi@5xDSzJJJ-~zrXrd8Hi*^fGa89bW>FSedYh&^Z#CND68=m zOC5LbDs<`syt8?oN;IlC9E#_Ymg0 zJ5#>={7a}5)^`5ta!2~{ybpPGj1m_TELuQh&&ntJ^nFOxilFqA7&{~n4WofskE$V< z0BJ>88|tj(%=b38Q!but{@ddmlVLT5@fW*gzakx= zGP+N^d%PU3MRHhMOD?f$B5>Xq55f>XnNX?J`6d>PBL-N9J4?`l1^QE>faE&Go(On7 zMYcQB1)h+fB{vd}7x{C+-<}BBELOgb^Ex2ECf2yE`{MB%#;F?Mu(-aLeR+l9srZdNzGF({5TOYD=`Nr#_H2H?0Ki>R zZrxjX{CgUeV%~zzHsDjje*OI56lO0E`JVF%SZFuLeNmks>4e(-u@z|hyShz~DX1>P=Y<&Dt?n$3D9 z;g#!>vOf}E{nPAbv)3JgEiH*&hxZOj?7OJNL>l#cGpQr6NY50Ac8{llw7FHj?leU? z$N&Waq7@KAH-+%Pb^9R>iX5eUCQ11~Ky|-Ho1Bm3QbCOpH-`wf+v)B(*2gqOz!dRe z3#Zr5OH%H5no!MMp0Igez zeyM~;$uvG6t}x6Io8`tj1C@3S#q|1Sjb;PXD*us;S6*_dgz#WdIv7m*;7*tb+80m)lHkR`Ur3F7JB~@6Ami5dWfZXREs(IaR2$bp{Ze zsIv%!)7foy1@R-1+FVb*Cw)xUc6*At0Z6gY)khOG!rWN`*MyAWPtk36r;~$(lm)w^ zY1IbCBSB8ZqDJbF&c_Q5czi|}MIZRAL1Bw;ePrt#lW%+3ejM_vTn#d_)U*k25$Dsv zsv6hW#lspgG0FKvCi1p%lp&R9#tx4so6f3K3X^V;to5PY z?=8S0QNe7b2oib$HupIan?bR~(ZpQftuukx#}kj$9mkMlI|^Wx%jhRn6|2b}O%;FZ zTb%o;sbzeBNH~8U_zsIjhxnMo9}a_VL&{N-6ckJauHiBW*LDH3xtaw?9S^s=0=tAX z7Bd2IY^tx>v9A|=_Xz0cKvz+E^Yx)>nG%>tgR0PO%4f5LsZ<-qQ^^-&OJi~4djYh} zZ&lOvgh4PSZ@9J;0zMeHA?~GR%H_H0VJ3_(FOR7tkZRa-yp$io6*m;2 zm%3gowENcxfLLQWJDNi3O6OPmvjQY(Ea$tdd>i5hspOv$5%r@{5Em&PDC?AHu6I{n@qweoAWozcA| zGW-~*D0Znr>BNG-kci?B5xl{f?^q7!^08sYm7cu?7ENR_nZHqDn_9TwAXF5+Fa|9j z49O9n`s`PuN`0l4_PF{z*Z&ik;8{Qd9iT3M^Ub{e;A{%e%@RR3|612?pVYc~%h{DQ z0x$)qKt^8EYl%Y5iSX$R1#YYb*y4K^s7}{gyS`A_c5|3z_&QPU-|5_TU^Z_zpB|6S z!CD=zlIrqSkldP(?c2bA5rH641MJBeyK!m_gY3$iGDZ*?C_SOhY@+qPQJ{}Fve3AxZj+^ED~JAX7s%WiaS9h;;R<^oxtoV_>xR>OX?*%wwW z=a2dNoV{!8D?rxU^je$g?d`q)MXLeQ5Jn)Lz`I`r=2UrK z&RCS`-5$-y&=$6il*Ch_9ll&`5C?*})a_P)#&Dmn%o}ZNUd0~Zh_zL7w= z94=(Vb8f#FiNdoWV^C|weZUep!6|z{`7Zc$M@V#bCj`}=wYf*A+vb`{bezBXF)jV< zGL!G+lc1;rut$VE0h5PAIK2jYT@CvI3*HkV(2RzYQ)gxBwjpWHh+{lFsmewoKigrj zNkU&LY21a0TIWbi)Ds_7VC4e2f8Z(Ex9bEZVup?Q(NSnxXlBuFLGn_$@>3U_pP-jQ?Y`!`{B^cl$# zQ3eX&2oi}7{bpzXWIlJ+b>?jT;XOeDonl7caMDL%O1X?Ya_Qun?hV2-$Vh=v487gb_;6Uj_y4?4)1rEj9F!zsq`F42hre39-b!Hr@q`@?3 z`@lQM3V0T@y1sDf_rV*Drp;E|^#s_yIyE5M*J()7gWM7RbU*jwN?ACeFeShN53e|i z7I5*FxeZmI(B#2_RJYwf-XMq0EMggor^LpW>k)%!`$&Xi7I@93|L2W;Pvnp3R!-k5 z>TssVo6-4HXF9smw%Fu`jsLOlgTv_3jSku&pj!IV(pzJ8hUaB>)Q=k-tfyYDgvwy5 z%j61rDEm!0E2=;?d$F)pu@TEtCZk2OER>!d(!XZt7r08!b3QCOW!)aRyXaH3 zB8MQ64hf1bCvvwfe(<)xJ7KBxbZ7R+i}*to?X(fO`ApT(o+%DNcoZJ&Sa-La$rV<` zev+fd<56SwO9N*(J_CK5_sdR4CPX`7z=i_gilY;N1wf1t8LMHz@Zdes0>LOaThtVM zew*N4YVW3@le8MF1>Mfp@_#9Js+Yn@&NO~jZi=*A?j-do;45Zz=yjvt(=GI_5jHnp zoN~aQ(a4z*i;Nh7eDMKFM_sH}MfT^&kgVCU&;QG_0iJAxVcD&vYDH;~W z`KH+M`wf_0BM23wMexQQX z!Gl;nYKXk zRc|*8E^@u^TlQcw>??R)0LfNN8g{#UO-P=g3<>(3*>=w@oPGjH%cF1UKtvKQh8 zBeSKLl(!FTLWNGg$2~Z;;hzN=r)0o` z=YH{Dybr%!=-NA%D&R#oM0n3+4KkF0c^G_3uIfJbE10dUDn`5Ic02sJ%`g+eI2ktV zfN}`Ao(>C;(2uuR#X3db0sPr1(e@`Z_U}#RIVnYhy z2T)mlh1Wn`=vQkzn@etWvpl+-&+4es$-hwkK3xwh_-UnYj-QLlEHfdp3FY_a>HvYk zbHD`=vV{)AK#hO^S7*Y*qu=5FJYJk8Q3w(Y0-U&i-oEZ|+bPzFdVL2a=H^{ldLX8E zF}zVSF;%_wgVgKBVrxTLg}zhBlge|3hgg6X8ba9#CAaCxyK$i=TyC<9?ZGBVqk(aD z*kk%Enp`k%+t?6Y<`xXd6)~cM`=yi6)X(R!cbm|->!cBc3|X>rQSlnjLB?Nmj=SN< zm%3@Nzaa!bSfcV6x7J?=;JNYAFdnaY`Kwhf3f%v0r~O@6@XEzjs&GLPq9aLf0iY>F z7$~4KZ$$MkpAxIVTqv7$GT5w;{3Lzu#@3%-UWKUyKB~fj6Nd`8?;W8)NYMbc$xPrq z6pSK9GCOln!0k~a?`|?5uCQzUTv$;{q5DD+8!uJ~#M0?U(f$nt2_b@^;s_ynmS_Or zkPtjrvs#Fyym{Hj(76}sj=vCjZV(PssJ+Sc@tCZX);|E7C=lfD+L;u!F(99RVLBW9 zkk+F&Z}7=1|NiE0fRh9yvQ-uz= z3I(DB>cv{oO9`}Qa(L{{Uu7>Z&?zPXBWLrc#*cI7gRz9uZrhYoYd%j0XI)tFRh7215pPk z^Z-_Z3to9?ZhyfxaIoTWr3~!H{@~=MvIr^*xx!&3PRXZ89{8VZ1uC=~oI&FLQK{dn z0f50E$X{GF&RXjBydUVKQ~-*KRJhjZ6DF6zV}Upr@iw1Ws?^@>P~x}~G1IIl{0Qvz zK$-Eh^@iJ~cubD0x~=vDiNAZYcBkd96r=9S7y#Mg{BQXK_L*PMIgJ9sNyT6D#AMTy zdxK&3H~Yk|e;=S1*+dcWLN{2>_Tu?n?<;E6II<{!GjxfgN@ww;jy5+sA2?=_k^yHf^#r&qvWDLT1iQfk>`56p>Q^KM_UKnR zcIQv{mCH41-YcE(pHup9+wvIZ0p5D>Go^raKyf~tu}I^$YZUgn+Ivel_qy6ebUB-h`~#q9Rk$Kd70K^?#RAi; z{hSN&bo(_~{e7?0@5%f@nQ3B1v8er-0tew3e%E6TL_(&Bv`F(}ON@wqf2n}XFwAxB zz1RA#^?SO^eoYT)2!zw5q+<3z!wh1&)>km8)GmDV7q4;Jv~Bb{B=@;hT zpz?urH`Xn;px*4RQY+`)6)J|fBJ;Qc#-7X6?v59Liu2LvbfJ`TwLDpw``Zgm7ksCN zr*7AM2IS@H$5v&h9YHAS)vh877W&QZqOW7fmm8f>tZ6CNfZ*904eI{)XIrFMkm$1O z?JluvkV7$vUwNEvEPws^!qBAlYNBG9PrX{oW%gm$j@z`4`{wo!aUU20^^{onbFY%u zE7d$%Q?jX=3ij2o!y-poCwv)Vu6>+%EAe*H0Q+^??TLP^v#Z~DUl)M^U*dKF?gfHJ zo8mW(5*}ZF2l506{_s2}p_NOFpRKo&+@Gz>pZ9vm0vDS@thS{Pc-f}Cqh*B0W52!n zC;?pf6&W9J`Yn_?x8!zPK)3QIG|vq{hqQV~{u`>&{o-6Smd+^CE*s1&`OpZ%MmD;w_57{uimR{nR{VH{=t@cV?7 z%9;6y#rh}OXJT)K%?3W(i&H8?koXL9RBq#EkHAAQ)RS0sm|)-W>`zxlbXEDCpavw7 za9e-t+0+hpxSF4H8wVUr`uJJ6L*rN7_h0$k5^*nzlk>$Xd*tr^Qn|U$r90A=Qex-h zC>Oht=Y+BdH5eoodsJ=yv66{pP8g$LY;{mNYk#XXS?8H_2^3tSlEQKXaDX zrJ_L2>~V@;;k@%VVbyhicEE*P*gd>FiCL#-YOcG@~m8)LAu}c> zCP7RYrO)&1s?vuu%!-(cjPi^Bo9On3E@|e=gT;Cyu-)*QX4ki!t4oN2tAi3Fndsp? zqwYcR4jusWTlalZd}rzmN^_<2Zn4O#EMxt)D?1?~y&`^ld9OZ-YVsm;Y=x+*vF8B3 z$B}&;gBxb9+gy7v8MOJPyK!UNa)0k(4fRt8JKh?HF5O0--8xXq4&a!qKPj9z>gmGb zYV@EECq@j5{2k)KhXK723NKgey2ub<9>9%9XXV2MrX!qvJ$ETW_1k)HOM$pz^$c>} zG5={uiE%>2rvL1wcJW}PXsp0v>Zz+-YUH6URG3)V8%#@X&@N=K?8_rX^skV#$WILB z|Hq4IH=BzC?veqDnWB>HskXBvv}89<6hL7n(LhHc-QtDzA zeJ@i5>)m#BAgX`|mFekc`o%}{&4YPzJffOG^X1OWvHROgi|A40Hjh&Rz%%qs(8)Hw z@o#5chnLyqNm33Bq7O!HC%`rgDuBrc=uclq5afqpk#o#iE{DamD*3!DQsr`%f+dp?n+Ite%^jt6_M#+(%h50u4pExu(~tI{Q1jzjA1~PiMj3~d zpV>BZzIj4F?#7=MtC#)x{kHW;FDu9gn?VYLO_OObO@PaA?y-wMDnxLYFi&m-%wYea zuY`(ma8&0rx2B5?DgVcdNBOx@)eyKkHP;0!vTP@ed2*Xq%Y`#e8>&UIwB{k4AD$T3 z`u51{nV1lXSasSWxuQJ5+>$1ntzTb~O(<52FWhKC0Jz37H~YKECJc7pC(A-lk3Jv> zdapqeG|6ZnH~nR!LJ^r+@CG|eltfxV2-9bf=>G9ebWdtsz^ZIwK48-U?$5O9H1%?Waqzkzv5avbg<%w?|tIp+8=7vN_wjgp^ zU%k1JEMjNTuSuRPF^~qod}9+U2L5@c%~4mI$}h+rg&o(M&|M01G`VYf=b>5+KWQTUe-TOc=QTX zg;N%2;q-`~rq-F?R>f}&7`|z{ZwN{^j8DqvkFwl>yv7={+Qe4ur{5(U z(TZ%VF;dVz|NS#1HW89UQ2D^DPL8LBt9FI;4B9!zxb7qk;7pcY2T@5%A9+1qE}B6W zb&rpK*eT>FUF?fN)!H5dG>ABCJC}da+S$R_4f@JWvGk!Cl^%JsUvJ#sOH%6ZvlE@i zMq#1NC8r)a)R*e=)cllcWdCI03v-@{Ne}h zw9Vf?p~g8L{a6V2*_)qbWupXsFY1Ly`^ip+A_719SEuKbONE=ZU6;Dzt3Sx+fP&58 z)|Wu_Xo!TqIjz;{V%Ple=W_u4)zPQ?B1M~{D}huVFU&`5%-VOQuDN5&;UOLCmf24i z)kepczTWI0I@Df|W;{BRkPh6uAm)mK{PuV+Mfr*3@7;%1a%i?sed{=|x|7;2_I%b! zqB1$t^A)aQDn*7dtZ@2r5UvYR-QMvGKNqtMX8GCVc3Ac4bG40kgD&?h)zVF---D@- zu@!PCULn54y)05r^zM}1QLglyCQ+=OlCg3+lalZGeej@~H=Xx1j`0LoXRQ3Po2UnxSjYi{P18y=`?eX_VNSSF`E#@Na_~Rb_cFQ+>>SYfr`% zC{17nV-?gsA~X^|5}%R`nf~GaMz#6nDbOU6SaVblAE8AUp6W)^|Aq^m^1?j??B;3q z?~WHrbVpnf(_bg?e)}nY^je+7G7rzIV@|uKeC5ny3Q_Bd(Ww%P3~>qM0A(pSyvK0PYb=a- z_Nw67nBpe)fFj8g2m)0U^!?`U=N`QsJfuz`1oC-f$e~fN^KCW)&Szc8H}x{$6FaLTbFX+K1UJY)ty0QG5fBY8)*%0h6-PT1( zVe5qIBtrvMKcqE8rR5z5853mqB<#<}v$V-HU%9%&b(u|4dLyPKhRHARNh}P^z{2qNm1)cXKLL%a`_S+sdK3^<|P{34<-k4k4K*?r~_PJY5 zjUwG3)x6)(lLVw;gC}^I4^qrDUmzEQOJ6C(ncM4?0sLDC`|v{ zq4Jnd2m2&ow>&tgZM}FJO^zJ+s6&*zR)vJ`DJj2GM4E$R2gS3>hwPQRIo((%lBUp-=U{=D&;`KgMS+x9~c${)N#=EJ`OB!y?x(oxIa)n*D< z>b~ziuUBa6e^^fR!PFDpbtCCv{F7Hg`bE9W3b(E{Th~r=qJ7K;P-DqZ%z|yM4=VSb zE?dRrt-0Ej??yl!#MUg&W}L+>Czx~g8u8{bFJFOVMmi+Liur5}bRM@B8OBgDzw1D` zYT$I|3BWH#Mt0OEx1IChgRwBcNqx`>t0&u->0EM3NxrV!8X{nC4rn zNsm|jsgV>^CjEaNJuN2k(5SPtSVZJmp6e1A z8X97%;W@|+z4c*17G<*j{k7cxwaX4ct)-oUohv5*gB11htv%$xG44;SeKH9Ri>1cz zZ-V%?V~sA)tw(RJuCxIy(QQo3X}anK}AG z?7H8lN=&+@ye{`QTS2pWP@-ablWx1;RdyScn6j4L=-PMIS?>3!sX&={PCmgA_nXpq z-A3nK;~BuuB#AhlX-CXvFY&SgCw*-CQOI+TKdB9{E^@$^W!~xzR{N&HjpI>e9>T z;0COv0M~DsLQWgZo0sK7cdIcik?=&ExV}h;Zi% z^xGaNGGAuL-lEc60iE*dN1fH;$5a3EKA&@*(>5HnRS&2W#AFt1GaRmAnJ13vGvS&A zjxjJ3s6oQ$ahW>N>+(>K9n)Q%_GPl^^S9|8eND#&bW4+#iwp*jh<@a`AlEG(_l|M4 z-cHN{KLE(ZBdN8!Acg!NO5$bc*l)6EFrkl*=W4jdLXE+;xdHRI#~;i;SB+;-i4^DE z1RHr(&vFjy&NPEGz>mP7u3tGX%?3%mJ-5HrW5xotdIz$%r5QixKOd65nSPl1V^&$~ z>gsHp>umn&qw`vM&e;Ip`PRT3UoM^gpIwlXrRDjOC$dw3x`SMx_^3e4`5_Al71#a< z@H^^EP97lnh~XWj;mGqb|1Q&4cD~6ayG?yYw{!!fUTeS}{JYxab07ln(s~;{;S|+u z@&i@AwGS24sGi?%7|*1&UrODk^4Q1&qnU=*+TPn}HoO1sq_|v0L7Ty&>iwjr=jFB% z*vsa8GE6F1^j*wWrxtP&xk70k&rR20%y}KvQ=BVHX>^Aj<68!hoOm89&7H+^ELts} zm<6As@*!aWD^;mopBSEu>}96jjdn*Q73%-sY>xO0h!;&ZkV3lY4@SMukZwHNzVe`v zJ>%$TUFk1pjICGg-}_r6vi|E!di=2D){t6(V)xJ`n9r&|Pa%tp$Zt1$hi8~0N=u~D z+;ALU8gSZp@1?|9tl|5(m7L-^bZ>*g6Pv(q9<6Drpy=ELNf7P;h2YzQBtOoM4HV^3TAf@rc% zDfVdc2z9C6q7m=bqUPP~2qpsAD<)c_8rz9jG{3juJt%c6(6uh4pA5`d5H>b8*6o9q zrT_+&DhPtOBpff=6|qKiO$D5I2N8Kqs(!ygjX92PwI?$D!E`p6_~mm6iTW}F2E{b~ z8q@wE7>5~*DolWyB^#@)AQgr|{tIu%+R`!7Nx_o-3*hLT7(p9x=z-Yo@D)|!m} zS{^SnM&xbQ5F*l_s>+Kf=}AbAmkhzd(=&qai>2ja(Xs&2Zub;$|CgyB3{>%)l#4Y7 zY{pp;ceu)%wS+!Vh>314`YkEx)>`spt{UE6*!*bLK*Lk`7o{^?X5gir{z9Yg8Ip^F zQMpDe0h>;r&0>vlDQ=+Xve$fr1((ujt$mfGwo^(5ao z4>Omk!nXt4Ubvsdxzy!Ur0ZGJ&unH$?OtZ%p1qzax3=bziYAg`N#(1RDb`TiC(OlV zkiW0~Qt^@o!5|pFHFO|1Wu9g{cs|?L$5Ulh+8Dv;MY)E=%0*~h@L zlY%N*I&$mL#qfwR=wW%v(63;j#Z!4z>M5X+XwUYiN03Z3 z#+T28eq2%rzt<}c3d2+pF`a_;9qe{T5F&MKtQC!E>eWn@KhBZW*M{ys-(mj z_}vajjq~b)aeq}fVhLfCoM90Q`@6lNWQLl<n#+ zIjQHhV#s;YC@F!+859MCr5w*%MoTT)Y#zOm%Z0cz)`sXSE#pwHXzOR1SB(xiq3Tud z9iCqax$amx?f1P_4-!!;yVON!nrM2i9zX@AeLElD5uZo@+(_r!<1|`EX!?e&8cB1Q zjPq0giy7fDsv&@d3h6ZOO-dSL{9|8!4@^XK+FwL|f~g!|s8SMS19UJ3g;Z|LnWbg( zt{#Qp3C#VO6&OGmK=RDEaJpKW?*>|m`-G5I?fVm=tN5y~8Uxz_oEXR;Pear2J;4zf zz5ueP+62MIrsR)zZ=3V~NsdD>?nSQtsX+Zn%LEMt%sWZF=)&lHMHu8HmFK#adAx-l zq($A*7>Vz3cQ%+&eIZ_(_a1fbV@P!kJZR}9>!~__rFKc&!(Z)GC~;Ti!ljS~211V^ zB#`gLBaNHww4Q#SweXjOEcq{fq4rY=`J~an=uH?j$bZe|$tk_+gs6_tRzRsUj8+2azQtbR0pum7l(r1sCYAL@&ML{ZsVm(KDs} zGAfJtp6Pn0GihOlT}%0uixPQNptr9v?Eo569?%=coU?an%{k;nB9!`Syp zyXl`kn~4L@wL>7C(Al$;QgkZ8I2He4A*%D;iFIG?f)VK!D^8ET%aRuaY_|H08sCxj z|Ni*u2KQm$Dj@EOdOg$O$FkGJDjxp_0oSSz1Ibb&PlfaaqeSzy7>y~LdEz5R;2(x`?5Gw;x)SE2TG&h*To=VP7)W|bQ=C=>c+?ALyXZyMFrxM(ZD*L)b>#0 zFf<=JSJJ(gFWO8lE*Ymf!l1#pM7T8JL&XB@zFmET4yKv@GWB!~KZB?^G{3Faxg@g&*xoekw){(&|Z3WVK9E>$O2osisN|y3a|*o@_MR+(w^$Q7uj8Ulw}H z5rp%iwS`@FyJbNm5dG`}hjPA5oFIF*9Jx5?g&Jb7wYtZ5p`g0q=@l>__( zEY5c{WISHvJie5iU~6Si&`xj5LQ3(cv%%gmes4{q3BFdRbjulyw3aT#fMCeT( z4XR$D5+Gy+2ID&t^#E1(N!Yq_AoI}ypj;KwIX{93?!c4dzFk7`qni?CaaXm7ljjUg zUyr14Ldp?4$(j;T5i=m;x?v6@b?rikTe z1C@}czX2G6_>d%b+$B_N^sPC&0QASNYn=d8Ya8g91Zh2Hq|SAhua zYmK?>2Nm@^5>p!M)m=94ciq;fJuc(9I&!;{zS%=)qw&A-B6ZgELBa}&l6y}bCZN;o2g|6Kep`1(+A`B z{g-m|OC3kB365F_O5f(~a*;<}I$!tUlL%Vpx)9|C^+4!W|0NTy`0*i^X;_ zsV%TB6#yH?Nwv-F8$t1SGGdN-yc^hO5fBNwSSua<3~^J!JD>=(cFzg#oKne2j#+d! z6jy%z^V>@2>xgTVTRQi~Ku_{FLx|wL(`EU`v)37x$c#qjubdZ4pW+WY^4aM>jdD6% zl=NQ1y+FGvEXfyb@jAgIDx*FT%0JYKriy?*o3IelqH(Ea*>5_}J4l~xysI@kWSTO+ zi{Iw9=``oFIuR2f+Wr{ZqsDmnyhmJRvy-JugaTtMunIJ{zb+A;vE$hF{44TK4u>pc z(&?3N%{d?dWMc%QC&+S})@41#DH$N{E(`JpzCQ#8!$y9)Id;Ppw-nLIuph|hXnu}5 z6ydz-?M)8ujwXu`OGTD6-YT9GLiC?gT<1%lu+A#BRa#;>ViV>5KvS|Ne$Q?&Tl4Te zo#+&4UAS=HpMyi)w4(j9bg5?7o4JLq)Z+cFHAMF<)cV0q%S==@J8ycFoo2c@~ zX^|03jozfd1Ir|D&gJ1p$Yy$!ka`GSLEy-TF!)Od!g9)PtNFwE#WZ7V0BuDHdH-~8nE z#0j27c`4nW{dVoQTJ3G(Xs8=D;p`#NEakIS9R4hVDz1FF{);A{#ulpOprOP4lXlb8 z{oFeitT^3D%{vMIA?S;;GHQBTK`?U7>qAzexsbXiuM0U6X*UC*M%c-!^n@3CRv`x9eSWBCN3M)Mc{o3U@f1y%%Da z!78cn_v&J7ApS9HM$+;)orI^{^v6|_{fBJ9lj5vco%ym>`m#xzNk8iH`-4Q*A?wJ_ zTgZ9-=d6GJ%uSq0?fT-yy~yh`sV!XCx#_55AGRe=$k6**WEU#J{H2jYS6-vx_yY5U z659dWT$i6^nJ^I&R8tLlcUiJ6k?<3@cY-25NQi4UCS}zj3o#{cb&E9sKPTg1lW?eeaN zj8&Hyf>Z*HQ*p+TUyLIP?2$j>U>j56ih1+d8Z6-PwWA@*tb2q6gi$Jl*T5uc3yPM% z|2y4@%?^}jq{uJ1{}l}V2fX)!1tv+)O0)5?{x3-yDbgQ;0dGR{@_)+cNC9smz$d<3 zjPytScc>Z`6NRT<8aezA>UiMWpwK#xeH0Y?450W_l-^{zzW{0oJ%at!ds_B)AoSHM zn4BFP0xnI}P-hGM&c53|o(E26W#lJ@i)LpR^K(Y`g1ew!A>EguFYjx@t3W9KYs#7{ z!L&|%qiVwz&(~Kks)6N~3JIn$Yy-Ig7}gZCz}otI$>~v)Rw?mM&iakkG?2xwZ&|NS zet<8X`A7EKl|Ub#0FHaKvGKV|sgw=Kyd-a*7L~1hdd6ZwLKqK?=$441SIQ&(aia3y zI__)5H0}^I!k5N3O-`F-7Uc1du%wPuX=B-Q%l9SmP_>ofJ}SW zN(8b@(zUj888F@ZV4=lK0c}A-d+I<$!1~X$L)vDH*oE%c()+^&A5!1lNtUnnW`rOC z#%UAhbJO-HDoG~jx?ENNc=Oe#=A0z%Yd}z_qhYL|jTFpsP3QL*C9!i2e1O`!8@~Q` z7r^3B!y#QV?H6`&%3%l+Js@(D*bK8D5T9_^0j@n|@&0Yg?Z_&<62klk6R9Jq1#jdA za+3z8O|zklRo9uA8Kuk{!S}uVEyi7RyLs0ey4H`#)4!~~%(rVj4C9QE^7H$P4G`ySJU<_aZ2pu=={EH0!oiauaL=dCERWY-UKAk4=$g9kY;Ph&7$3qOvwOCD6!7<_Nrt#{-S}01t^xbMofe?KMbUX z09ir&cPw3=)l~UNAaX>JdQsKc?nUPIYK$1Bs#P0EFuJj*=Gw1i`lCr@20dgMX1;yK z=Id<*M#eh-y>4>4WF1Ho3A3Ci5#Jsw$dL?-9=<}2bN>tUmUNzS(Jz$-)KtBYFn^L{ z5viwouJ%pLTBq-aQl>_~D+rC9ggvFO*q*b4>#en(m>ey3GyLidbdbvN?*>CvATpEK zr}14cnoP_%%J1xoBdqknA&LisUKXwrDB6ABG?7b}0bg1D?5d<9PbKcdB)b{?uS_a6 zUb%un1BJN4QiTXzvh@ZK39vWdvzzBsUFiXdf)z_HMgpPe9JtOQ-EU!s45f=A6Lj`lsH_w2B!!D&aG}4=O z0CE|6U4WYXWo(%+PVzzc`1rVzVS*O3v*m2hbgqDStnUX54$xm$F_|gbc&Xd`>@-_- zo##V+;YY!?OQ{Ua;u~RK6KmulD#iUW2hrN<9$|>+IN0JpcO<))R8%X+xC_YsjFn zSnT{ssfKB*`aS^mb$3yG`U`ZKWC+DHH#ajp7F=(&XQEiF_9#p?qzkz^5NV!7=F%N{ zr?Ax!TV>j6bpERFYP?h};=6j$dwB^M?D2&Nh)yHpe(fef?uYfvRP=SpUN(2Q(3 zCCZn;tP#lWs#&h{73M_}4Rr)v!SvQ%%Ym`_aoZJe6;G5cTd{t3*>$b?urIiPTBwpk zGN97r4%64mdN!`sDl>J*4@A89vDC2*X!kOVtl927sK%H%>%Mlr2*800fKag$WlEk~ zYFwXOk_lZy$W{T*=|CULZY=S=M6eRzk8!wr{J#A|da!JDyhu}NdzJ>=N>4=p_E{U! z@v4)@&Fmh;XWc^VPgZA$Sb@HPQ!4EK3dMGhfuq|7rEahS6}&wHfa-SdAlT^EJGMhC zs4&b)xGUO3YXv6DG=f4t1ENyTV?G&HfOF&_6 zPBG5K-bhpo?I*$<5ZMvPecdjn!B^-JF*#o5L(iyICoDd}K`HT7h>7P7*b z+OY2^?s~p^*@S@IOijFPxV{g+el1;yC(-w7S@K?v&H?Odj>rR4Y2}WP+LRXIg;Uy* zI#UKlzfXwiI_Nc8!!mjco~26cG88#<HP#oW7uzOUXqd7*l?^~h8s&m zXawUUXKjx@eVZU4tmeS8jbgkKW6u1uuM!UzNeD4Y`n`KTq;Pg6Vi*nn1~rD>ZnVDPcx*lV$bI5O-f==Jv*L0AK%oFrpUZK z3rQWPmrEQcDOZYREXUM-Ia6zewXKMjj$5b1f$AUy+^`R|pDInHRBFse(rDxC!L?Z- z3CH9G%i?*P)iuHKNHmxRw=ZgVJomg=ds%$#Kb6G%)<`RL(X>spC1GIn!vU6a;Lk%X zs9LVP;-Gr!XU$@k9dB`wcaGoOp4EIuB47tlMR+Kj^?lZj(EB5}+V;dwPp{tbU{bLb zres*hRFHvx9V%29;xL`394+tuT#6INvwie#&3_Oey+B4-whUXI9GGk7`PiZP7}+y$^U=5pzUx*?FV4eNJiv^jGlfHtB^9AU>k^csp_Rlh$Y~3u> z(75DbsM*XL``RaqyY{t=G+?K`0f9yhr^v!#kYqn%J%3V&mn9;UM2S`9NI8z%e}+*+wNfdrQh z_dMPhSMwqgN%Fn3?iK`eESf}C3G_1liu%fD(Vj&$c_f3^TU|hdlm=3HLe*w{kAi~F ztK^-bpO)=l&=d>7^JhYL6w z2r9xdGW;H6iK^#AU+qgCi`mXu>!WE9?|*sVg-=t6IRIKYbn9)EuGpy|fjynqP7JNM zf3Zlf;<}Cu@TuopeC9v=eFQZx7K3}l!5@8}ue+cIa}YJboAwJvA8RGJaq`oRMEDds zHLgg{vob%+lE?0`!N#afVA2qsdaScv$g(bt{WV<%|HUL2@O!hamrjIh0dv-irGSqd zpIk2&`4`SqV1Xy)mE}4Wq3j!9WL~Lyrjs4Gmq3-3RuSSK@IIBh$b&O`EQfS`Gv!@x zbZO9?$h%R*dqt6sBfiR4Xb8rjx1w_@AOO8Ms3PSJC}?_Wm&4rPR;#_B(^B1_CwgU| z1pQ2Cz(RmufoQpC)-adUFh?N62AQVDR`rhJIO&P{8qaDVSjSdjhf>z@y#DZSHs7bs ztWWRjAm)ifq8RH-k}>7+eAJOs4>+UW3_#=p3;%r%8i#r|yQ0;Q{6j(zKk*!*#G92a zCIpoDqcjc!-5Y4psn`ES?v8$YX#J8ag&Qvd4hZ4PHQ5u&f{v`ngrNOCzEiN5ytay4 zx#0d)Aq$T~8c;Dfg+uE5LqwbsWcOMz3AFQ+#VIAIu&og!wWU#|`UM=XXSkpPa61?< zVd9^CP5=JGD8gUG2%^P{QCGkGFdi>Qu?2p*t>Wng9wtn=Lq9KS&Ed843<@W9u zn7z0Zxw$-_gC^i}NpUkKiJ(3$`J8Z45JgeQNfY<)=Sc|(7DXUGk9~GiPt5|JRY2AY zIOFVTH!lPhhUqD+4r8U5cQ3f75vl*f=Rt|2C`U;|!j{cmC;$c!17TXHy-Z@P)55~K zb}xRq{}55?fOoj}lJ@-VCEviJTY?+YWc~f0`4==T|Mj5oNIyfVFocmT0x7W*vHeMK z85n@@BsmO77y~+3p!`44Mu--~eLPfL6bY{^qlH@&oBcI(j(ICRBU10)jb4J4`o+#EbsZ+G7lN;ke z6k0Khv%n@<)2HCK1joAuULJ`yToF%a*C|_9C4z_$Wx7$=bx#H#6P?jIl<|X^8IJ!y zfF4ZDxSuGU@}DeUG#_Y$RUC1-rkxAsDk-}GVPNZIoyk_i=VDJ_SA_WRa9ho=VEu2? zX4mDXGf-RXrS5m%1buXRJ-+`^`o@STT2BNZcFzzTC` zwuS!w_OMNGb~zmWp?=X#IZD?h#jV{cl}<7BU7ylYF=)8ic|S?rmn!VR=w~3R&?v87 z^2%bidK1g4ISm5R$WZf>TKdCVP$=Pi_wzR;{xzs2+5znwroR)_&hfWX9_{`%80*qq zd4jTwKSNtuq63e_AZXXD&=%-x0=NRwr1LoazlUl37IRW(lCV;ks&z)?eZg~u-B_qN zB{LPa?azhNO}2j|F@0ME&%h5aXuqUnIbSCt;(ZnlR$0=WWM6}9;bpnem#Q>w#VnT- zWj|rS_*4O5cf@1n?r#7(x7VbBOvu;~P7cE@eWtxA*uCU$V{M0$P^x<4^XlYpOotx#Na_N>{Up&i@u}RzgU@VOeWB*2Go%VOla#Qge z5L(j(Z!jm}2M;eW;LUP0a#fT~|L-?VHWna@OFSD3i6CTI9jo5 z3wlSAds{*N+y83zm)0-fdyp|6yAd08J?{D zr1)fxDqBTiIzsE)8h6x!CXd}jo`r{3rLwy1L7<4BHl~Ua^BjYedz>Upw7A_Yl7sYWQRA7y?X}@62Z(eZK-JTeue3}do37!mt5YyczZ!n!TVJB zqr)@Pdh?kc&{d{0h?0R&PK9aGrutaTz(d&|nNrP6E0?rh#ZFEE;vKq)$7?{A5O@S2 zT1f}E6Kz(21ddKuTDpJv2$qp8{OA;q=pcMZk09j1An0LRyu!1fTPn;5i zs={!<@NmA(P6geP0H%58zRIP$ao)YK=}Q?KLXRkGQronR7f_@7uF^tcGu`kpv0VGN z%CsXhXaFlx3|Ivmj|?Dd^T7t#zPVgi17KmF)=Jfh1l=r^Km>=LmS%Xq$(P&&B)htb ziNF>zhtpJc3~(gW6Dip0*--O-QI|40GWAkj{MlMBEITy`lqTtN(KCr{c0$IrElu3w zj4bfp9R`ry)>@I8_x_$C$ZN*?nqruZE;rw~6#Gnav<5n0WZ ziPJ7e6x0DYB#>|z^=iyFKC3HW@H1!5&JRBWt~RmC|Tv3N*Cl!rJlD z8yAQ7_3B?{&;AHPvk3}S82{wU3?tlHyl$f#-3~#$+P}HC=q^@)o-=JrdgVj7wf>|e zw}W=G>JLMC;z3~}C=}jM&;<33+K(*H(Xt-r{-cP2x)3_$s{RY$v4Wyl(oRJgsPowt z#{mkA3yb^qpvzM{1_(9!do z_S(%>BvzR{27`kn6qXoXblb}X2W+PsiA?<5^IhNfi1b?w8?$xk;|K5wtQxVa=WPIU zRt4?7GlOt8U{&F&S*^Fp4^?7ISVFX1_*Ofp=(0Qh?RQ+-9Gx~^qdh`Gf$u$vV;_ha zs;tZ6Tl{U{XlB`waFuxhK@vhJJK`aM?^?F`wCQ=dBR=2gy5X89_cd`GE#2a|v6j&@ zg3rJD`y<8{S~ks{jC8}}pv*|}x5iIk82mbT3sWU%;zvGbD^t)%$a8f+-{JYn4#4Rb zFTnRbXA`sA_{e=g!Pt|W%cj%p-zzJ>#H3L+7~!5YEc#uRnG^|egCHi z71YCEi>vFIh4Bq4J0BjiWyEd#+6V|>GB#o`I+EWVBvu7eh1j;JECb(iX=1^U^NrQ# z>+l_su&^`G&$0nraJu$HBAi&9-ya(2YQkeO8ef!Fpj!)iieEWYGuwuAYvIL%pk%c; z0i9k&fd9hUG>30`vR;VHJ8?=PZ2ORM6_#M9;YSQ%QrhnnVvfP{ZPS(SaF9Mw8~dXK zBn3&xx&+ccP={ghusmVPq=X1a-CTB~A817yJF-H+2nzjWt7$@u&MQJK)HBJjs5Q#{ zw54w(O%eDkf4tRl&1Eivr)y+P3Yr39D8w%Gj@kC=S7JH=m(ppPx=-{|^&V+!g6Zh{ zVjx19il2O!H-hm6x8<2OdA(Mdw(RlB1HFB_|IK;v6fF9qYB?Hg@BqdGjDkZ^@Z-LC z{H}cvv_Vy-bM5A4Le6Ci}$E(Lds;(%qdCSO|(h&p+ z2H!hMbpbP1`2II8>A^u8XxaRtrQbFvt)%*=4{DZ&jN@!cHdnv;zBh)9Pwv2#Xq=gn z$Va8|89FL4*L&sSnzlxR==a!2fndt~^{iIW)4(*;?TdZ6e0|bI)g4S(vT>60r4B?< zrPyhg32LhkU8w3$mx8lvR+zYC)25Z4_PsSaRuPIj5R_tPGVDYOsEGD2_AWhKUH7V$ zh-|s))>x${Sm0y|gMX=(pKDlWo1e<_9nXu2NjLC8)l%vMQa0%`C1OIWPX6%MQX*Kx z%lz-?0jUtMxbkAKG6-@`dqy<~i3sQm{RDEDX4MUD>wOxUVGpi*3=09IJdXSaS918CJQaJ#Klbi*{+<*ZNWcRq*kC4Lqr&cb!&r@|K>|zRyE|i zVL7Ym@7R=~pc5rx`7sWj?XV0UIzr$WIR579Q!^BE`_7S2@I)!jVP$Q*DovjdZk(5jv-8yO? z4Ek)rv8Z6EG$@jpEH|8DjtB-dBWU2&)?IW4r_O|@ixzb%?5j+LFBEQlIg6hqJT`K& zylsrMLH$Cp(~}xYb1`>6B@_$kU8Cp}s=?etz4%0V_OvlZ!1oR*Y<+OQ&^A1LzE7&s zDam#mly&Y5x3UyWJc7pSfr1_b`Dx)mY?SBHAl@SU0=vwWC=h|2?jXy!OS&qz&3@z^ zFDI*RO}OssH?r#sb37Qtf@wmC-b%{_EMh2qZCuXr2`IEXh59A;>^5lSuNY%c`zUXs zx>(weNP1L`sH1L%MYkL#OIHi9P&*KPQ=`bfNm2nd%gc3tGDtE$43!t{^=2q4CPwpe zXRTiSU?{00BP19Lsp6aAH+POy;dL)n@?i)j=4|cZTFo)`sQA?d!h&BRg+wM%lTC73 z#hgaBo_*8~<~_R)-KqHi$%J*4_~-U|f_OfH$&Sz_vqI`WnopAL%_3d>Nyf{~;*?l{ zuk2s=t{Y9bhyv7x3q4-x%{+8y;#@$NN*Bj3RpR-$VMQR?SCcyux)A??W#8K5tmbEx z%F$=`pX@>Kcn4Tqk|noD)C<~_!9FX$R7|w9qz&xV109G4@u58q_?8N_<8F9H?fS1~ z*nvj$@dm}E(0ZoIG2__MTH4E=0j*lFt|b2|tdzl(`}5{!y=$Jr#&bF*xKI+uC1!>vi&dh3ee#_qH+ zm;)*sW}dkLzKb_w90C4BK}+IEoryRwSJ!{{fFP!V3q{WCw!O)_+%|@Y{i$_Xq7Ibz z|DeQhrS2jdT#`wV1BPEwevj~eO8!;ABvn-=S=NtS_>F~XK|Wp7^D2YrnNKv%pZM(b zr4{0xg3$2qHP6{Ok~uvP7Rx}J(?+S2C4!OCtmf}l8)ZKKrGRcYd4@=&b9#9?+v15o zK+skzh~7tCFnWSP@=hUCw8vc9H0m?hs0C6P4GHzkaYZKyHY@JWS~d$62$K#hQ*;_P zU?j|f(ase%L7K;15{8?KWCBbCh<^IjiG(=qabXXCUuU0Tjt> z@G2m+*1Y>|Dt5Gk`#=m+X`LeYm2`?hnh5$57&CYMU}@@y9(_btbdBPPHHJSt71SSn zJ2L)MR49Q%K2#e@iT1a}^E=i5C%WT@r>j)kts^k3(CiE5`d;V`O(zo^^V$A=pJYbi zcUQ^5I6Zg_QKsLc&+0QuyECdq!t=}75Ak~%@P(F6NNS>#UMcgt1__01k$QRLT<${x zwZj*}00x`M(*eB?>gy)T*R!p{N-xk?B^vlbY~qg7`6l5X5Qe<|(KAKTS{Q8)I_FN2 zK`__C*l@G=6N50Xu@++ExWqr6c%e6${-yshW*~15lg;On-A)=ZW> z8>2clmCZbuNIdzBbVynRTN+?^5?cZ3_B zSsa{>%*FX!YhZLHECrSK!UC4zXM<@U3~zI>m+4`O^D<#TGRL=(w(`Z;m_s?)y>qHl zXQOKpQ-9@rdWKL(&kDzh80$E=bHqZL^TVkryw`HXfzVtDXwUAO^M>^g$N`kpODX|W zze>L#fEx$pI1U3)l5F^Wzj~J6e(llhAC!CAbNhty8ZX~6v`YrY#;9f4}{5_ z<2S}V#=NiVe+7Do1n$3CF@eq(3Ws!ck30}iKmm?WQCgl-zuFgiS`ac#=$x>xE~rIs zpGTI}JH>20Y#-L}7MF4w-^>fdtYbJkBOe{foUTs`+0tAwVmSLLQX_ROeSuu`_F|kf zJhn@;$suq(jvF|N8gaH<9!#5yA5MGQ_)bS$_&=2=Fdrc{2;MFXAid%D45fhz@WDqH z=pjOU2;)NYPNYG_c#}`s?}x90py2S*MCJq9pTc}1nEn|4U!7pM5V(3wJ*3e8Xvji= z1pk?RM?xuH|0ks7pT$ux??VpfuLbpgy_$S=+)#P@utnl9DKj72dSzj7bO)7B!6j9R z6`%b|!qg$r6C6nUJII!+yV3CkbLwK^uLwsC`0cr+lJWtaA0!2i_N}ibC4L16grL}z zbfaJxLe%-{`*I4(<9zhE5g_JIoitwlqH!T$Qwh?xyeqVY+0B*m$oJ4M-wBb_qWM&h z7I>OV#L)&S!gy3_Z7{Y&i|E}bT-PaLeiIM^k!Uw)*L6Gw4~+A#o-gfB4i_3Q7ArMc z&YKdedBi#kD8+}P=rQ_EcF*^Rd%K1-^Na{3V(Bp{Bx6z~5@^JL;=#5eu?xoWB{z%9 z3}FeR(&`-m{2Hg~+wJScpiys&gH4=Wx>5z3K&?J)rS|v7YIGn062Yf{&ZKufF>j0$ z-o(S|We`T*j9=H$C8tPH&No}~D3@!8AKwD%jwgYBeU;SbjpZUw!-8@%ct{vMvG7O>*Q|=5Ko!{dAuHo+dSiF z>^b(9^FDlOa~ra%27`^9`a^EJTTl*)M@^q8sqxtR!1DfE#~8$^{4++$xO}2TlS%KO zx~Mg%OOzt{+Ok_NK&18_9Z{}BaDQZ6;L=`H8$>ICwCBP`+l|o^Jz;7=zZ(x$kqC^j zlpRtcO5lW=%oSnf%QE;KH3lLcNAw8Q~LzKq!;aaUb<~yyC69WzBoSW%F%s%KB3z z-gJ@g>$`E9f)Efyojh7$I-eiITznAh1XKkz=d;U^$o*kaH~D^)7%vyIvA@(<>W7~hgkC{_A^NXQ{1J25{D&ei3` z)Q)c*N%O+8=Y5m5<39Ue&gS4>QD(ZyC;qi2Q@wvvRtDY$5Wx9O;{6pS`|AE^u)oWzhe00`fqZbh= zw=LZHaNg1kWG<~lf9^T^5_QeDd_I)gz>>jbpSZD37<$dU9L;6#nyQpHpYJv|Cu>e- zG2i-YvSCHQ3jPHfA2Tc(OhdWiExdxM0j4^|rn7|G_D^m9;)+nQ3}1Yl#OBuzKED39 zIDo2pCJt`$Ug?l(4o)DPgm+yq6ZV-AGfXSz6MlZMk5s4qv$53Td=PZEmSQA1y4-a> zLH(uqYL&a)ecRX}@lo#ci*TpV9hoRu$vCACJZhriD-Sc^?NB1#A8=Z0Zq!Lp2?t%o zliTX!eY9hXB>dnzI$(_9xPempmx#|@ah6FB%x3W%PS+_)E2Nxt`(HQNtf6i8C&b-d z9EpP#GRmowYzEDSA1ka0MC(bkkE|u4yc9C=Axm}MU~T1_v71eNnUb6>bd((88odI| zaTc`7MNSMvu&^LTESo8{2e<>pX{aWF!RKr+P0YT&`SK)0<|Y)z*E-ZE?_Ps1twb@& z>f(2|h*G`{=?k$&v*~lJyvYA8rJo+PO%Ih!G`}|zZxoct7k@3Oc96Uz;NI{SURnSd z`nkBkoSni_05D+0#r?w@Hph8{DC=8BkDHTtCOt?=DMur2(w{nwu_alMGU0W8RP9QqhD z*vGRKOKu7v-0b#Crdl05lLD;iQXxafjQ!UU8Idq7%0+(V(kYT>SBNJ|sEhoOboR7z z%y*wcI#1Idi@0Bx7?n5wb-!g{SH1pg&#n-X`Sko)=gac-Pv;0lft-z9DXA>52dNnS z4FAl2H)nY`f642H!njNoXJPJYfMK_AMF`AUu+>vRpMei05Ei{x!@)_IXm}cVOO6v; zv@eB~hu{5ZrQ=`AL5AcL#@j)8pWPUFHbaOozna3&y*EIWTA0pdKf1$gKx0t&{h?F7 z?QJi~WQ(zL>IY{m;H(SBVn`qHa_JO!i@YbBCx*BUJqiY`Y7ldH*-SRCE|Irnen%7K z>QKp!rq4nZyCFVvy{1#brI&tt;bT?=I314=XtXVIFk2qpDL>95&3BV-La=Nj9I+oI z>yJXY>4lkXsRR=bageu|dv(}q+Fywdv;=uxbC_Pz*+Gt)e&~GK2of-M^cc zuMG!Mr*>V?9 zc6$a4cSyD;aY=<^NtehRraowo#8G|t(&!SS8+^P}h&hzX71S(DggXhlaaIwGeXQONk);tw3 zSg#dBBN6HI=Ob=HNDQd32bbis%-E4LQ3$mT6E5(;-OW9j{fs{k!=x)z*I?GaNsC&l zG3ft_C5QxqMNm?De7NA|e;QFk%MsOXG=ZwhXfNAF7ND47L*54o~Q7lrE zPrAfgSoNRgT+Uh7i?YJz;ocnybsoat@a)zWjOnF@WCcgwFRFZv^&H!uRN9`^6GoIsr-aV*0nSZ)s;fNZ(KQC3fFK+{z&Sbw|e{P4NSk(sXqUy zCsNJl2+F~9<^&#;9NwGTwha#p8nVxdLvepG(e=V6+sus(I7sV}YbcZ5SK7?m;nFz7 zxxGB@w-iZLn8a_&$aR-54|Ct?D+%I@kH!aP4{KJbVqCm6>Pt`YH^HtqbU}D6q>%qv zu)Xveg75C}?yAsmZw=DEz#}sggYvs2z5qj8lf6-O8`+j{vA9BNO;5Q|gNdopPLUMi ziOo-q-)ft$Iwp*VZQN!%o;#Z;@Ic5uBn-p{GF?}lBH5%UbDNtJ)G~H)6n-xf@4+uV z3YkDqW%t*bU4F9|i(9wyXXwy2%N(5b03lgs;|wwp_s{FwmC|bzzp-fH;74hS@?U|t z6r#ew&t}MRrpNHT5f;Mv9f}Uk`8RTLbVZQ5Iq20Z|IyvCyO>-G1DSAE3@21AdyA7o za%<(=Z{F9^y1h_i4{X<~Rx!sP4X0MOxB00cb`P$(1WYIL_2~SL+XsF|TGL-6QM{{6 zO$!+orbgy*I>= z>;nU@m<1?e+mXcp*r)noOu9r6IrtEa0t%XjO>c|S2qGPLYYAZS2nQ1B*A2gvO%;oH zm+SQKCHJ~C&TZ4BnbL;`jRm}hMmnw12m0H|9$MTq4y8)%v?PWy%?6Ic<)$H=3`&`F zsW`8U(w4j~pIekTDrvbqlDVPoeqWRu6&TLuO1nv6* z!ir{-QMm@$SnK|L%utd%+-EcB3OvB9u=NXNlqY2P%`7+c_G{5zcWnDoP}Z`a*G@K3 zCrQPj>f>&H?1Z82NEAVQ9gY=2<&!&uj);ItdcuI(38R}pn_vRUi{n+LYZ(gNHm4*^ z%;aOkJs5dP>5eLmN3s+t4K<+7bs~s5Eht-3cOvBP=gfXWU_s00iRS#+pt7m_C>85Z zHXi~__r`R>rfKZvc)139N^u}7*B?0%t_||Y_F_6o)t>bP@wFO$=LIa>t9a<2nyn_j zQ~J7WKwa#%IOZ!e#Mf`35mzfA4rt-JsqFwP%3@7=JOhhpwqn&Jl}c^NCAW+9?QuXC z!^Ab@jLxO0f$K~0a($TtAxyH_4&)^KIr>qww5pza!d~AP`BH9ry_Y!bw&k{~SX2)pIen@~JcVEi};mS?!cR*iW%!{?TgiBGWkE zhC7-Y?e1=UWlzEMsT9@?>#n442IYI94QMywE^@=lh|z4YvS%cp4yjkdiP*HGWPl}u zYFB}bIx^SV$3YC>B2^Lf#i(H#+XW8F7VqPaC7V%~W=O#E&V&rH{zoVL;$)5NXsw6& zXbIx+-b|aLVSWIEuEtdVczLQDrvXQtiI(!_8zDxbljXurp?vQNs5D*7==AaO++Fhz zo@{s)c*^+*WW)x6U1Rr$X9;v>sJ?oOjiS8ta-%t(JelyE##abtbK_|m^;bdI&IgYl ztvS*#xmhIFVX&U7OSmFkqOzr9KwV|2(ho?+uYSX@$-=@+{3V9rMs9-&6yY`9a%$y! zZ@WI^G4+SY|AO3W(wEL=>LT`wg3rqyiwa##9D!wsctU`?-o?$o#cj|-oW@8zjDg&{ zXS_jrqmnYY7l^qXami9hai5bUfHO6FY0ci^<0}??`S&0xt?nRK3{G~0Dscko(CYEw zVJ9}VGawVngH0UWWT#~O>#DLs)d7w6)6E}TH`WM9{GXuZNb&RBre5^~JA0U~o~osO zd}44jP4l-zk=IM_+hZPoze-w7BBh15jW3!vIr}$C6u=HUzC$h;UKign?Zc4DJj> zw%;&AxE?nLZ7lc_iS#Avzq9h#EN^?D#M7*OnM|y8xs2#LqKwCrP*nM;zSoKA#=a=| ziE^0hE89Okl1Eo2`9$wcN|bRAk(C;T3~5egsc(t1*T~-&P-Up#Wz&;H)%XRynig-M zViNXIv6j}Y2+;Wq>juZ(Vm`p@H-Cd{<}&c`R}G{R3k;QFszbOCG1+ZXe?h)P6Cp7E zm2y8H!@z*yAW5;5+(G6YaCu@gG{8@NU196crKSy74pM27{9|*m&EElQtM6%FDT2~kXL67bdvZSx zNdb|EG;QU__ywy^7RJLI3J&uo^W+|nTt7c#k;G;S9%DbcrP=qC-6oa=20{#~z3_fs zUj3I242Vt{~`MR&CM;Q`cNZSdEVup{QD>W=krLu!iIFC05Oi^ zzuxvg{NCy`)F(Zg{Ryf->=yzM2IAkpCY9_gSkarUn+Z;=&=MAW?3|dX?sfl@n>ZQh z10qT8=lMp`;x&Nak>t9Lk9(a890mnM5-oxpR&%YjG{{b=8vMQY<2b?m;mus7wBF6h z+H21T6P9~|)+sp7U7X2_6I>Tn&*c8TcM?pbx^bN|4mus)&zVR3E&`$G6xHh5fGQH| zWcIDxZFkbrmm`8k=#Ck_j>862xdZocyL zYD~bLt)|lPkT^%sbC%JQz;$!8;>j0Ty*{Iro87C_F90)C_iPuy7v%$;h9KVhJh5Dq5l+JGrOpt53B`)VTWXO~2c=ReE{$535_LB_ z5~wG5NSz5!KF~=grtHRCs#Grc1`ca1+P`^Y-#uj% zNj&i#V!=pqAh+YLsEg<vpoK_)J&C&;CE_6? zOaZWCZdV61!5MtUnPwp!FYMUuY!6wGqA4eOv2EOz!&Ib-^h+(%U*0^k$D!=;(lKwgUC!?ED5moxF7S)>*G4f{X-dIwi6K#qElC z?SjMIlo|RjTquw$kiM&nGyF*m!hixSBo{{uruGBMJOvO{wiCDq<$86{lsWUi*+E2? z_qz}XJR7~T`O=_B7}zopE2`` z+`DK7KwM?Xr;_S;R%4`8R`<7jo@env(FD>j+i6#O<2Mp$YXhpqQjC+uq9EI1inK74D);Y0F+R2FkMNKqG=srW+0 zSV?U1_k=W_%~at9Lv{D`Q=2?6YH+UXd;%acH>1UA;$bB|R<0E4esee>?#$&$0H@nx zy7q^<3&90&$9rY)`X)pW!#{uBBA&eHPo$B)F^we^cC)e|;D3syM!>_E^HPK&SegOK zup9(j7V7iP=-R!JsmTgUe7E~8hQp->;Gjr3$_efOxiAi3d^!Ka0Q+3fpg?>hs5QC~ zD0Fgw>gv3P6chkW7RfU9$MB^|C%R5?VO``#?g82&{oI=oeXK(!K=pL}eFQHSWJwY8!dP~Lfo{t&c4_WAJx}7AYIvwvH zpYDkr?m&8e)=1n-&OAt$uuuL7&Gf)?d)#ps@@mkRkWUdoe)oE1%f6^P!n8x7)#Y@S z&TO)HvQ%1AsClm1i2ZHRPU+zf{W&y=WG0Ac1u%J@800VB<+|r(8Tf?saw!84jan;d z=WBOaO;5h*#Zx(3?Tt5)5%Rf36w0J^8A_$FyxCMh^nR%R+9&l98fhGWKFEaUTs(E3 zuS2Qo*C0{{_Zlga-&@d`^+w`e=hev)R*>YV zvj;+D0^#wa@R1sXK&6Jkc=K=oi^n~rSzDON*BoQK%zNb*mW6V&1?vDR z>|VsZ6NNH`Cu@)Q)S9JZJjr)2fgPnPyXCL>F4Z9Md*ugVIto{rQY)6zJt(d!4ixoDgq;lal2gVo;MD66* zC`aXRSX@Z>zn#eW0fAva1FNyrM)~Q=lKb_|==e_>z`;A*4-*>56$=V4hI^yrN@~83&$G6qof>mKbch4*Id?riW3dd zUE8Z8DOTx1Xf%1i9vgf`J->50o*S5ceNg<=CGm<3{u^6~Vhs(76b!&`hfJPPU$?W` zjKi4mq_FE^-VA+VZ2?}pimksF(^I=#89$&fAZ1W&+yf!nTwOk|iV8DA{XN?pxDY7M zKa)aq?+wEYU((g^i3tH6+iI2p1~{l^M@yQQ&v6+Plo%rN920^GtA)X&^43#|kS*KH zd63sOZ|exqv|$=qee+9Wa%kY|_k6h!Q%%aKJe3fglZ962RyQx(wxj_WyXtA47c`Ra z#Wq~U5~fh0Kr)FunevS?6E0r;yt|X7^Bk?YAcKFv%NG*FAoRgv9S^&i+REUfie{B= z2g^baln#u_#~#@@7zi>{&fYNJTV6iXfkaCq2`{yt9WN_GGZc9tgY_we?)`@Mcp#g{=HjI~_>nnG$UfWZZT#4wmz<_1;bF@v9blprRH#ftCP{QTX; zPkN_Oo1N)4>@Ebmxiidz{SEtX30V26fF75~;Kpnj;0H(&UEnu%{Bf=6_$X-g94v)9 zC;yyWrRPx4Y!cHQ|1R)wbJn0Y0&p2*qlqT)@E{!U+dGknpIv^j?d1-HO?j3+ToNuS zCR=^fugL{H77E4c#~mHr>d#QY{X+n0Rmt=dIoTwh+*BVZFl9;Jy4ZAO;*^y}5V7-q z>)7(SMMRM*o^&I9mN!ED1|nt7IH;ff;)qm~yZ%ajuQ=(szV(mI2*3pUBR>JZo71^X zaH^V-xnGS|$tKVi92Fi0D4Q4Ol&3lF07bt{&+^$}gr@0)({Ai!Y4L@l3cNPGg@f&!Tm#o!DZIx^0@nBKc zX&>j>I)UHuG^FGysj4`nLQu=j9G*x7gWf<*yU{n03?0LD?wk%ckAiZTjGX?ky?2p5 zsNx3;qQV;6x_yXxXq9QlN|noKDpN1CzR${<#jlc9YAF@TFyNJ{=yTehkw@MevSn~P zrsk|}(9WY2io!CweP8#<6!>0?B6%qR?!q)_|EhSh{5ofzutXy!zhq@ph&pp1*@i zI)yuf%s^aKa(AhQXRQbpk5)Xk;N2{%L3b^gFW2Nz#uNI7x+?Uk=JPnE<;0`Z4vWCf z22tH44NDs^N9Bb+-Wo~+eVR;+gB2>&J)hQ0CyYv!{c{yTMwjbFr)$v??yiXc@Hgsl3A15&Kqj&UFcioiEw@|o{(Z!qC{l!mPe}V`NDUT7k|*iVg~{1ro5s2Dzr(VaMDkPEJ1zAa|_6RP&fqFVDqh5f335m_wilUaZhd&g59!UOA ze;H?!ijqX>qwnkW3M*>!=2wPB#+ft_bLVd*|5WJQQ?V}#%DnJW@iXPL6*5wk#FbaaG+AwS5+malhy3Tl4;d8x&oac{i6ZbHZXs?9F4 zS0+_dvDecr#ao6sd#I*Kus{Cp%(6=y*))5m+pd~oFwG{;Ahb6-r2OJUkeWWckdMC-b&MF4Ay?v2a2~s=78%n=AHe%&qnyYSZki^*T7NR!+a5;^N>!$-hV+pg<{=|0 zRfVwCqF^HHKDPfg-ve!`f&ayZl2LQ_SIDU;JEie>43p4m2g!-f%1lKutBT|VFkXtd z#Rj3+I72>PdPYXEh{3z7akMj}3gZhy(htg{PELxG`% zNLpqOg!7OEBSZ+&lS7nm_je0iNebws*{C-f$L7?iSKQl7^RLkJL(D$rgaG`h6ZHa# z+qHgBQ2iTo{u_V_h53w;?@m~xee0JV2amab;!>ebOMqlo>x*RI{ulHW$^_7tbVSYW zUjX^PNY+YH0OM@~pQD1!|NI=3GXQ;ITzg~v|AU4}0E&H?K;|+v9+t!bk?cSJ!uK21 z>Np6enco+dRISL#{y*U~XH**h`vbd3R}}&w`h}9N%4wuaF=P^cUB6_~)KVkJtl5PN zfZEl%`!T|X;s6YVe*kVF0jifjBZD8Ng{Ovi!K}8Rp{rE0GDq8FLk8?5iw%OEM2mC} zqY2o9>W^o|y{*+oPNgZ)!Q>z0Y~G~xMi2qe=<B=DkLO{1{&9p}iJ%Cj7uM%WTSbLR=kv1qLfq`APSYX*(ZPU@`c(0E_?4hg1H*SnA||q+pq+s z0u97Fz}uABL_M(cRm!eBvAGZP5~&x1)QqHosa0$@-tJR*6h4N6!xi)9Wq*gR%Y;IQAP zk1bd{AGNtVoKG{n#~uuw{Pi!9&Phmxs9i38#^6;n@pJIm-(kzGL{?)bw;ZX-mx26? zdgz}g=SZZJK`J)lv$8ec3m{;#KBAY1C5Lq{S#}l5n`x1zd$Gy1fI)VG5+l#yQcE-% zmAJ)B2N>gWA>lR3apQe36A(3o$gJ5O&f6Kwn;?ewGN7O^pB_|d7s9mqTbAjO`OE~6 zMO63+NT<#Ri8L=jZ2jBjNzS7iSv$yusdZd%dV#6s+BHG=GyBuua|cs8^4?zVkIg&o zo4zOwO7vZ$;6=X9R&`;pibnWMLRxb>U#jY6tKHlX=oUcthq4QPQ7zXP;=jqWqDx}T z0l-cT-fBWR^2BD&;8&O_7Nw{F*YT2fb{195E5KX*<4#`1f?AZ&YqVK=mm`m$d0Smt zq#ZXNs+qShmqLYnd*8UT; z!Equ>K?Z3l^@pXNbyjt!O<(<^J6_ENI&b4KL+ymZk4o8FHna`(SM?c@; z%|B&58{l@amnVBS4Zp4{hQH=|f0d#B$NBgnI`31|ztmy>>-t(^yrrKhIoPJLOl33E zx8!H-2c+JbK{QSlC8&ZuISvipjV1GDU}hV~E=3IhrcMRmEj1k|M2zQlq`mS2^oupw zrxP(OQa|WulYH5%OfmFTTuKWZ!}ouWbzc0FG2K^Z)qmj6Kn22=9H6oJzT9X%n@xm^ ziHmfwn#I8lu-I!pIXL*iT;SXj0n`UP2CK?n_Z|X5V#Xti#_7<5euD?K@@B=eK-YL|^}WTc$7Nh}GNL`3W(Y zPg2Tv9IuPy-+_kCq_wU@|15BO=02XtumIy~Mur3kKYbC%gHQihXCzRINj{p?2rW*B zg}AK7wz4m2Q!NZ2O|WgL%lz13{e$x=gm_IKaX zWwN+>!afCE>|Wf|J6V&*%`D^dErcQ}u93vo@Px{@ z$MbE$FAYkPWmc;86erR_Al>>Tz)UJLXm{qXI6Ad))Iojq)Dk{51qg1fn=BV&?0$&; z(sIQ_H%Qe56Y#Pl`tL*O9?b?m5Q6Xi8{k6EM?&z4q49Z(`G->Y{59$?3KPrQ=J3hh zbd01)|Ioja?RETXErovn0?MZcO($DL%=f;l2&Xc6(P!|vM=|B#@PEFtm~;6=rjYN# z=05Epl-HV|Q4Z_3d{@6*K72@-c7(5;r5 zrITxXtrgX15vI3;QH|M++Upiz?6RPuv zQGPQGlonR3&75n%R4EGY4W>945rVvS)YvYPvOgMvZ%68X+Gw3d}2@a>$po} zC^A{Y%_%73vhmb8%r4oU?klW5kKVM7`hIjcf>p{KmJVIXP9V6+G6=b}+Q&iOE7|^K zOa0$Rwg(TCRw)w(?Fru+mA=fY73xk2p?CjXJ1+~Va|xz#lDuz;mT~icZ=Xq+Ao9c& zRvF#iAThrGpMNC)6QBULz5(Otj8jXD7Mzc4aOJ%pUd#c&Okd!6f2D~val{;-I*?R3i@5RU9Bk6rJbUSBb#ma&2Vxnp zr%D!ltd`mjsY?}JJ?#96$@}{~_UfsPn3eA)fd-*|Pp|J9Yz5-^jEVN9+lqua(Ehn? z$q5v=`Bu6E0{?bnD;bB<*@O3H$_D1X_Q%I8T^%e*rm%Ydw%Uoz6Ln#^-o?N4TYor8^o=%wi=>U0`wNG%cuibXxR%;s{r z_`I(oDzw@pK!H>h=8)l$^-Alen@Wj7s29DO9Qz$(T=a-wsaD+2jil!QtGD8>uuZN%mId?yh#P9E;Bvf6U{H;zw_eK6VjSRI zT$@$lqrdG9LG^dLJdnwBJ#Q4Q%a8cSbLfTzN@-(p)9zr_R?+WMpA06 zY0=D!9$s=#(Mo>0`#HJbx72ux)F%+VA85EzaTlc z$Gf_rY_J_!=T)-i44!MT?tne5(CMi7El-A07?sho_nR8KKNFzA31sa+sdKNi%6nYr z`0j4g5uKCZu{k!q^{=c%_gD8IpsQqf*H<#HcS!2b#%UZ?;T0jbZQELu9T4azO$ep_ zl8hgFTzvw)XEh$R{yM!;+Ycb_G=A%Ay>cExA#jc})BRQ4+J-9xx^3HJ0J~HA6DkK! z!ylCV22Mso(J3_zMakLGey&kTXXb8$x$oR$wmfCu7Vi8hJ9qn>*VLC#r2iTDJT z7WB=={Mb+VTibSS-*lrOI%A*{EK6kehpyDlNAzU+POrre~<>AR# zcIKv-NS{(7HnPk+pCe{dbzW1-^NB{Owe6)YTO$5j>q4!*N{iER4u~wP);XB{@lqO3 z00OqC3gmf7U#hYVB9xTM6;k3AL`q{uyb_<4HZm0orMe93E_zP$f}?jLQ_&R~tSs5x zCIT~gy@Vd`5dUVc^#m5F{V+K7N*qk#8Ug$1Z*460x;^|;$=flR0Jqty$5BX1ybP&9 z7Q$Xm^W~1n$#athF0E!EoAi?-ZC3CHv-n;GBN~omXN49OX|v^4n&{Y8?`PkUlyZB6 zGNSsMZi@qOlMY2uwQ)ZKe*Tl<;FRJ#Rp>b>$KdIYHuKK-qM)mgQ%ZuCD?UPTnWM8o_9u?)6}z{j>(F-KSo{sq9Ur3FE=ZjQdOCuY0p4@XuZu zERjnLVmMFNGjF4cwm5fAN6R9XEnwWjVy=O+`W7b-n-gS<&=BK4&*AC0fLx#7xZD;2 z%#H2QEV)z~=zk-%O*Q?-up{z)BH0Bs1aVW;>pk8ijm%uLP+f!Ry^*7lSh#mDi)Iwl z!|CsLQI&&z7iu^n5Fe^sK(YKWu&oPTgIH#UmM_hJ%UVo%4(FZM@7D2H!}4Cv-}Ftc zMZZ#L+)M>(1d~bb)-Rh|IaRSu)pI*edQtvQ|3Dk7GEfm@2=B9csyy#y+)jlqdz0w9 z;tU5sDNI{0kJC{;+OKe$v?s{#ax~~9ZWY~vpc6sj_RMv>U)g&cY>7?w>krMAg$b{E zI+FYAJ%HA3GTVzUmCGHeM6>|Uc2>;Y`FNGcjW2sX6oUqhxZUGc!rAKr*x!`xaYo(K zshr;*jm_|7EH(iGESODZIEsjm8&v5I_hEcK{t`vRkV^DhHB#-Ne9Jfh9y^gJ`}gw@ z{?f>NlmeQ|yE7~Hbu^%(4?|&wj7C#~hqvOyzJjvgcxp=l{DrKnax^zoT)<&yjiCyn zp$w&L=AxUOHou2#DTHkQMK}{PVeHH*F+it$-#3v&ixS^!%9Q3*ihk8NZAFN9*R;N< zwq?l65v4>S9vjeRLj;bj_rAXW*+7u7n_VGW7j{}|m0#amyf`LD!8~|M0pN1m5ViV| z6=vtM!5@&pYzaq)YyxAw(B88%bPqbPH_G>NrXOFF5m*l&lg{W0$QyIn|Bh8fqmc4G z<8c&=0ann7TU@F+C?6R48_XR1yr|_uRA|uOQ_E?*j_0#{Fb5*-gfcmG>D-T{l;PWK27z{c1Gj`oD|namujy`vDj( z<@8#1=wZK20WBjYRs!}*y8|ld2Z|r+l8C1rkO-z5j%2;n46;@7ia$S&+%M6XqP39N zRIaL0+@572=K9A@xzk&`vw#d?r(Efc6+5!!@g>t1Jyjh)qW= zkNGYo{r#;h&?mIalza&Xu3BsM2tKik&$4MB{rx?jg}s)C74j%V61snUJtg+w-Ab}; zg(_txG%z6MyI)-ZX^dw3OR4^PO-h<=XKz})R;LQ$adAdSnF_-nQ&_jFCfeJ0)J+%4 zx8yptiO)qNa)0=yu^LXVn}wmqqguubrWN^Rx>*uI#AjuNVXc`&iGF~?yo)(eG!jy3 zGa@94w7Xiw+0RKB`=e@a?-Y*NLUPrg_1;n=^=$uBJJwXboko=n>ZVGb)<2<&4L0O^ zWt{1Xd*YcEz#s<%#ZvBE26tCkJwCgHDznArFl7g~m?y~)4EB(5GcLIT(M#q-X$n@C z{p60$>ZdS)s@17QwPL}T)ALr}EqAPfR;H~KHEEHig{wi%Af?W{{sif2s z2CTG8iw?WWM4@y~Txv3Rn#8M-VSZ68SKBO_Gg(Lm_;3 zq4hSN%rBKGlTzC3Du`N?s>*m{Mcx z+N3Jg7*x!KZT6EKa!BW7Hx1WD7MBUFT7$-8=a*Q?kSp$5 zqMn?AsbpmO2VX)-Z|X+%{7WOkDpDMoreA?0UwjHCBnZQwXl2RdR+3c*R1bC4y5XVY$tF>uzlyWA zg2jK$s`dwFYL#(VI(iqJBoidyqBqBgUK6!(B37|y^2Fn!Pq549DNssD1oKe3kP)qy zMVFciUN9tnV=oMZoiQ7Rzvtopka#ITZtfA&z_E+_#5tX~EJ7fa8l1)w+LP0#eVcHW zm!TMYiL0*vW7n}NiFU$e7e8JoAxCUk)sK?m*I-^)!H zsdUbij_X$w6_0x@s7yJf+U2x*xy?KwElDecIFh*kSy8@>vYo|D9(x+kXPe&$0(~^F zINpumbLG~m(F@KO>v>+r)M#4?_bv?_b7U7}|94aBEh-?2^Tc`)`y=MnVCV^jjWeb* zV@Hc!lHGsjbELwZoZYKu$4miIF%(nQ89qF8@SGoxw4baW?_%#@A^wfiyQoOugiq$g zrT+bc|MM{?LqKM`md5`}LGXY60>uLmB2ZZwKQSQxb3iZGe1#XtZ`*Gb=`sJ;FS@8! z?E<^zomW?lhWKuNT}8VdWOnt9x-a@u$IGwd8IBDwp$`g}4{BYN0T7${&Oi#B-P3NQ0wY=w@-&r*iCC1+G zd5E4+T3^XwZ-nb!9mAXc**SINfC*>PtUu1xY&e#?w*rB!l@Jf+drK+r@3hBy@SNY5 zF{(K4^CY7b3SWD*eyM$sMe<{N=JQMh)^f`Tw1Y=su_$|7d6|cKhduG|#PVFQra7f1 zVjFs3f?nMt5fmm8HY@H&@fBMpEo=P-cn1%tOhm|zC)Ll*scfjN67rl@DtA=&jm_Ebm$I~+&FPj)W!YPkv@triJ+_;qnBjC} z=LW*bvGvTXmrw0#IrdSF+K*cL9^dA9|6XlZocv#Ff(ONB zxBHs1{0C;Q##A}C2E56{VOCs`$&5BJt-|PlQSz#bR-;1-5ucY3>SR%+bJ>Mt(oHB! zzO3Y0cTj=Bb9(+UOJNamsg?UR@!!Kk9%mZc+yIBON!;Q0)<@}Vj?=u$#LUIWzvPHX zdL${d7dd^y&QHb+@R%$R43%~pEXIA-dqWH2BU>)E${%|jvIG2fn9OW+0KP>$Tx`kW zuwD?S76X{$unH_%W*{ zp>Fg|%{H&s%G8>%-KFPG?4i!#y%%f#25*kdX=@DCp9bvB)v;6wps$xl&z=-!fcyJH zv8g||2eFxJ@RcQ)Q<%uD9ASKLI?&Cj;=QKurWKm2_7IDCCxW2jh9>?(7Oo;e9W<;8 zWxvHCRjctLkh$)mvv=rUxnuig`+T%{rDZ1w{T2*RELnVLWw$D-HljF;^j=-zar)+R(U5ETK zcfYbwZ@jJtIVXSj-2)Q>cgAv0VP0Xpy-1~$Ldnwq9VQL}9t+jX(wMw>wd45h3lTKV zj27y=Z?DaAeh~Ha_&toKHp}oyL7o;+(DK-IR~RaIwQ=@ z!A#O*RsHP!sUm5a9+_Cw*TkjV&9Y?Uyk>UC>rbT~C=r>R5eIz$lHL4tJ_Ty zpV)Z_l;owg-}%FA^;fIKHg3A?uFdOFBV(Uhsl|<=tp6QGrcdp{`5YtpnHoqz@mK+TD}-6X``)+_%OCoU~|iJ05=oL^BHoR9_v<`-=5*Sj_h? zSg5nww@TrT@19oqw~9qRCwN`lWjd9bj}~;D0Mi^+Ns4;Pw8;sLfl>O=jF{kzg%!W`An-_i?Y<}2#6IU#QpL8 z57s8*lXDJN4{yLGi{@G>7#=(7-0mus_5UNzY;9WGFM&^QpsltR#TEqBN zhWpedPLu=|ozSL4(khbgBQ&yoQ&bkGNLzD#k@$oWt1$H%)nt=IDmk;`{%nwbZ;0SN zhlSl9o}K{2vyDu3*}3cGyrZ8f+*uSh<9lbA&hl>xcDB)iGU?pX_#BpbHD*7V2b^(B ztcFTGI^Jw9ce*a^+kcMSYb$DPvI8sn{OzvWL;G>1OKXWVtYY`?ci~UE+*cSs@~7hT+K{TMxoj+}Cq8&#%fK&Cw&32+45*>zuMI zRLCl!r)o-M8=?rdj~Nzr|D2+GSm$i~KI5fQK7K82@g*;$!QlF%L?(SGZI7Tt3_Xo< zrB>#E(TJ?1>VkeP4k6x-M>h|&p-8aY&k)pN*A2ILsV@e@AF<^V6U?8)@`5im{xWx4 zMMC#CxMv|7_J7MYHp|0ML3X-W&-V3y%YdhVl8;Mggs@kDM*}>bd4{E9es3dL zP2YWVyFxDOCD10i$=P1IK2vtF%|qk!b-B%y%Y!XZ@<(=kI6pSM5)(dmA6abGX_I&} zRn^i%$q%2aPH$S4`62HA!meh=mcUw968RBd2;%$Gx*6T=O zuFo_Wb0$~E;TJYnPTAiCtrnXL;_E(_qAGzgBToM8bsk-*Z<{9j+wZ6KnA6GFvw4H+ z?E_yOrmr{3f>NBrDS_7t#%!q-wd28hrAf@uTxm1N&Po|MeI;Udf~J7%^@oo|QN-eJ zI)IB&mzmc+}^APYlZC`2kE?xH*qGDrh^AuqAm$vEb980wNp?9-#?|Ff)`an;8kK@)y1E4K}Yc1QnG|xc=(k0#9Z6Mu^l+qz3-5?^3ba!`mBVE!RlG5F^C-?h4 zkKX(D_l>c~SU+Hl6<4e^=XoCI5fkuK>&Etrzfyc&S{&+>H>pBdAH()z@R*Wo%V#KN zix2nI97F=&wEWwZl$1drUDys?MJ;zo0rK;nDb7Hr*l0ZB6tudpYkBB%2uemL_r@b5 zaa=OeOXkutYL3#EkM9sEhL&wx;ON|1v3!^(2Fn7ePfMZAwO~p~*S* zB9FM&CNfvc5?QNzIyAK)mKDF%8$JhB5sTLon4uJcHGGHLJD4Bwoz>@YKc^ITNQ&Z1i<00+t^z6t%tn*{Qzb z@3ugmz-dQyqeE-Of4R5yRoJDbHeZ1YJ_<3;s4Kuy=mz?2Mw*((Cl=A`SD7-H#--O2 z`HDZY2V(1M*^`1pm2++cf~Gq3%ofParYY&J?i>;+khvn^}oy5uY@CWjE&@OyH?pQ|Kd!7OvI ze#jBbae9-pAS3tU@y@rJ}t~NJxU6eS}f8N%HOl=6ebYV;A;Ok54(lz zr;nOtqx3{PasMDaKIc4jx38U9*V;5egHlpRhwZVs)$x`S61jY<2ZG_@SA0OgFO`X2 z4zV%omY=*{6}-3aIjFXO*L#01ZY~Wr1=bN@Znt7iv7#86l{cDUOH`f_WaJ+zhJ#;e zBL5ob9eM>za3OVC_1RHKW)gMZ;F^kz@Iz|})cMjmNtpS>CbFA86ibJby88#OGP>L! zf9M&&LvxIi75UyCRHIW|ut4)>ZdZ9$KLwhdN+u)!rW5wTcrs{4DPGV*zBw;pEZtA^ z5ZfgQTzq!2voc~e6e{F)xVKdj111X@=#cG5XdzthR_%Vn!-;C7-RWUt2bcT^#fw~O zIjzK7j4zXamj1aND_!=y-+N56+@+b#ac;P|%fsUk@|u9?XJ(jrZaCYea|RZi`3afc ziNn=VDwE+}rrL`w49G5~yo|0%EaaiRp2KEKAx+|MP)f|^C$h(!H|a&1IxN|~`4}$a zqes7&M)~D2>QX!KOYES3h9=i7_ZvO_3ONcMoFLQ-jPiL9-&f;Z=>e0m8fwf zSioS<Rq)64rV#sZ=bTsn=??-xcuLhDmQEY!rD=x3^SG?59%Dcg%H# zQX@`cLDuftbxUT3lb>0(*A;F;)FW}P->G!sY-;3A5<+s2>SLymS>hLH0wBrS#HK6+ zpA#``!3$xqMBZw~5+wM1!T1rB3&+V8SiJKrd!Lze`k5r|U?WSxGfJose2&jAO4Vg< zqfo9t7}EKg4=21)A%Fk;J*GGu<}8^f?Ksk)oVP6!7fXBv72G5|iWvXTv<~raHlbfw z@50PJKQnAE(Ci&}H`Lg62M6CD)IFna2n&^sME*Pm(t)L?z5dGwkGHc6=9x^x?@w8_ zAbjUI=?Zl;<%~YX`)7XCx?bT+;cyR4FjtI!N7r4Tz-~E7yH?*-9ZdWocuZdh?%yqaHD0l&2*bl<^@-@XL%A?EaXH7cx7?@T4jB zsdmjU=S$N&B<}>-!2+X+>pFj87K-5X*lZ6svL&kSdMUwiUi709e}p*6{A`_{JhVh1 zNlgHWH31O@IddGB(P!oaw9His@?Uzcf7oH=IMR^!2iX}NY8X2od=I6%a5B*^vOFGa zZycNIEvn{OrGf@P*;NGeV57?WaJwF^q1h>d=BWhbyLv2ck0TahE~AVG`8QC6A$}|c zWM9z{7*X+!PW0o1;K~39|AVjFhT8pW#+XaSGH1oJ~l`D-NttdSub-%DiskO;~lr{)I|A z{*?8j%PTK_o{M=3KHmqk32P{w$@h|bX52{g(I0mT-Vdv6I9!>?* zo=M|!{$P+Q#wmQ~>FLOkHb7H}$1y{eM^1&#Wjg--opNvpm(-HU*$krjAeL!{Kglwj zS27=T|9YCx;;ev4f?^DQ?LjY!L8gylfx1FMCKY+j!Gtf%FAO6ADdmqO!hi{$AUibjdd@Mu4 z9bdqfAr^BFBkyX?etVD;t&-vWyEu6i(T@^AR`ID0`|1tz{l=FqP&+Boy;~@VrZ}A5 zZX0ut3igJMR?eTK^6gyJX;+(JQMcA@NIHg>Ra0UN+gNw3cJcdCIFT5YA{WcuSU1~d zFz(~i7nLC(euYjpvoJ#iQh7Y5;j80HZsumnABI}UH0jM~igWdY6_$QTYF;X0wb@)k z%L|-yb77imR1@;6ZSfc5jna%`Sgl8CcemUX>PzLuDOQa1=Kp*+U@wC+F6-fe~`k%;dQET0@Baxhg%>i-GvNmzehX^&y3~q-U)e!R-=%{v(lx{ z8|b9~bq3$9X2U>I(tYK0t(jwb#(lt_n8PY-eL1=oK>YaScGIdn{PguLHh!a4>; zA&yAz92efn8!ie@N)(OB;yfK*y+qXjZ)%CvMR4?6dG}&OZF5%04^5$NLY)w5Bn}b zL-_Iub@eXw^BVb{8Kr6;_(pl(DK?D7B>D@ltjMi|-B_(ty9Orm`RbN-(8z&tSkJ#d9;ZE=jU+mPX zCc9eYi{Y5W=6X7?cBhVuO!u!#3u&rbFpe-T2*pB;mDz`o3ai1FKuDKWQw=W>B+y10 z;)A7kwqcX;v`he1R)Ap<`INUPYogLlYk)q7Preb)#GC#L??XHR1R^F>0AM!}x>a<1`u9KM z5dZ^v0dWrMvw+(F9hm&{Y=B1(J!x=;jCFVZFEJ0{6X6ho^!Tso>eHL0$Ah;?zuuEt z{P*X2K>?he=NZM)zfXdc1JimtHpOxOJUyF!`sXPQbJB+$bV0sJK+y!|9ZUH~7~=oe ziFq>v;|oNnC_mu0qLiRan^fQjOe{DY#;Qsc^=F!0(TcSmao&D=o}yW8A^Id?FOV&P zKLY}qo1u0XtT>^>O&5PG$evDCt8dEjDB17Z)n$Bs{3IhpmLFgtPq_XuN96hJJ4r@#P;E*$K!e7g-w3B8m`Vb}spi#?Skr zK>LEY{=kd+6wLhO_6W#4eFki!iQHB~)VzWRz{@;OIo|fByrvRp3xM-Q9Mk)zURjlb zsWeEKAd@Agc_9#0^T2B}wtZHj-6%$L3Ph4c-Oi6oVfTOrW-$3E#SdHUG1b3iw?aC- z4O;Z6rlEzqyE}Nq?)h4)p@o0ROO`R7tUt&QHqW-%&sN$yMKl(G{c1_U*J64@FfL76 zusSsweKL{-p+8I##D>jI$BJoHDT3}|v2O0$i&~Fj1&)cT!1?e}h51ZebRwx7`B&-)a%@#jZo;Qy}2{@;&s44v5dW8+Sf+ zxi{&*IeeQ8EMh z=@Y!_jS7LiWx&#IRkF-?4g}k zCEBNZ*(3ppkVKBHT^MOp;u{W4pHa#qk@_~+Y19&kUO-?`3`=o$T5i;aD?u82^10P@ ztm}RbhX^AU{^sxBWC?T&t=}(T4mwe-B?kzCm1nNz7z`<4DT0nEo7scD zpT1hr7qrTLmW=qDB>n;yWq&IVe1P2Yy53b{jjhAOfbF#`R#1l|Hd>g$fAst^3 zUjM`SIx5i2W*dsXCD5>q=h;^4PvXUlzO6^*_L#3e%%dbQ*<~=M(PMes1`>Bcm%A_c zTt#w@hl?LKjDMG*uW!fuBVyBu0}7xVAe(NuS$pVv0LHqqP%DYBTNab{-MbGyO2RbU zCMC0w{8ClkM$*R1GU)!q=RgUqUIQ4-%F8J^{CzhZpp}^m2C_U(YeZ!SW_g~>g6(&g=;d#!cPF^9 z8I+?IRYs~1eoef8R%s2AtBoBwA=4VwN)_hU7i)YyEEeKf^fCO^>~KDBacr~22gcH5 zl04oX(xPTs+|3rhzwUn@3R3KM^%)#NTKNwfc`BGV*21C?DRO1KfZT)b8as558q4cVRzay9O1l-vcTH{nB9zjv3=!mu+X)g|FxO; z?Z?x-S-Z%cYvV_W$lNkdc^-3{$#0=44$B@;yny z{>SId7XM=_{+~BY(0Srn4$o5v()r`9)VWja$|#(r5VW)thv@xC@+o)c z*utLWXi#WCW8>vsw$t&dBCzQ0~%qqvNARG9$_@jt@-$Hs6T5@NKhqH;<;UMQ6o*HIxN3hEn{QV{!8;%@C6=^m7`moT~Vdd2Xwr&=9yMF(M?K3J!4kZmeN<@%eFlF#nn^icr zL+U8E>S#0Ojn6|L-@C@3kqBktf&Sp@%@P+g=FeYIwkVqj5o(0nkL2A zcP2{S;<3$q;vg9ClNd?3DyT;*TRsHIN}i6-4&oMqVOtq_9qvdjBy*zJr9k<>$`2D; zY3LHu10n;rQalE0;eqrqo0|kK+XIj7Dhwl8;JP${{X4h3OW}0Un%&*>4QCX!+$U8+ z#T;1@6ReIrk-;Qh|MA&cN9vJ}?Uw>&Z%Q|M-=+=z5dCXA)unwpdw|go)6a02F9q&Q zmkvmUe|Y;+yxL5~DM+GxM7h+4%8=uqig(@bJBstjC|Y=Zp^w= z!qt-U0T^ApCC#2DNmKbsz`5ZD5Of-@y6;X379BUo}Ha3hv+>%2Rh63?*StlMH|NETC(xR`hD}>zF!qX z@Kq2Z2Ob6OB|~0d>thzDP06w6@63CW+8@k#zmo~3S2U#($C>!C^&&MoLQLBkt(S0go|O4F3(@Con2aJWxCzo`#a_+e`hQW7BE8<%`bvMY;AcVaL9c#JqjS_=W+bYuyUWnMtr0=#Dk&D`GH*OwQ#$N0 zce6LyJzIscGd=y$_Sd1lyU@vI-`YhsywzhF{ps;L5_qhyn~Yk8imK_ac<*J9Cs5cK zY0P;puoW8}WSZxQ{E~PdbM=1<3!QEaapf{{MjZ~y;)X8!={drGIQopF_$@RJTx9MqrZm2~>=+(RS*-F$cb~mKZ_%*{{2A_t;la7Dw zr&=--t{2;`GN2*S+z}Q%eq~3!G(iiUw@;8r=COI@Xp=F(zdCCq6!{Dx{C^%ICeSr; zuWc$4-mdiwx=oiNcHV?&89c;FQMBn(lW4xih*vp)wKo&b$7q+EM_@!-`*SV+xhNSi zB2l!+3iB2*vXB%ae?6>);d!vXHu|qDhb3sh{paKU_d|nUiujR%(DC^76|3&Dw@?T?q6`OGzk}^p(*I(g!u`hBZ6~rc;NkXA*clMd-DQZ=KXY5b z_QR2QP;>>glD-YX3YEu8H$2I{t3Q1~)7fBe=C7(89KdhCQ#F{(t!p-w#{_z;K-PVu z*Ajoea2e2dW)WD{=LIAEkod@5kj`%3fdzCQ|Htm7$SUJfp|L8B0oM+4Olv^t&(~`G z4n8_{)wX{9=6v@%?Qrttvgc#;f#7m@(d;8N9-Cq0Dy5`U?X+v%+Pym_u9K2j2r)Te z_?_ne28~_+#eOS%6rS#n*pscncxuoZon+V1$YCx$IGg_jZ0hLk<#(Y@K`qo4(+7*T z2MhJC9e97;b5=huUF@%UduNBOkNQ+PTZj3p>wS@{yX*TYgJjS}8jZ3afl&mVRhW}f zh4ILXSN%sYwTClrS-cMx@_pUd+ykpj$2C)2cJhy~G5qo`_ve{G>{2)|2Nu(3L@W6X zH`ATBmPW9(L^`d`cMf5FQOgg`|$XBQgUGI<7ovYcU@qDF(UD!)&jS|KBXK)2fnh`Af@EvwdEF* zW97{S0cq;sl`=@6NRK?QQgho1?L={)R zzr#W@4!2qfvX>odDcx^Sf0hlG+K*Q_9L!OsPnse<`xu3=8C|Fx32OV9 zJlB%iPm2*~!Z$NR3f&7*1qoiii4Wi0AS$1UER-B2XKL=F)ScQs1Ot%XK#mK|Tv zqNoQekBCW8=DwaeksRwN{Q3QpHefjAVSTln_n7KitQ--d2jbpn4%4w-;m5noz4_Wh zia3s`LaleeqC6Y(c5?aAdZezclyqB3f>S`ZWsB1D!7YMR6q4O0mC4s1Iktz`RT@AZ zO!J=CDs(C(bku{Wa0VkO!dbeH-UZ)&1Q5MTaXXPfsnsq2&=s*WS=bjQ{3&W!y8$8U zxtu88VJK0qA-Cz6IHVQK&N@jT?D}DAnE1|w+%D={5GH2Wd|X-9kAjuJC~P^|#}*X~ zzLcuw=0O*HsAx@g@3D&Ta_=Dx>JZ@a-EDeuV(RE2M@!BL}SQzH5XZ4wQeE)35=fS{s6rx~ui9Vn;Ns_-gR4AUL9OKO4YH{?Ip3m zt>dC0T=!@(wbedn<)CopJiKuI&B3qM6IbepTQYF7XL>}~YMSL^%_exEn5o^yfUbHE zu70)djY$IacH4pu!UDTK&S*TVS!8iCqNqfCS+ebF!F}puS)%;>_%>Vg4ttG>dg|%^ zdgn`$;JdUw(U&&qT(t07?*`B{J9Rx_;og^lJ!6>%ImTgGlF^C-4Zw7!QF`g`EBVse(E&eMb0eEE?#h{cp2uY` z?nfAju3eT?tZa)``TOpWL{cDu3(xPy`4x0+w-C7l@Stt|1NNtxxfJ(hM#UVTCfdwHfdk`!D0}|b zxM`3vX_pDp?C5_V3xRz2p<-`oDp<; zhlovEa%dotxuV14sj#NqUF-eoPpLEdT_jim*Y@Givtq2MsFQ1n4jwd?*hNR3%0skrh-rj==ZRlP%a1NqV1aID`Oi=r)i{AB2*f`rB*Qvy3C$i@owzN1VQyMCP#!7a!6P`Ba5%(0}weX*9abF5g?iRhR zE?GQR&I!csH-89kNs634nlc@1tx~t@SC`Dx6HVJgCgu=1yuUVdbOiP@d6MGdhPQLA zd$Sd$=(v}w%nOU!6&oLnA%)0%VJ~8tlKIV!=qJsQu#|FncD0%e;#;_v8FXLD6Q-{l z3?>QAlmxIdAO5LoRzxwup8dKL_S|5%#dmJ+$Ja!$5M`eh(@0=^$4{fZ zo`E0ShDdpv!CoJAN_dm=P+*fyTr!7gAcb_eJhhp6b*m|%H@#}n^G)0ztW?}UKsiCK zU?V9vn&EezMPg8-b05ueIuD(>?5qYF4w@0c%c7DtE#u{{-5N9fT@h;hMc}J()SVk>XcPlEncq6RhtOz)7fy;$Pis_msN zQ66oKvC?^Z1 zPc)7LC`IWnrxwJKKD)Jhe!QzoQ;N1$R4QL(?l1dg1)`LUIaHi({grYy2Op%}Us+cb z*eK$}HEXn6nXF_{6E5DkJ-lKwJ;4$@@3rUeZGsQf1fEnI)Tf=!SMEW;6xnonAeO!` z$K)F61@>B6X&0{}nZ@_3wnYXR9+uS})xGr=5s!yIK-Y)zF(> z`3;Okwz)$!L_MfxM$>Da$04;_(B^k;J+t{u7drC;Jrj>DtG2Jx-Vf@Gw~{V&(rl|d zyX6HWwZ=?6wPcjMJ89B&+BXXE=ejbXE2O~^EnCM$ps&n9%cU%~DRZEM9=3hyipKAH z_0o&=M=akc^IaKnC`Q+tS=!z(i59yID}e)F>4w43AY&KfrEd$FLU*t*xYTmQ>oofe z)GzNlohYQ^diSB$7|`Bbf;NX!BA%^xy&up27WAY;jgDIJb3AdC`flY=qGW%n^WS26 zLA(=`g{;kE?55I^_0PAM#l^={w%*FrGJeGz6iFf9=ozvqi5=6Mg+XMqA_i*NdNC zzY3kikIdS=&AYm2DEn8B2M*Kw8%A2ka|PSMcYoGr@WTmv20@1$)<#+x^ygOuTAk+# zkjl6(B}1?Ol)>QrSI|jC{C=V18aRCa*9X6p^g?|7rjM26U+u2|By=8&0Hp6A)1Tt( z>G0nm$m@Tj;oT(v@5OsMymwxpCw`|FKKu4x$Nh~493MJmkK4Zv?+E^RTE`JQGFzkg zzs?4o67<^{>v3WKIy_R_3Rxl>v_LO%&njzq;@?auF}YPI8vsy_f$w zn>SJ5_`XGa6#CcUIYGnA>pMZi#%ojj>uml%z1XAt-b%(JE@65!e;tum;xlYmZp6O| zj(;j(CvcZnB)*LN>vD-eeu7jeVR_%`v8b)_xd{OoIVv(&_H9p&;DjX~D8_+oZY}+Q zod2({1{#z%jRGzxo*1BDQDdH(>F&W3|G+iV1gA%;^%VuTBpCdaG$NaW?xb%3l zi~7{j3*1_unogI5WO>|}^4gw#15m3}e@rn=p_)5Am=9k%RtrUg{?#L2v-oS#wMX{) z;I;73sqG})$q!69@wqBfH@Y=Zs*|Y)xLIP|UE<>l8W}$Oeu`!gtD}B%zR&1~gnd^1 z4KCrJW+3~#Cls50&|v|-i80kmkdB?3=zG@jTHly#SI1x{^q)g~o{Vti?w`K9D@b9J z@utFQ6lbjBExYq}w&WOalh>&E{5^@sS@A2fyOOhBbSp?EU^^Yx7|WO}Y(P?|m*muF z_3%xcF4n4D?+g=)0eFVQ)ZSyH5u@e+Kp8kahLfDfnzsll04$rYRGQu-xI@IrXa+=v z>&)_R#$WSZJ@|7bJPf?ml}K%YI%_#KU#vO+_V|6vo+RHx0K{KsISxwb>_;!rhSxsY z5pqUu1;oz1BrlTk1}p&j({NLMZ|&It<#w z&555u`s<>?t7hRMjY5)O?z-+!BCV--n}^$81xH@ocU5p}!Iyf!VuKcJ_&w(HEr1cH zf{x|5&TgJcu1A?;7lE-O#P>RcI$|0dKBn(`zN)v%{RP6qy@jdwbo`;(F-mF&2L{(s z@d1BUEK+=oqr#47N|B%q>7y@Ijs&eYgSumR;j<0>wQd>w(171DsWk}_Zk_=|^fer2 z>be()!ldVeU$7!&db7fEdylx^iMK<-WC6T(BUaZVI*dwYs_vthO`AcrXeX8J$qxyM ziwPy$Uw1I|pRm4D$>2*o?Uuj^pUvc_ybKAq?0bnDWyR)^X!hLokyN>-zFy@67oH!) zrcjN{KEFfEeohI%uOxx*=TrCp$*us8U3e0K!$JNDAdre>obEdW$!^saThld`!cwK& zHwlX|#)cw~EfL_uu0bUGdi_?4O17S_0&|t4f<16$YcmEW&PAWlJB`5Z9c5;M{aRryBY_f;{E=u2swA6@7OU$|0gd z2>>!ApOB?o1x>MF!s25xo2lO0yXp@kc*V!NOw7GgE!l^5`$eIDmh=oSuYs@+RK+FA z`KdNCaaZZnibFrN)`t3P2Z~H~!3-R8!E`7~zH_Qrt5jd3LU7A$yv7$K{N7?}^V)6M z#V5l}|8`uOtTL+D4!vyBIO=*OV|TPd7IL#WkA2f*JaTvw%v||8O&z!bEzgDm^_yxRPWZW4V46RkHqsY6B$_z(zSH&&$lkaZl8mOj_GV4;Qr^cH1szD z0QZVL&i*N2q(-VvEZ}7RXj~~YeO4nUHfvkkGc;pe$!7giKI9$Y4;u0t3oQ6ViW%(v zBKh(_ehU!L0RRXOlQ<%{HGuC$Jzy_rClQM`xOVn%waF6gbJ8 z&z_VSdI^ptE>ef|#LnjbWH}?J7(VzxwvZ%XNA)GHv0bfX@w3Ej>3a>>qKP&Qt<+$PTkrREON?(ye%c0x zXzt$FJk~d2AN_qZ$h?PytsIki9fu9w!nHjX^~Jxx^efnily?9+Pl%Ii)ztO0 zuBRzaiyou_<=a5g$M5Qnd&ij6vRMiK50lM!mvz=@#m7JDDw~mnC_Sr3LvO=don_tE#K)MG|s2 z9VYjHX#)MVox1vq|5N}XSQvHuk!P&YOOs~F8&T@4HU~1uM7-mVw_a=6}Z?vQM6hR@eaQ=}Et?Oq(NIf|rw>%;l3&QE! zG;vMC_8d909cU7(z+jn=rRg>`rg!d^N^FszX(Qi8Lq@qjq!K978q@V9GnK1O7OBH- ztMgSdEMtE*IGkyyH~gl6oA#!S`NKdu(!(c=fj{M>Hx1c}njZusgL;`4gGPh2Y!VN8 zQ9HlG9n}>UOA8cPfVRRgKv%MwDz(0IY2)Jwn23~x*>TQmOhwU-pWBRg+xuWTi-qB* zk2r8H=7i)yerM2NOJ?ZQpO)0q``w5GOu?ejgq?J2?yDr!dX0xtGBwIGLeZiiuSDAp zI}qLIZ!Vl)d)(S5Mlqdd1QT$#i)~NNJ{YGsi**8s0XJA7ek|K(BcS(nK@=_LJb>sN zjs4g79GuHAF(NF0;bZ1l7Q$R|JCw)aOA3fgQsIb3Q!cA(iD;Uw199Jhf{@_ zc{HnR*1op)m!ah=2^ZI1$@qbpP<-_DESF>{UdT&GjV6P9JwqY)o3k6jlv1wJ8pYF> zN$W~g;^^^$#-F~;>c6t5Bo#?vHxghmew8f8H<+BjWm=RAo9+eN&dhn=g%L**wch6p zcOHkVS`Wl6hgXC7P?96!PZA3INW^rY+w*L|LKCd1BWLax>l{ z?dHqVl)9X&`Ctc)hXe39TXwX2`qj>4F?pEqLtReacfm$J#!egF6g)MWXxVn2HI71n zTu6*}hZuA@(x0Ev5jO7xed(HwWy_$Gx!zLjuSm`{_3mc*TlhC(QyEDZmeH|~dvmUA~9|DdZiGJTfly^BjV{fYW$&-w(9;jVFuX>KLq zNHYA;-tlCyIgW~$YO$tesQA57!{>uM#5eEl5+ZU5*>azu5D2|eTSr#|5z2NEqvi(& zlS8J8JVsc*rl=mrHl)~W*ZFN%RzzjsD0defePy9erabm2f=RelgQ2Q#;Wgs?LY&O& z1k6ud0{2Us=dQwUgnlL+e7+deP?5BleDKBb6JJa}1AmzIcR8+mMjl(vQA`^{{o9u(gG!>3X~r&p&uu@G13)1sbC3(t z;h-m!C>AW_`R>h?r{4ymDE*0v=JG$fCRFl2s!uB=ykkXgvH2bIb^jF+mglb$yE1mcL8sN8Fy^uW$9^zMI8)2f zuYgX4%}pD5#Gm&l6z5rMswSZl)=NFjSu(su-mh79k>>k*Td{O3}Mwqnx1f|nB`a&_&4lko2BIK@H zVThFfjipnN*FXN35S93q)BAS+zwlkwkT$5qb`c4$?cm%1#m{iK!lG{Xqt*JI%VRk) z%xbD%2{4Ja6pHlE^-GUC+e(QuA0vg5o^QV4D>@-Ia2v~#px~36AfX_blKHb}JvE-3 z5-zyU{3hTCivv|o%l39(_oHFH)Ez8*)jwkWulD;J9pvQ@1#JP}bE5u`qDLs;QNR*t z0rj6v@(FEFhrWB@N7h&Ab^6yUNxkD<1JGg+kG>}APh|=2zyUu194?7?>Hpo(|G)HX z(of2q_C4O1Z0kXm72Ch<>#cWLi>Wk{2oweGF;9$qGa|U{cJc}mHvWaXJVUq&5Q*Aj zIZHTh#v(k8WDwkou&X7GoyU+8u6w8cFq*g#5-)s}kA&vQ3i!M@u>;uq{2<7W3Xj|4 zRh1Pr!QW8J3O__EJ%}jme!xmYd?h~Q1$lfBL+?}dD>^=PI0C>014Sdo{@mtOGLMUn zg;jFECccvoC(gTR=}Dt44D8! zq;{iYW90+j6DVdH5ycC-mV9Tc$DNQ&`i+s_37fd@BJC0eD1FLgKQCJp}SaqTt-BS9_NJH70- z%halUE4yk5*PcxKB@txV4^UOwdLEBa%6MNq4Y#k4GQ+X~F?;R;Ft6Zm(Biq=zjBz5 zY98+6%n5$nG`77uTAssM(Aqv1)7@UkPIUjQ%P4n5z~K}(Wq`M!sQTM`7DRePcka7F_zK&}m#zmK)V zGU`eUywTOKc>AwAg%9bWg1fS^hed-lp0E)+#;YifcN)JW@F*zRnUgu4r17fx(RTDh zUjk*&=bafHkXucLgtL^KwcO&aRdZLR0^H{rL9ZI|f-#|Qo834I7-qLEE9{420IVPS&LUO<+EMSb35Hf zvTXBe}G%Rhw-*vUG3)?0QI}=~Tj>n)R5$c*CRcU$!R`wDljHYD^}(gV)}Mye47R zMV+fpI6KC|d)S1hRZsIe+dFQ!I~gcF8O`d`*)=Bpd%;^rxH^xbaK(?XRISX|-Y>qL zD^hp4HW*(%*h1< z_-5~&cd49D`l#(+HuTaE30o~QfhH~LE6UDx&y?rh468=X9RVJv?HAc^th8T|i7s+< za-~zSJ}190K&`gv#Ft3m$Uobe)lRwBCDy+7s5irw@huf>^oXWTae9@imKc38l<2EE zy@>GANvX&LUE^-@RR9Rq27wEzCq?fmg&&)=`=c2#jRJh(P{OkZTDTga@7GIPFA$^r zz%l>7?%mml&7?lInAeOQ&kxg%~OC$TwfqIefHdd$|3E?IcjzaV4iw%T@=q} z1fZ0O#jn0_X!)xZdJlw-#8pI%$h%XP&(F{T6_YhN?UNP*22qj<3R@LrmWbFTef7Ix zPOCr(6g^XZU+~CRxFPuGLJL$t9}HJzKJykHIR`0!$+9m&gkIhzw~9_dQ4X*9bTJaw zFMC;hr7ZE1=8hluPxk-KAI=vLsY;jQ5gb7R%SMs{i@=aM3to4@MVzCUWXsm6Sq-3+ zNZD@QrP3k6dqFAz9{lxjk4%kvv!)>gn);%!r*SN)dIr3J$aT59BuFTWd2S+Y@Af@6 z#YJ)n@4n{q&VmZ{sTB&z3$q&I)S>i#NDPh00-#5MuM-_k;m;*;Kf3anRbPD*U@&up zj&HZG`Sa7*!~Ly_{)5d3g!1KdLmqnVdapP7BKV)0y^@rP0L%Tc92e~{6mlJ;Z78z zk|rCdQmca>OW;C^7a>6Q&i4dQ!;uv2GyyC+#rN18G+H>rx)W?G^UpMzej_=1cEFi^ zf;nZPu>cdkpFD1}YokLw$TT}N#$P{Su~i3 zekohsPW?ft?0qk^x33SY)VIf?4iK>X!9**n$XK!dwXkIsWeYd;>HO|0c+91zId#^@ zC3iWY6vA~Uq+DH|#T=PLF}3w%LlBOlvVvoMYLw=>ifF8JrN-mPljMA`)l3KJ?cH$t zOT$W}O~tv{m3jY?>ti=G^^Tk|biBHA&bb>bl6W6RSGLADRm$k4fN^iQjw9Mi>l1-j z@7VfH&u|;}K&*JX4~()+Zyax<>lSn)jr(mAnKqwe)KqCrB-rT1fz@U;GL0ikamF7S z!!v|8f=|2g70lMg=DS(zoA1~(D5^GP`WN_G?>E&_gJyf4mzLH0SFxgY#}#RTS(zUl z@<5CyBft-8+J$u2x&sP5P>-~E9PYl*QF5JXk&E>WYd5YpJ1&10boH3llXm&#r3Lt@ zD!49}`_wW7+!5Hz#mdHO9`3$`E|z$|nme}V5^hKjmHJX5oVpii1tJ)=UdV(dMK417o{nsvs z2)I9=FPEIWj&~_HK*(1uE@<1*|6Nv_<#Er4LG3^|+iV=BpxR$26Mli$vjr&qyF7C@ z*7)P0%N~(iDXyxk1YN&kSjl`LA;y`@{7EFq*3RCyxbd=kY#A=M+>h1SgtO@ntVI9) zwGoEy;3h!H9BztNlDfVw+z$!Db)u!L6GnYtbPHy~ zjd>lamvtPR8)Cf^nqZHVr`qI(n%!`H-fDYW z9wlUrNA>3RjEFQ_Dw@qm5P_4Zg@N4BtG=CSxGJASs%N-V!e3UK15tKp2F9yi9=MW+ zkl*C`e0*PPHO~O3$ZE1Yh6p6^gROzB@`DqX8Bw6{4{LkNGl$?UF$U~H4Ay%c&)~Jc zw6FhObbPm_`V|?UW@{ul*4x*A2TtF5E~n!e2G*b2{7Bea-N?sb`fIP;#u`r2ai+y^ zGOY3FjD}YQOO7G0@ex!#8b=O)u3Rct&O6g)*@)NXSbIk{U(6CF@+ZA{s;VyeQBnJ? zDP+S*y_XBS>FsXL@apv)Ug$ZjV_jLV_%1%9k5S_5;!-bVk@B(Q!9tU(H}LRX z)m>xuDL=;gL1vW`jORPK5g+=mG1@QcGQ(Tuh)Ke^(>PvLVM#1l4usZ~U=qX5Lx!15 zrC%$^sW5R&*A5smN|4%ZFB0yB{f`b_6h@lXe2tj2z zgcLx&g|spET-eKHDvNs;-L~M2^i)j0+VjxUOG^^4r~7gBj&}8FzV)J5Cz_~rKltR+ zBVa~N*#inMO=%GJe^<4qCGDww5lXHU(J_B4`21;J5ply)Cq*{8!%cC=eaoR3IyF9Z zB$kuP(rF#+JOHVGPOH@tA&^Yfj76*Zs-fw^#VPP8!2uA2PtvHkpTI?6TOaeO^zvw~ zCJy(T1mDdUgo1&MhFm0|VElQtq3dZU_udI|^B=zP*CH0s_lTWrFf)M82vx3JC_9f#GEbiIOB<-|8ImTH?7dF!h5$LOaBLxLOlq1mvydY>@}IiVt0q z@hr!gdWI+7|5#iUDYR&!; zWVqo^G9|eoiEVh6SDS4Zv>Cx=iLYDXpO9lz>{4OjyiQe`uKHYZ-G})_|Np7$tfQh_ z!>+F!LO??44rKrVX(<6Il?LhV1_@Ui-@dl*dknKV1p5l;P{_;*6HvaEi zLc3sH%I_mur&}3XH&hU0Fn4dD3gk!?E*HqCsnUI>zRRl@>oU~4Lla#0_q6HK1f8|l z*w4f-_uELCiAsJ&ECLjYZN2g4IwngxMy9%0&-5TR&uOJ|Xd~F)?jLIX4hHgp1QI$4 zEog^9`K%LFVpdHUZl3pQQ5xMxtAADxP(0VNgrtH|E!S5Ume4|NKARyyO=S@DOaKPk zFh?QRDL|#2R=a+K*QfL8U)IO~QS;lC@W-Ti8J^^fYW^5s&K#5@WYv-ao_Z+tuH1b( z7$My*Nas0Q+}hVvHmJPt=(~|Qp3+tX*}N6NpN~GBYf$P(pkZg9QV8?9Zv?Uem-u&JWxQN}Jo}dQkwzXQz3*Zs^(x)xI!>BKF)B1J-`acU znJ){HBkpXM7t67S_7bz}=P!qfEFE@*DbAYBes`7?L2Q`JHj4}W5}^|Ho}r?d3K#9b zItJW;1UepBu}KK@lO8ae$6W62FIsJbM%pMT>k(I}k`o)?9z=5(iEmzLTO@M3q>WiE z?~IOfR&4H8LW`2zYoW7(6s;_O?r+cTG9ZO}-2OJK)l)L5p%ygLj$&b>oJKcpCFEwY z)RwS~bez#|fXlYJ8GPXh zr5L`)(H43r8YmSYqc#$j1v>c-dbu}O>K)-d0NJvd10{4eQ{Rs%q!)^wSGIXX{=9-4#ie;o2j=D9yYiKT`1Djh#*>wdfRZLxb!WgSzks+ROP*E8gXP|C? z1{;ojtk=zP*^%l4U_>snZk{})%oIYVj|MqOp6b$|E;y@!K*@6lRtcZbs=7HS7%4H( zSZWP_aCLzPUbhcE6ZMaIN;))b!jEaf`UwVd-+>%66GD-Op4E^@y zs_AF41lihgbN}3;%M&;s+!;#^H^r;za;(YW7Is~b&De}k&8hR<)YT;siNGWBx4?iM zVER}1ZRJ5}Z=bRMSth|Zo;Pga^%xF%DIbINQUsOInhZd?6SVB`I5(TbB=ms8OTtR+ zhK@s>WAxkKRogP$cjain)~?dBf_CiARxAGK^`E3NkHx?N5s>&HV*&uZ`qMWrvRBSrVM` zE9Vz5iqBbSan;t9Hm%d2-|8wUfOVBFs8?|;2bV?zck<;mu#WXe*@KmzR1}spXfH)n z^DU^Q!PV6LX0{uc_$jEOna#uhqWwrXHgw;FPqD?g39=NiU7 zWu$&MYjn~cQSEhds<~MIMXNTAZO$x~#;(AOIIXHdsV)|K_f5CGkc+h&I2^-#MtwXL#oq{x{8g!;5{;t)BEa~& z$8CM}G}k3e4;uu^&n>T)SLK+~)iw{{6mLpE`nQ!z*gjbKh4?K5i?-x-> znQ)ID1EsIL);FnDU^_Sh!GU@K5oC5Xn=9h@<1-U&!f5`>j}T{TdGiq*P)hNsxa>#q zJX0dM_r@Tdb6}&>YUtEzkZKOj>A2>iaK~%$$h}$Hq5C}u3x+zow!MVIRB-_V@AJ4l z!Il10*?G|&hi1>aa6D*9%F;He;Nb%G@}ei7@bbCbA59#e`wrMtzqIqVSlrkraHo%E zpA~iA@z3%WRD($aRW#G8&`lXeR`-x}u(2dbx`~*wY7<5)f8p>cA(no;7)@TCU^emS zfil?k^A@Fcy7!SrJS+cRl;;6rF~7e429`jlK?nsOLgIbQYr`JA;cW2$EG&gWUs65w zlG0_+GVz%#U6l*ktM`T;cJ=K3!+thKX}_~+2Ok5YZnHK)^mRa;{{k_r?Cz*9^kg&VXdr}ujl*4GRsC7sND_Y_0rXcQm)TJg zA#{S@V}9C{qM(s2i7zHcp1h~lq$4lf`=-GwOtCdaJ3du)`SugUM+2YPOqShm9s9ZX z6uEd&XGu1Yt_ih`x%z_ISR_>hDr20TYdmYp_8biCbL0|>rTOf188zL4Vz>~vJH;5D z_Xi_!w!QC|52Vpap!ZqyX3o^!6;1x?YshE>N`BpGhDsEurli4sgTbd^eb&w~p4WNm zEi=e{_a?u7h^I7- z0%$}qCd^6|u6%v@X50L$i-1c_gm~f?SGhe_Q1&jSy|Co=((?$<4*uPN#urA891~@mR*6pw9z9R(VK&0TfWUXD$Z6$Tj1~hBrxvQ z@RZhAC^zeQMA8>k$dT(-B+J+p&-Nj{yeUNx^~Df_EIf@Lm0_u%O}5wC5!aUtOI?%B z!CgN3VTeMSMlwM}+CXv#>^?($UHg!LN&DR`*W`XVSf*+i6`%fa9Ef$Sh4+rr_#l`y zji~vcVX-Va?Rq1CK*oaf@LTiJWBo-LL^lxN-@UaV{B}ZT3qRbkS#pGC*>JmwNCpB$HZK0dwkP-ZG=C8Y)(n1M9BQlZjdv%N$-KD=+T230Y@-v_~M{> zg)E=PU-e;Z0A5b-M`C&MI|pasP*7&5vF`M{z&e?5v)DHB2SfZ~-F-a8MVH(9tVCE;*n z0rV-0*f|{`=zySYOM5tZJ|>R$M6qdJ*_3yy(J3m+(MLtmWF)QbPrG7RrS4Bk>fohl|tvtxn!)8UCSfF&HJ^$E`zF&~OH01r=nJdkpt4a$*KT3pxeEQ=~F=7#H4vCh2l-K|+27_Rw zq6s>FkG+&8p1)W&6`HpMQ`k~Z&Yn;|K>%S+H3lN|^{bbu?N~u!1 zQX_hv^rvf3mWLW#w{P(ic7Gu>=;*$1^Q_#!u1=(!TD8&HpQim4u334I{ADL1KD@CU zB_LEf;J|G4Y9B7;=2Zxo=>Po$LT8ojjZ_&q-@A@pT_L2>LH0-KfGw+3k~la74uI*C z>=$+A3pab^lJ|qJQR7*)ARXaL7>9zeJB!tjC1GO4v63qhaV!FbFD-avj}0#3E@Ml+ zd@_t^8l9!BAll9S_XhqHaT)D?%on5lTUcQianrwa72>jQ&H<< zfblriC5=c`$l0Hla$|5m8~N9Y3QhzpcWbYsNdb)*3cofHM0@d=!4*~cYtKKIk@v#| zRnkvm&!!{8&6OYiV!FTZ=;|~7)(-dIyAMeKa|kmqyE}&${a=R*h%~{hz!HvP=tTVU zyO3Y}!Y2R>IzHax+UI|s)W2n$C?tbkhMJiE=V<;urqsqr)n=*LZ2VuBAAEzh49TF! z(;ohtR0pPYlY|^#&?TOJ+adq6HTn&AOU465sPq2+qanEl1EAhD7$ZUNUjz!NEOjD* ze=WaW#s9oKJWlXt@b(wVZ2t3Z@U@V@zwF2J{~+=UbPxdi(^2a}!~gSc=w*R-Zi>kz z`k$Bgv=s^bx4!Qw{tr?A%oGXyvx==T{s)l{f`K;^A3Nkg+uTuCQT#@0r^UldK-5jEb!icUi>o@yoh%OE~G7=Su=W+SU(2QJ0Z>- z9cy%>#5sXst974TyiJrcbBrWe&%Xcf9G=P_m zx6nLW2(Et76&RlEnJ@4Z9-xTEb??tf%#HM8JvXNZtp~c2s2R}4gF6A#w$jx6p37|O z$ic#(>}}0Rv5M%-IU+Ulx6X6Uw`Y8~*7ZR6yGzU`snBmfT$gE3t|{Nh(a%#0Oyvej ztqC9J`C)wA04u#RpPSnyyrSK32iH05l3%2Mkbt3-CNDM8(sGWKh)<W{FIS!u9P!c^ZX(*5qKWj#{(0E~1i6GKK=W!OH`>ejofA>*8K7p`S<52#w~XTD{E z5P%MQiE0U(R{4#hhEl4zux2)V;+!{;?*;cVK{nE?B*-1%604BcImYP6{k|33F*5d z&DJYg(H5mAQzqM^VVR!tz;Z{@_{g+Gs`c<1+f*RXc79L=CJ$b`J8QT??}WL`4|{)N zo7hj}awh+z+En-^;q}4(O5TC-rc*SFqW9)(Zli83v=4#_l}c~wWd>v5H9^)M#4cuO z&~?R)?@z%@q-F%xlB*F}qo2byrC-zzm4L^y(jMtH%~;_m-W<)qiQqK#l*a}5%gk@W zO1ylO@TT6o+_pnu=Rl$fm6{beaF!O%y&~=ExGM}l$R08mzM3h2GW<>YO`i*3@x4yF z<4dy6n}%zC1UiBa#Gp$!>guxIp)*O_q{V;>>28P6>Ea6-*)|!q%3Uxp=H3wvm)*r( ztFcd&<#)ttEqW1hrN)sB3v)66JKqvAOW)-!Z1m`%5E5b2uFv0Xdfat!OksNuMFAPb zH;HJ}sN++>3=!SJ-K_rxif2z|%U%Fj0B!#yV)83=tjFW;i#+%->K<5KrlNJ0uKLJG z5(cu9u*DZ@mAwSvb->;dHuu`2*w4j+_<9`ZDy^9W^RAvVF2Q(OeGa6hfu!HQiUjEg zl3y6Ng)%f=f0rQVv8Bhrql&Jw)0SJs76~c<{wzk&*;@LOWXMXp58q15Ikwa6QMPo<^PjmciZyTdqwkXQs2=(sAJ4oQ z=tk!Dan?-C_Ls8DJO<#=!X8kHtZ{G>p@OtG9qh~Nn^g@vS)l5W0aT;x`BRwKW|A#o z;3GBt`y*GI7Ue6SrDN`SMVj`)#BzMNR37WsZA2ssUJ>lNfNSJ(HK_ez7Uz7`t_3or z;YPEB9Xl{x{aO|Tdn@;yuqq6=(qao*wrXDV1+MJ`K99O<Mr z1px}wHS!mrwR7X`;xKnw7H2(nzIJ4(^{ff%$34>E^?AU^%ia#gp0TT1b*8Z;kL2me z##0;=qkuRTF%zw@N8RH_SR>Kpb(5(!tCr%!DA3@j?Cb?spF7e8-tZ@QRuouq7EVbWM0@XfpHq=UOu z-T;u{owH@G!g}PYX4?{_zXX!`+{WRK(e7JTFno;+_*?aws=c?LtIM;>uhZ2_f6;hY z+@4G%;Z=1^I}@5n3XpCM6x)xN*6Omn{e{#vq*H#-BJ<5eg$-m)VWaFaIfC$iYoqw7Q7V^bU?1Ws=F+6a$g)6;i zz=#o-20$iRUj@GhPN-D+zP=kl7~R}GMKZ3p-_|ajJM75=_KT6ZhDqcUg}FB=y{(g} zYXM8yNA->v{I9=5x4j4EBg0QTgZ3j-!5BU>$WBr#)3|o^RjpY1kVx(Vyrm)Xx!ag) z-2f?GSR&t7pGdl^*;=>A^o9P{gcNqza}0O7OOAf_?77ZluswJlflF*o^Yq-qgME;1 zz1DBP9kw_hH8{tqgHiJLfBjbmrtP&`?RHpVbrO}c2B6}acJ9z@=e{d&e*5@Rzg2-6 ziF6dVY80y~_mStGG!X~57vd}sJWvn#`hHQqWlQ2|NB7Ngr08QQ zERy)q0-rnTvE^4_64g+g&e4EM6U~!fmK3I-XBa8EGiwxwxnb$F^q7`#GRoRQ?E-OV zGKC&s6Qc?wgC0GD{CZs}jfK94N`~!AJd0Sh4D<~RxhG!Yi4$gi6E2S&mn1L3Mjcs@ z!#g0(DkDeRl+Yx~9J80@z;_C#IqZMgb?T{KW7<2-(Evgyh9PT^tRI%9=;j2b8?D@BVxJiCaElO`e> zv%v%hE5V%+#NmLb;2b+43%Rkrq<`CdBAJkxq0iQcl~-ff{f85yE%V2KMWVlo5ZGS& z1Le%%p=3^bNS2~)hxzy&YcSa%i0KPhGsq6t*aydAAE}6F77(*)G7+($;$g?rhGEE& z0%MPaRSgg&M?Po$m^!7XnNOQ~GnJy21T)_+IWBsBObj2oFVW@9*IgLAC2_DK7SmbF zNiXEG`^w{21(?B-DUKFyH3%o)wV4F8gL>^&U%Sig7zUV2Gf$5S{m&pUjG;g)!9weV z4C@xExvHXQYBzdq37DJz(fbM`2~+JI*se>wzi=LnZBM23PY&idhr55I5(c7JoomR* z3`UL?2y92QuMI>w!W9W&U*&F016lPD+0zd(Hzi`exGHASLg7QQarqUSKfkzrD90&1 zhaPn-vKoC>Nyl3qX(XdB?42A~I?JK_<@KYnE z->SZauPGKt#l2IyFRH@=ZuV*xE1U_>$zRN7D9Jq*Cm1-h)hf?l=J2Lyx*niXh=9on zY4}R843ky6c5&%RBh1h5GlQVUBV00W3zw3LsR}yz1C{K%+93fd5Zm-r;8VQynyGM1 zH9UsCPdpkgn%_t(OLa>*)$HoPxF*xAu^XyOo}#SHs?5d!cGLxSQ;$Xt=~Mt zRX-Khc*5z{BE#t&(|l(2Qf-;Xvl+&}(v_;i$j5{ok-5Lr7BgVp2PJ0HT%E7APd}kD z^ALI9I1CA&Wkie|EJ=p)7#-iX;t2U+#f2RAb8dc$5z~l{v-O~xjb98sq5^1lEkWK! zEw+prLVcvonS>Fwn&;0A93>gj79VcQmx^O6XFs-y=SXy=IJ6$k&u`%Y3f-$lZ>kUi zp$Nz8ITJ{(Sy_y$aAzm2#t4?RRLve)5dgRX1t;6St`pr#E2HA~9x_%k)<;b^lbJ&PlzaRIp z$a2>Y+UtHH&JW8#*qMp|#u9i0Zgc;zhms^H$~hbv@0G~M$g`XGxFjov`76-~_;btN zl{u|6=2i|8QXAaTcg9&bLETjQLwkpIu`Rs-kutCEGY!v3g{E%|HTHG?{ zp6C0X&zH7JR8;~pvl2oMtjV?f9dPL<$=ykMlN6fAI_7G1h2z+=^CFNPqh1uglZ8a*O`&ioe7W zU>XRM=#0l$5QhP3M~omEqkNn@=LRZ|g{$Rh9E%!PuBE(+)UC~5E`E)UcXqBEt;dj3 zYd~qHz~r45{U@73O$aX}btSfD_ws`aGuUBe{g}s}BNHlyNBRt0!7-ubrH6h7G#%^N z34YgC&qQ8YBJRO=>q(JOB2gE8F#IE*ChPs?!`N?Nfs^&e6M6PKHS@%ayZAJ76{I^8 zeqB4gY3W#&X}X;A5grSA1eZpka81EXf2=o!&nWpzr}RbP6pG<^xCvx9OQERSA188y zhyb5IMx1jW!r|*w^E${n7q3J8jd8#5lN+F14pQCU;c~@NhjZSmh!GpGK)m)9AbKu> zSn#S&47VUV5y%o*s77&k$Sh~Nun^+iBH4`<@x}iKAEfB^ZH?8cO^axulZ_`h#R}XU zPfgY_x+!C+3Q>a~X>JlGeoo>Phi{)dhSG>O)C$5hjl?5dI;U!5JL;mP)SwN8zVZl)%{@9W3M_|6g%uN@!ZRg~NX@6onDc*g5&!NyV5He;! zU%>+l7E&zzY9%p6_N;DQvQxyw_$EPs3PUt2pvq`fm=&`CuhV`Y6EBZ--MZ9*yysR- zdt)T(RXm!ULdz>+rHDS8p1a|gKm9Fwjef=myn)1Q>o=QTd9z?!qO}hV3*5|&x#1xR z>q|loI5V}bB>;;MuWHzRPDwRvUauFuvWExDlMvs0oWrKfco&)Le-h!Av-(Htd}@i5 ziL{9Zn9h_w(fHQ4dz0tzz`_(S9K^FA^=J1uZN}k6U|2HbUdqZ!K@%p$H93FMaUWmM zMN;_YOIkS;7;HHe&*mOouOIA>YD{f};thg3{sQg6#XkyL_V^{M(O;}rN%5)M;{7WV zO7h;$m3}o?5+&sv&MK;nA6(pxvLFe5>_fB3gMX#bh1>UlxcaWCT>HP`ASP1g^;b5g h_^(X)|NZG}JQ*9(h|5C!>RaGLT1r8(SlrO>e*mYIdn^C| literal 120619 zcmd?R^;cD0_dZO6poEl2qjU*KcS(0iD@b>P!~v03IwTL>A>AdNhwkp~j>C6*KhNWh zpE2IQ;Qir@amL!Q)|zYWYhLr3bMGKUd5LGJFHvD&V4g`yiYdduz#{{{_b5m}ON4iX z6$}ilo|&kqqLip8nWDXov6-b23=B<#Q+T6{$VZ$2(!MvnX_>8Q!e0VgevPG;mMPfQ zm=1S2H(q1?&@JXnCpR_aq|us9ukFZ6OY`?9LqScLxq*j^3!ZhWR1O*JTjsmtYCa`F zuW)r5?XUOw{>6&Ym>Ry1z5o`2w(mINlW09plYpIo7l+qm;h;)loA+5T)0om)o-b>m z2|DDueKB|0a9Q0<%QG<|F1l0yY3vOnp$_>+t62_QdQcdp%o9@D!HFwbyZlc4@f1HX z*(>>)zV2;{Uyxs|s=g^!OW*Hek1Ks=7802lsIqP--Qx`sY8qgq-u@DM@$p+Nh;KNb zx|52E_kCd$WdP&B6D8S`OutRq4{u}@xnw2f)T*JLZFRBr{=@Xwz50`^0gA;!mtMMr zQ;3UCbx3~HWj*LjsM+DOMc97l?Hh95<0_F^BKR66(d9~AArMr3@?@pP5b=5a z4jEu@E~EES#&U8nbijWU7}x+a7zE%SEbw~?{9s_v6BpX%$n6^zOV@ZCzXhGiz?`ricR^YU*c~wdNYR za4wa#W^5K{>n%rK%O!rTS{A&MRyQ_r0n3fLxfpYxL!r>tuL!UxLNIWc-Y^JcFt8|h zKjCtibzV=(67(Dg;I;&b{(A^Giw6z`-I41J{G&^JNHt(uTf^M)tiMFl3=y+dKXz!RhQKN^J4QWvg}?a`{9dlL6Qqx$bLM+8iw zEnAO=cwUzSc%DPZ~ zD)k~Q(HKyXv@a4y#?l85-LA_wT>`(S`}@W}ZD2!ueHY!&)6NKV*A)Unbw$jawr50( z>DAL6bDiD`ogK{c^4R^-hR!v*GxtXFx!>sU4|autfezgvBr|nZyl!AV@$!v+wK#1H ziZ@4Sx=r@tP>-84shp`{!_Hy9khrZyM6~VR@dBRR$Hv3-`rR|L{+F>v>xHtIP*TCP z!P9nX&^&7Lpme{OyU$8Th?4(P0vf%B8r7m58+Ac2uZzc4T(H_21~S>qSfM61w{~tg zvJoU!t{!#tk^QkSZ9`i)s5l8YJYHMP)PyD_CCP5IN#9~hCQ>~k7O084LT@fFKQE~b zgAivZ&euDo4JPvh-fP`kSvrE>!~R*{Jq`vD@eAAO?v$7w9$-F4#*N$gfEZmSye%B!w>B!fm zNf-)xhFwnQJM1q7Kc?#pFwN|G?aw$Ru)BxyX@Ec=I`s+^wS3hG>A(pjhXwc$JmxH? z-HC#F5>6;af*{Dpad&e(z%}Kvb^EY6TXAoybp36Z;QJ;6$|AhU!;|&Cllzr}l(0Id zeOb5DjR?0Z!Sj2yI-K>v+Dvuz3`$= zJ)YtZCMz8poc;(nC*<&-@vr~2H(%~+&>o0oPRtw*Z+TuYlM z!>F`%(n_0C=|G2jou}9%lz?~PgMeOl%pF!+zzZGF))AKZ(3?!c?_U0Q2L}}&@6Bsn zyFzux>a5)F8Fu)m7A2o~6zlL%*Ku@2aeqEPPRFHJ-?pLJ zG#N-LMsoSs*zAu%NZ~6dC@6Tspc`6Y$Qb=AjATJ!dn8N6_w+YdxXFpxN;?)`xOBGBmGquErz z?=WF$q?S8*md>hHVEpVQS4#*qSumh8lpt$ky38q&rzkmBs}l7)3Hz5C`@Gla2wK(V z!q}Z*X$+dIeH}zeM1poA@}&IEI?tZE)~nISDu*5DuGhI=9t;oa*2koaG;Z|8#U%vS zLo+1MbNtXY@?t&?)@(M1lSzPHEg*x@H1uLBJa5stdt9KEKEiEC=VmKk?aC@edn|U? zmD0sjOAQcj8ckdhxL0iwpp&ZSNww_j*Qc9l9GkIEI1XlW;)t+Y2(B8gL+iXy74+_{ z&Nml9S3-LCNVy>nYgbZ8&n)$Y=Bze94Z5eAgdM16N`9_nnaNIi_8QTh3PgbM=*p zfN8de;T~rVE{DmN@{kn%oKHQG+l3|cb?lWEg!Td8dGiVMJXMFu%y-XrCAdlk{gI4A zgHpf2i^=!udfi)YG$7YdK%w{Ul}@SezVDRDM0#n?TwCe;vgmObpGmGAD;ie6ZfX>ear@ofx3P-{+DuiKnf#^$W7B#)B^DQh~gRLr1FTq=-kRNUf z2&3`5IqRiEMTK5`rE*F$rZ9^f!mUHbAj#$C5(<(upg*`AG(58HV4-nSA!@mMTA1r$6-Aud)Ed_Qr-LGWp%n`$;0RIOMdngkbiznR%SIA&TM>J zBAQ>EFJbaE^M1|UH@sw&C(<*_^+IOPJJXYOy|M@JJ-C**7KVe*geRLT-CUC+opA_y>zD(K zcGbj{ILvVQ&~>W8jK9{nCw+iq;|%$Mxh(SG#1O`bAwI%uF!~2aQ165CVZd4h3zm`e zbM4}iXnI5=8Im|AK`(_$V{kgovEk4W-2--O8~hF#T)@*HLzP;vqjt!zId0Z}2_80_ zI-x`Ae89+fK%CiO%L(6Nga*N4-sn z2PqNhg}#*@3F!|lE`bVQzIXT#u=?#j{q?WY1a1c^_S<;2%`r0v>W#q%je%Z6>SEnm zvO}E$w6-5h_3P5TG4ye)W=HINZqIPT_4qWUrI5unSnhF#FJT06=?bH{RjAneI%ML; z%&e`HA}_9@jXpNo@8Y@at_bh(>G@+fSJ(3Dh^fTy_2g=BYnba(WH79B1i)xcLt?02 z=bEiW#LnPBgl?_Bexu%Aix0X{x6Oh!e~P(w@!0pm8NN5<{X>OQnLk7=dEX9mVujtk zjZV1|P=fbQ;3ie`Yrp&{$p@k*lBsBMb#`C^To z=zK|l1nl&F5c2d6GLJbG87+?!u{I?Z`bCI`>>XndI=ZD_4{<=cHS7 zzh5PWDx~l*l3bfZ-Gyd*%cPz@-N?}}t7klhR?vc=9y@z`)2CbMuAdmfl%{KUXKT>+ zkMrW?WRCa*2x`Q6RX^~qU< zIQ_wAGCsJ)n}n|>^lT?5QD#@%=Qp%!y{hI9R>c+b_vghoy2v+fjL9YRNCKy122I$- zXSkEG+)lmVXqMIL2}C1Zj)CfZ^cP%W9*eD?ukTTNZvq?aC$R-mVhSd@$ucA$ddqbF zUI4N&@BAH^6chk@8iGwvRQ>sWkxKMaTT(*l{ZaI3imjK&f$k~ zIHlD}h-mrnmHUy*K2Nca!2c7NZK1uNvtK$pXUg_!oqLu6YHr#wSf6#ccDh*{{e&2% zaT4c4DHW-TezfK$Y6$OI8<|A(41`va;j@0n@PONgAC)Ftdbc(1Xf{gMvD0u8<#VM8 zO$xx)V69c&Rr=|jA{ojT3;p#Lx`1ZXtdla*wb*ez>tuWN89Hb%-McRH)LCV< zE0u0BEe)iTz2Oja>3brnX7!;3KC)6j-#GuNQSyZadbAm`kw{PGWrComNP-qM{x#o& zWU_`%CfntrGWeuel~Lj93sIv13)&w7Ci3L{9i?fS^wTc51Cr-gl;JhtnkoZ=P>TM+ z-uW*FAxvcYV$Duo37DV^+xIc72VWe|XV?i|(<};EUZBf;dWwP)t{^0x$noqLJLOkY zjSP-pRZN$K?NJ7}LSLX079E*1jD$aO@vMM2x^LExx}*%utlt#KyLmWngRUM+=)P`4 zzai0l!udK!HeTc&&ycbiFayc!!7#t|RFx-hV_3g6p0ID87zR=E9yzExo4Rc!|jh#@uo=ic#>dLNtsoF(q#O6 z@z+J8azl;X_ynrMWlxOa`t0|C`OYK{B22Nfx$EKhWj7D1#c9q>LXYd3fu25LonK#{ z3~D4e_e78LB zZO7U`GgZNrbn zkQQ^GweVfgm;cEKzsHT)h=@rfcDNgm@t49e4(Jx|zduJ5ek7)leMTeW3S5}Pkiht} zLxEcrJR#%y`H~X#U+V&fJi`;R#cV7g4JG*l2Y`9hv2DGsH@=UfivBfR;B;JD`4~R8 zGycy+f5|6gA4*%2-7k#@w$wkdP@+CIw6`Y;&~Mo8-{$x30OrU*B?kN096vDt8(r8D z+u{Ay`W{%1j;}P2W(Wg!LGHcOB2fHl5bp7~@qmf{{DF!1FCRb=Y7qcTG}YzQuK8E1 z0nim5b~?=Cm759xmh-kIlnd=~>w5;ED+3Xn{}HTzHs>v-knIQ3Ni36p1r5N&A;22F zmkoq_yl@{#ftl{D@~klY)mjR474k&s&rbh6<2_Merk*+yNr;cP9s{sMA6Ur$uTkF< z5sek9#Yx8%2|>VulUUcghoD@ob1j~}$2t%&XffU2Qjo=@&Cq(EhG}~@3y;b5&NzLY z+iuk$>tEU!44jfi%gU0s0%KSrA)|h2WGE50`H`%zuW#Oix3FqO4Rx(?pRgY)enxz` zae-Uo&9l@XkbL$rHkI-d2y1lTKyvmydvYid|GV#rZ{toGwW_dMI~kn-T?YWprh6mJ zgKM;5Zy(?7(Hw=H&$5r#0|ssx`=_++(zfQ8oyh{3#v9j&=4MdtD-=}J$RZTn-na}O z_#36z{&=&a@a40s9_;wbkt}-FH&dsJdik0eJXuo73GYH*in5zeq`j7kRE+9;X0}_k zUtv0|T)F4G&oh!xxtlA$ucca`{w#sbMPPs76kJF0uVnELs$3x?wZ+Z59(d__(v-Lh zCqN;xJHw3y^6#jrskLf7g6axox0O#qAE5Wh;)+TVOqXX2c^lV@4`tZZIE7lJ(SZIy zK6yA_4Gtw1WXdTCDVF6Ebnd4yA8yFgafbd9pK0=>R=ch&OX>nW-r(LsR|ti8M48et zk^aw!WkvnbbIixR(427Q);C*xL>-q`#HAxBYIR64k2D81x0X`nsbykS<#|nGvm_(F zr0}7enH?S1zM3S(oe#CXz3SdJfuvwPkUfS37&t8KpIWvp^)`T-!>Cj7qv7UR=3K3P zzQo$*rUW7yZj2#ZFg~+1vwlP5t}BCxSR`vzOY5+YPLim6nj@`3Z#cQASa2K1U&Hps z{BL-cA@M!+^Eu&go=Og_dg1%$ZukCjCrUZmoHnU&h$nqy*)7 zhWAP=Bxy>n9O8P9A2gwsKyNi)_yq}C7Uxv^tYCYX(&t1p^MTfL4dTG9pDS=^8Lip& zYbQ~MB@$KKLB46?FkABx*xFvEdqco^4KL2V{jE5E)o-CwpdVS>A0qd4#B&Q4?l9_$ z&bFR+k979|Z+8wP+BcS3i=*SPLR*H1kCr>W5kzhDrwIG__>46fnw(tK(o~wyhcfBa z6_)G&p9f_FP`FS(xAQ;~n=#m_K_RTwRP7=73cAKU2&8o-hblJ=T2EO&N!my$P2oL8 zD8a%O7M9m7>?-eeL%C68*fh&NeKdM9`I)4=MM3J3FF$}$ruNQiY4S0e9 zrWfe{5<}ntt$GfC54~peH^xW@j<<*Hn)X5Aa!cl*|p=FL{UXRG}c1YP?v0r*%pV)QiucRYCkoo z=Ng7ChD1qZw^wm-L_c}#%uq}b8V(yMAVC424vDoZNW{3f6ziUozZueDlzlvs7JoRM z%L}wCgK~bc8tsUIP!z<+t`-I^g2H==OnItQ2l-BsXk=>K@Bp8Z}b81@G7o7>v}7womaJ^F7S zl6rD4NB&qEzJn-gtX48eeXbXE8Y#{o2kz ztI?VGd}}(S%*aXY^uu3nby1~dgAv(k#yJ%VkF=dMy2tI1XMO(;=fw+2-TL!Pzz3*L z-d@e6nU7mUR9{NfS`M#Jsy2xj*$>2uLTxok1-+toW0`cBsu#UlCqyQGq>QfIl;xLf-b5-)hug*(KzCI=q*I$1BRW}AFhI=;&kXfWll#SYJbwZq=klh4oCF?}?_<_M`I-(0G;*|F9S0h$ovli|)=jqdAbFGmvpF>tZl3wBdnC z*IhARl{W9A?`Owfv)V|{XI=I(Uyc_j8QA53e*3(YT^7AMN57kMQb~9ubSi2I>R@8t zD!%GyT9w>Q+zEv0*S04|M=1346OT7|3l$v4p}K1MJEw$;Nyimb&*YiPKzN^em&4(s zG|_&ZwBUlNY&WQ~EpyKaz%iRIL`_|U*>JE=6_HT)P_JQSN zp$;)7rck*tBn!~e^QYq&btV0p)W^7K0!al0tmYepiy?`eVU}u3>i$6d)@L+bZjxan zgQr~QxIM<&pS%4Kv(+{+g==I`T3V-_v9-%#g?QYq2w8dL z5JMH^nYU&XI4_Q(f_a>_spP1>5oX{7Xgo#-4;R=(WMq$b&%`E)4xCRf_IYPsfBE{^ z$wKAZ3zmMKY4xI6tjh<097qky%a@Il>n}7ITlK-VP_n2CNqJ~IL~WF&F{Nu`Q#b8G z1m}2;__3?Mo^8+SBMI(K7RAp0XxfC9GC%In6tPLhR`I-A-^ljp@Qa@#^cUhwUgOM* z7hCi7E?2zG~E`31mvT-?&Y4F#(z1SAN%aG$kNl;0s$TK@i zg`H7b^t^#cshC~|$GinCSkh5wv4{T!bMbJClw3>JTz8HsN*P_xF%$>Ei@N-qM1uEX ze1Si}@@F^a7FrFC@pVjg>07h>4o!uWjaXeeTkK(yj2qa?vs(@C_&+uH-y9&XBJKY~0ikP3d5aijKN5zP9zE zG=T9L!COT>vn|SJL~m6A5aPr0@r4&P;t5`LdOM`G{9ywcbL!-H<=h|3|9`v;}ZPv5K;>)Ph)V7|hh`-{+kSF40(8j1BL_|OsxktXF z^{hKvTY1i-cYU<%du0tTD4oLhiR`DdwoLYQNUtjyD~7=ao?mXHa<;reQ`n(IOk1nn zBljmp?sGKJHo$#mxB@Fw950qUmDC@*w%p~o(C8BOo7k#(6%o@9g>G)O=`OeK-YzY=>+v%*VS?s3jAKU84BjKX%AIq!MJmT7~a^ z4C2go1Ocx3#;(FP`!QVvtXb^tK%BI@QYDGSR4KK5_SUl*!%dz~%nMfE($10w`!N<^ z4x!k7XmL6WP3Z)m<@Jyp^q?RkgE+hCU8=TTq;!#5J6S%C$9zywxI&mW)?0+~LL^dA zq7ucfWQJs8ga1v>{6p|7uoRmR7=Ia}rw_!h2tD@HFtFu-Q~uHMRrc|ie}aAoq@eG1 z{*PPV$N*uQrfm0LWb+muAW9FDFHBG#)0F?e-IS|0X*tRSlHOmsB=iVVM1Z&)#p(;? zV=G>f14QR8JdT+9*&~bj3;{GsBvEXZ(&b<-ODZa7Bts%BeV==&<@lMh6=%b9>+-cm%P?rbi9^YS{2>BZZC*qn6!Po3ba&o zHOkd>lkIOlKTCZj5vr`H7@p7-83d#>u}DYT0su|IuD$BdFqb57ZL21NAPM{dKoU;w zx6`BgPN-QhCCRusFw1$8^!Z}z9j!{XY8a*4$tU}p<5g<=`|yviOfLtm*4EZCH{w|w z@N}Cz{30pQdiv;#b(@6=*o;&CpY059LSP*a=G^kh;ryZZH;UE8x?@(Lv!K+a4^m58 zOPEw0n=NEo9xDR(?rS8J!K)&Kym;sigr6QMD{mM+#KiiRn?QP`PrRXej=HMe%sLIZ z2Lp?X;C!_bahX_#4C{Fi_2r%4+WhH!{pC>EI{kX`xsLV0+PDpj=iQZ(waeL*-Aesl z!yLOdy7_R~v-t)mnc`c+mDidiad~%dEyq;n7n|;pKgBo9Gsol)?XX*nJITZ`%% zJe75_@mJfR={&m(|FJT0wggr;3fz+;%&r=F&7#v@s?!bTRBne&m36vt)7cB+v-Ljw z8~2;1({!zyQ<<7F!^%gBCcOJk>)EYGbhPYuk*kkrg0$UwHF6og*A@cgG;h*d?&fTt zxq8)8A)ZHW`rFlvT}Yro|74%m_I!?z{mxIHvS%|^>JjzDd|B#fkfCX0o2?#gP)4&i)nJV3AMv-p4*(BALe}sujlOPDnLfqfmzu{%fITxiUa7P1`!< zC?qw>`#3HZ*^W^LsTDlNNv)a~;@MOtnUYPnentSnJhg@$OccPM%*XT7240%qv;|-Q zd@}K*zCJ*!3yn#)PUQXuBNG-LIVvgxyCal1@&ALDzMO*s<3E$>pc~21o|KQKH<91j z9oN4(MdGxc1B9Wm(vCT|hmiyRhn}*V;FeF5#q(?9)%a?~su2s#4`gxf?%?Q2ChIw_ z_E|*FC+UkFmEQ|1?nhK=cNvpIhMq7(x_O@N{Tpgd1ql80ld5=%VA6A=aul5h>WO^! zjfinmyqbWUSquJ)N~SbEpIh%+Ck1$9jF>)G;v7z!gKy?v%6X>C-?i)}XV^%U)P9I0Fh4ba|P>}4>bLDHbPP6T> z0XJxS#0V(z8MxK{9kn&Gl7FdL@;D%A^^i}hk;QsXTQELZwa+W9Qnl`Q*ZxFvqiD8z zoUfCLM9|aRGN$gXNMGnpr8a(&QHwkfWR<$Ny7?bINvXP2x4oZUXg#jCLbc|a2ztJUvT4I8UJYWvFmrjcmTM-j;!7QfTZ{oW) zt1$RM(4>VX&jRZV*|`QcZFwf`Qe{oCxo0~wl@|6XjS9&;a>y9Oc$?wTItDdE)DM`O zH|IOyD9(}yROyI0@M^h=^44whozL6D6xzgg5viNQ$crdB#pw~M)8F&*yFbJa3VO)& zHIgcNhB-RkKrOUBufSH)N5tzyb!GKSt#+R`Z^%ncE4OD{8|(=`oX7RNQZX>sWq2hw z=NUyO$1Bb0AD0vT#6auz=ur*CPkJ94_6wAhiIWmsNMu&8vcIgR(V{Dpc}MR!7DI&O z+;ldDxXj2*)kbNe`&Ta$FRY;Yf`dPqd26ae3@b1sQeA6q!y0+@v*e zH+HG9JZO8w5d`j{_}w|C@9COyf9_IKrrJOEiH~g79PL!iSCza>1fn0Mn0)q6lMLup zHM|6W4mgXcYf#xQED-_L71H8~y2j@7E- z8Kvc<1PsXBWfYQvG$3T_C$ld#DnaI<$@XR^eq>89N6Mum-J%E|d6 zKU0h&Oq7gc%A}b8DeOrgW_oJZ_f)@L82(meQ=WK@j^xwc(u9Or)%1IaT(L|N5W- zhgV5tIaeWWP=V(Q>WgN|f>v11i!d!#aDVc2?MO;a+V z-^u#gN!p51!JY2YI|K?Z<{}GB?VJWE4yPy9%eA98ywA+yFdRt;bq+Sn?fpsRFCCyla)n7S>FhFR zeqqeK_=%F5ekoG$*4jZbJIaNhXqcf-OR3BE` zmt3Ya#~3*$1|Fhq{^vadjGV`%c8cvTSAgG=j-r-(&628(#GVufzIgk2&9*nOOHN=v zkhJy&PywQu_4T-oz3Xx9@EN>5M?Xh6{t?WQA<-*QK2e~p@e-229&iP+yygagH_>q_?-!GIgWL@+eSDrCw?= zj5&`L57KOM-{0M(ub-t6_N_(k48vLPip`dfkw$M}cd6fsK0Ht4JV6)0$lbt0v>I_P zSXsBcUTPH`F2f9~xsBpD=K3CHpjdj+VLjtC=eIkY(FI64or;DuIEEx|h^gg?_Kn}= zfCD9dWlNLTn=G1A(HFTMl1lP1kz1utt=&}?M8^cbWBJ3&TMiIHgHl+Ebv8rtTYct( zR=;k!BOxz65?SBq3q3(^I=5p1X|H{Kq`d-3Hbo$MBi0+s{Vcs$Afu$p4f-#Y(Q z$)3vusLc8LW5WhO&bsTuS4EiK+{b<(YajzffW=iAUSt*GoWogQCv`XrwJcR{~@kKulk)* zgkERy!^P;R38alLH2xHCc#Oc|S3kZc!BrSYjNSn)wm2I1Y=kmINr$ajjyrv`2loms zu4E(t!;AGj@qHK0%ugeUufg>XM_Bv3j$-r9kS1S>?s#b^6g+Me!Txc_<#pj@DgAP%~u!D=7HypVXtGI>$UXP-l68IY}8?r0$LXpqq!!BxGR@#z0r&=g{`%1 z2P}ccBb_jmLj=NqR1I%UxIyD})K}PxxFNV~RQ$)D(wC%snP=N0{-qsZ<)62EE2H_` zSf$CmWV7V0w1?uDtD4cwcV5UA%{EyuXUQZc*x#QCs!TsEu6I9#tu&tr=L^ASsaa45 zik*l|r&Bo+aPYZ`^xcqQ|ot zr3!nxE8`u%jK~C8y;4T1DoueY4mf8}`bW)?L1aKjuc60UBrdtxWp+I3wwtNEgsHR~ zAse%^qukvJii&t_BqIdU6sAx+N&*$O=~$Xry_#*$2$T)?--C)&>*@ z8umg^eqTdW7Q!vtm*)zmP*$hBQ>%kxZ6;( z{3F_TQWnVr|B}@z-G^byVddpmUTU>sMfC=ZvV;aNQUQ<5?efjO*wU#|LzQzn+&qzK z<(-er`i*)>Q;qd5k$p%{319VDQY;kQLAN+R;Z$DT+;+5HB_Yaef4#c@xLd`TAW6FK z-ZRAQ;N2F)XdSc|Q-67&0`+jqmAO^7ue}fyZN_L7ZY@K5QvHI5n@2gwz;Arf?SN+^ zLJ+(~aT!?iD~?0Xd}qG#6{T#rkal6x2JgUa2m6NS5qrK~R-^~GY`99-z5Q_ZqrJ0p zq<|V-?Bp~Q2!r0rRa6|i&ngkYH$$c#wt`d~HxKyFJ)XLO&Q8&stgh?qs$Yn1W|kW^ z5*UKzyubp(W1x+0Na5M`=yQE!Acfvo6jXy70B_UOy0W!JMobq=|B85nNr%q*1wuqf z2pjRG0An2O@e`vNJw0X>B?4G@4;F@+T~YyB$m-Oi#P06@8jm0y;58bc_m2< zCm7;I8y}wOhG~s^v0rJngC~34`)|)QkKMp2Gu7tV&F=mGBv8B&;qYAOSBx(%Nd*FX zRX);b{K@e0n zDtjOtH#86XT~GZkVU=avv@7TpvKtn27(O#o(QJhrlYWgQC8hk2aAUb!QT2D+;Wf^y zcy@@DYNJSRt*?Zp5xiE$H1Fq!zty5`2XBpx)#}RZj90$Ol8N#!UThG5VA2%E?TgE9 zU9n!bQo8<8ZyP@s3km6y#}rH{%w2MY$w{JKcF77(q>w9yVi;h3&A`Jc(RZL(&>>7 zLmDGrONm82q)^tQ#%tv;Cgyke_y1g>{U z8jFgpwikt(pQ>{?DO{J23CQOeLwipJsK1zdfo)0T!D5JST?QY9X-Yx(`1mswRyjJw zp3DS1Ch^<5AjXbEQMTjTZd^vuRpF=i0 zE+=e<%^egfUfbK6(AZ^F%bj+PDMh!&sGtXOmts9wpRn%?)z$75JnWoXBvZhN(-UfZ z6AamH5o8wsxkO(q7eaOJG}skhon~RNljP`J7|(J({Z_^ido2-Luh}JyX z8{VE{DOeCWlc>UOrIdWmimvOSY#8NOZ&=T8{mpYqS)rhIlVsL+A->=23iZ@C5{zuZD!2QX^HC1Z@ZYb!}HSl^4YOk?(Tv;RVmFUq8yBc(D zy>k<9wgM^HH8LN5v%)JvzU%sm9C1SGVa4Ng@rB_u=*st}*)0=+t4@Q5obhroVzr0I zrz{G+xtg=RYqZ)VdQX!M-M)@K4QvgHm97x#!;O<!eb{4Ud^mxw7c!Pm>d&&Us>fF^5OsdRHm+ofy=j$gsnU6H=%XVf6^!#{@1V z5jRVZ6#s9^Inw0xR8LmA;x^M3Q9t$v9HN}KIjl%?C^#TWB+9zkE~E?M<6O|TvtO$% za5vK)I6AOa${W8qBCY1Y!7dW;<446rrqyBMB|I^}Mh|oq^qA9mGD_9d*E<=n%{CGo z8~3>~zXz`ncoD46va?uUewZ;t7n;DXpq_MIpLjg4-z1PTn1n63N!#oyh!}MFMq@54 zh_~^&{WDXd3e~W5=jN|NX7uzkkPBB>n#0C=Qf~|$Ey(3sUXa@Qe1yud8N;LB1K(rO z=hYe1QxfrLIwX9Ne_mwrTQL2DfSm<-H3|v0mvZsSC3bU{ zbBHwC<)}%RjK(h#-%ncrsuR{#eN2wB@1N070n5$htLjupA3H7YM$w9X-p}57i!64R zvIa?<`<43Bba)Q5m0P!vWYJ$l*C5!WH^tUf8 zJ;%5hK6O)lugfcBbW-|*ZuV^N7MSVEfcl)T(Q)cU4Sk3!g4V&CYlRv<>+4K|i@7x` z5L(H%*SpjQo7N1+$tH$fmX?Hi#f+mvwRS`4udREQh0EvUeCdUIxoLM}CVf2XLh5Z6 z($U7fW*V&Uzmv=?GQ2W!-?wn}sF?NZYCA5VZ*D$|%5$1&xasQI?LQxhZ5x%|{bA21 z9f!4kpFdF2=ygXz%4X@^pv`A=V|ke?#?BgA84xI3Tz=B0_b97VrUH0wo&6@n@`~tB- z9JXZncg0oa6Os#0CKo%m8a6zBov8PsK)7jVX(qKE%7Z2ymU0`IWA1KF8 z@W5=n?jf6%JyYyVODPf+s+}>y9Z&33QdNV}N0c9C=x-mtHnBZJwYHgYk#d!Z zV$LYIUgBt;+uCcf)TFuLw3xKpv~R`CBq!Ya%dNh{k-fy(K8JJv&FsrWPDksR2K@G` zr%eu1HF{R|eet4ID5jIAZ%N_nPl;s?4Es+fJ6;LO5!it*sDFK>Kf6)b4a&txOgLHU z#lMu<$}!llul4XPGU^Sf7eAgt z@FKM(E0jZQUa(rMI~BaS8dgAhe^=k)awA6xZCqk9Pc{pne^LugwAeprHpv!c+=Bmr(ud zx|8EW7HR=w==~~m?XJGpH}t#HqF9foGv!}>tfGJj(rKgac@T{N1)TUIk~y( zt#r?ufu8OpD&;|4@r?!ywiy(RYUhG6>?*WuE6MJC3v3vx>tb>jO0((G`aLg$T;&c= z9|^f1=}_Typ{8 z&z_lmUrP-q{m_DU69mzIn5*10?ALynN#LP|&M$Q>Qz^vu7ec74pR7Q>FB?*igezgM zvCEJzMD=b)T@ck!KwNGc+l3tD?-m%f^J_Rlc#1$NJE$y56XghYyT}+>_RF4eOXa3%-ooz0w*{|GuCYQ24H>R;YC$F8p@=)!4Wy222- zyc)VN(l{8QMLgEc145G^K@X|*d0r##MYRn)m@JtuWl4wD4wPI$rnEy;P4N9L)sfUP zFY}et2D%jPgQz>{Oz8zStP2_8ubud*BMX*-E? zfBCg6G%TtlV8jI+mw2*rnZg01-n*=!*TsnGArVvm^+#0nrN{e2VTWE;PJlpgxM&aN zJlwW4kqUUdxxz_5itZy7^su}+6lU_(1`!bviGIICQMgW;jIO@;y<+>fJ#!GM18F3scYceOH5ST_k#Am7W&d*)oZXQ-_qZr(BHaB9&l=87n!t z-Dc{k_^1>2UNH0BHMYJ-sCiWFq;EBg^WMzK6|LUPqR~E&?0*8UYq&1?WGW52Hm(t0 z_)`o5ZqRF3E0|Q!LiWV=W(Oa_o~}2Dw8rAOU}X?#xy6pZccb>w-9Oa^WGisewjUOX z1_KX~wMEE8;N~brYT@(rz4M%I+yGtE9e+vvnb(X8B8xwwk58k%Rnk9t+$>t^ddo%U z6#QdxF1^Bg#QuY;qhYjwBN5T#BV-@YfU=M(Me+E@!e5~OZ|QJZyp8;?`a3+J)iN!Z z^-=i(8Rj!YhR<5VKCik=J9eI(Xn)OEu3Zf$hw&j5@&=ddmpHT25e(XpKw(X7ga6Zo zk0Kl%A~Mm7yM;=4vu19NF_NGE6tuiX1thO9D*1clR^iur@Ov-+si8>^`%R-38bM(# zWH&k%M6?KAlCVu~tzfnP>};TN<}2d|C0k7AdQ4peuOCZfTwFiy?(Pbmb?@%)+tk!u zPXh{UKRwcHy}fY}_5`?GPcImq4n~s8j3H9-WuJ0@dW8!2OOgynA|VE?hOa=fU)E`t zbEKzhmr5c29q>wl5|9$xj+0h1-7drBajRe>3YP_P{|8a0aYqcIz-zrD#;tE3uV4B% zZ*l!Ect(T2x_vyLck%g?xr~#c1pHq{GUesMzM8>Ljz5A_mV3I5L)>6V(hJ#87O@_H%pJo`2;`HlIW5eoiM45MNU9aDOM!(WJ9+swl1Zn*GBW*Rqt;li7h{y$&*%TGfATp( z^N3sWK3oWU$sT*Lzo-BUhR?NM4}NR<0Thf&kH@H6mxN9tAe%xd1AHzcWE)8Rsiui= z?+h2w-}8GS5p&swE9XebHDB(@1Lf{i2PrDmdZIfcwXgQ^U6#X$`Q=YH2Gapi#*gO~ zNLmjAFNJLF5!gs4)2Zd2-=LM5K;*kZi88N_SC#6~lehZg%Xu^b?L+b|BXe)Md>BZD zZ|&7Oq)*Dc<@>4xv1AO<^E++bjgpALm zJ8bDA^}hsFA?1;79d4z?WW+>Z{hpKgWV%+R`4I5E6|>8fhc#|K*Q@Y}VqJw*;1$($ zaa5J~k*qa7m)7I+^DN_mgdw0R!E`N`g%zZzh)K-r_2nhInS8nNKw73$RCq{8h%m-s z6g4eSM2aI?xLE7Jxj$7ZQ-RX(_D3APy<&^t29>qc(%SMemU&b_7)g)O!SzZ0E`W8w z=Pd3V*L&5R2&e!g(S@y^<8ETK#f+K(msq@t$i)7XH$=;oyBp$bKM8w3~z_$!4>_&~Jm4|XJ5AE**Fi1b)C%YUj zM3t9E>HI(T{wk`EXWIfsgS$g;cX!ud32wn1g1fuBli)#v6D+v@u;A|Q?iSo`v(MgV z=YPh1yD#?}<9mdro9b27tCq|;t2dyxSgD)VFc8ub_W=XtR5+0l2Ee{dY1VmcH7mM3 zxUi+NZf!Y*g(4BtYC#OD0R3}On~UQGrzVCEqAOc688lxJT}5sDp-}z(G5ZtvsT~@A z6wuG&U~af%!2B<;_6T8KfM;ErqX@uJO#nDi-sj^5_D$lq8iNW@n%SmbGf`K*+#cB8 zPRU{3mL=`%e0@pL1@s;WI)Tq1>G;y+d_4eIE!2altPpbPd})`BJB10<3QAYWHpOx3 zev2(;Me;A_!}fE9^197Rt?n1*U5)YYk)BVse#bIm9|#wO$3--V0Rdh#!nr&6DIY~2 z5e+d^r?@Q{O@{`=maWFW8zsL!ckFh8sfY6N^6hOAex^zqCj#u)gXMazH@z~ zrUHPflT`UCrA4|KNf-b8IbfG2?^S$*n-)Sf9w`bFr7{G9Q~u!pN*3^MkgvZDq%{!% z4b}#r9HX4j+Z+d7H%seq)DdfwE{2sNsIA6pklKe!!-#IT21>XG%F(!V?Ysyt85|+W zl^Rl4dn1K_v;kMM)L-YM>axuSZxG!Lm=>lg4kXOK8L8b=)=cr)MrxN?} z!EVt1DbClHf90Q451X$##9Kup{%C%^!VLc_>dg>7bhU+T?WTLXdD zFYg{Yph=39#H_A{Y;rqfNGc7W7Z)NLY9xg}x|WM@R{wV1v*^D(hCTo~C>Eo~ zxYIvWur1fBY6NUgoz|LEVCeKg83e)B`(F}SVHaIukz?}-qjGf3%)j?h%BJ&0G&`S= z!iYF!bqQmmZ(;Nf9s>+kDk7F=z^^C zA6Mi&+LprD!HAlMN(CBivhdM_tmE6BW&dg9b$Eg!=c2R~ezd!Tw_U79V9ekgDT{IE z27weJ+EW?CLjRcB7?;ZeJRaw>7awU+Xx+Mz&a^Vu|s^I*6dUt~5>J_TrrW&5{hJw3m%8av4cf5_sCkSRSGbE~{8~xvJ_swCV0YpPoFb-?*zd({;UYG#bG)^6B?{5eyfPvM(B>0WJ zrf&Z@+6H_4KK%_}d?y6{gxmAaoqr-v_GqR;5~Juv>}<6i#Y^E|Q2;Pn9+venI*R}J z>8|&`_GQMqn231)BLvxm|J?0*i>4 z-=~mOd^MZletRm|=6luqUE-$+0AnA#;_>a$*1Zv=nyp;>B$HAe-_q=~S0NDkF^2kCV|Ycl^}2L)8ZjAwu{Rh89uj{z|leg zJ6d4WBXht|Gb?Lg;4ct|vBt}z(PYWVsL}W-s~2b#ZnFw_hp%^N&0+2Q7aH${_6ETn z2H*^~E1>^Ea5aK&l2*!%NaHP z2bD=U(qB~zc{EFq8X^enxosa11bgFG&W!yVG`n0qUM4|5xU(zY{`sdU3^KvzEB9-B zkvELc0}75f5N!^;57(^OD?YlWK3#45xcohb0+4Xh9AMV14_rSe z7916$l5O^P%e5U>_N%rIvT@lDo*!>7?|9Y#4$4lIezUjc?#63lIvTP1kFK8Ws<)^& zOVEpHj()Fq#gzx8A>(|k^&VmR1W8V(%~OsjiZtpCc>7zJx-{M42=>tfz*r95Ex(a& zU$TUP-e9m}HtRX0zx#6YW74^XY#N`CGn>bsVNeoPq4gK?9TIk{8H9X6YX6%C0A?1> z@>dNE;D6=tO&`OjGC~T3|jrr}*QAg6)6)K`I__+WXF?IQtmc6`IkLCW(&VUM?Cb~GW z3e;DZ{rR-r(MR>NF|h(^ugRY?R>h_#&2Qjf<p!|b*P5dx{PCXidIPSNSzzD^+t$jj0tiIZSDJ%>#vqWzl^VeM!F0jy zyS&_?*Nh+nG_1Z!>|Fyq)>^AuB3vCrBFsrVpxfL~_H;#Jff5(U29W=}6anbbj`M6; z#q>`Xem=ib0ouJnmEu&^FAkU6w0ZZ}4a`qYq+0E^b%>?_#Zc0g-wn-9ztEYMb=SCL z7-Y9*E1gObkm38Z&)VkstksUg+wR*9z2I4G(XBO@#ty@2Gnv}Bv(eu6xLwE5i1zpX zH(%i2!w3TaG*gm>^yv~)bhYVJpCm@lmfH1Uh9?H3GTnwUW#q3iy(yc*J4B^ejeGM& ziHLw)&u@?{uTp3;mb(a0Da0G@cKiLB#wrsXl1hFGYJUTk;trdc|4_ae3xO{wMR-5V zTx7S}T+Xz0fg%m0F-aMS@W^WKLp+g&$flt^5saeLP;7VB6UihT@9}7~*m!hXou$++ zlRErUc=_b2{TidbTx^|2}U{SMok93pC zfU=2;v__L7XA*N!i|jzhiXV@l*Nv9jV(^H^!G@eEd|FzOD{aN$fVS z$5`daiqE@LI|3OQ1T!SL0Lap`!6>zbcsuC7n5OWDx7xrHjpFI#)EmQ*L<>2w*05Vr z3uJ^P=Pnr9h$SMd+v2Lr1dzCcUg&top6@+a9Be0E11kNigL^U!Lrl>*?Esy_j( z$ANW&sa?j;EoyR28SStBFz?KUQv;|JvZzv2OK;MfV;c6^#}rWMb~L}Kh0nrP)XtTv zt5*QY=l1Ax*6WQj94ps{^vm55F$?9QWCd7&#mQ*gP%r*+NFa#2(2B;;0H|*zxuWItDuuO($af{ua_Jmot~<=STUDqTm@>#9 z9o-s5DAt!C7#n+w=`3SH>&Q0#bDh zNdPoavqZ1SF$PfaQi+y~A)=>$&M+I?7Ip!Tgd}@6&kz_xt0D>>rpB1A00K%=mYl4t$S7eM+Kq|x6L z^d487q(i~&yMP6m%h^TCGV@s&M6MFK7{ka4^W|SyJ8?g-FoUnw9(YeQqq3USQAOC- z32_)Z{P$AdD3TSply4rjXTImElExmtTmcl_`#U)z=Tf<+dBv9vYBF3Ifb9IWv~jZ{ z9{`YEUSDJBrzT65XV4!ds=rLK?^G2IvYpUl2>MO!0o_P1Wgx(~Rv++K4Xd|y^*mqyVXhf&=8PW%Wk~wp zvXV$GHjnEeM^mI8XRp;F7moz>#c#5m1HRtKH}p}bstzeisd5Dpe4;2%>Zf$BFUS7X zvad6r4LUsTV};#xt4%kZSWP>VZUE&)?OJ|YCD!*i_MM9z{(B2j2c|t&|7XV@34%a| zZA_O2AHneZlLinB5s`Fe$HF}R6T1J)b9K1Baeu$Gh1)PwHwi3PVghO~< zA8`M1Nu(AK*G08q!a~OU?{^ER005e~2HN_G$bUuU0p{dD60@j7M?mvmE4+iWrU$Z^ zdJqla-}g^QzU%x@+w_@7@$Ui7t99g_@tQBzN7=+77aYMdWN^xvWtrI{iGH1T2CE2= z^BAEcnCoDmN;mD|{6$-d8^S-{rKuve)4dj?NWrCRuz%Z8$X_eEfqXKj{R0!zutMZK z`f%4YM)@A_Gk3DNTg)F5o54+qqDxSGbRgK#7>7fx3`ldZxcWGMf#-%utmA3Gnum;} z*PmF{Vm=A7NE^>!75bu-HIw6G-Z)zRDf_kP-&H6I)3~&LWUIq2(y6BK%vmhKd5vx6u<<*>mm}s|2;wffAQE{ z7yI+=2xdXgr+HIfyH!v@R{L!sKrCCbCce=)2RLm9S9@bO?v_e78G%7TQmaya+`bXgNSo$?gSf8qvH-87UQXu#|pwXKu%%%J0ncFX3OZ|i!brB z`M~7PpZQR*@EH`oZ}8sFyGYNgEtReBo30vF9-=EgH~8j#;`>f))c_#S^FCX!1wZg^ zoX7!K()csO*UC<<7%i8Q%DHzn{#^G{0_Yuu4Yt^R)A#Ue^WSO^%9H;HQby>GXA$ z)|x&B*O5Da4tIw3kuYLnS9$RWOFfH6v}$vM;m{{%NAPJZCi4MEpRV**fhIsuVolv& z8xCMh1_9N2i|PK2Kkdg0;q-ZD;zJ0Nw-~8x_X>lQLeWWYtn1BLP8+ROd!Zf$pu7S* zrG|NTNlEkBLeIxbrOvW_qCCB!*lTR{t%!)nYot-^t%2(MrawIqZYZ`7@Y8BNNBAt~ zi)l0YoxUj-NaMbAAzC|}F7qDFwo%Vis`qthF-*b_XR;>D6zbK2UL4^D>1s7NAgGMM;P+9NqXgIA1B;!T|5 zZP%Q$72MIa2BlXy&RAZ5dt!A!zhDDR{jLOPmMU{~Hgig&PU^XnCiMNnn}^b@03Dub z6GhNh;(_Xy(GK0_DP7e2;S@jdm6a={jcf#^`^%j#R3P@>b0OMZjY(nJn2pzAQ!)kjL}{ruv)s5`iscoetR!T>>rDL}c`1Z;Vx zRBj0wP;AomC3;pafp@S%p+-LR{Buk>cVRXGl;U?TfgSWa+coq3LJuRFai3hyT&4by zi}0pj1Ugk>_A?O?l*M?5?>7TL;9}!Nhjn|T9lKjH`kTq6872%-Mq>-@Ys|(`ukcFq zt9+$i+qWa%?bo}Suveg#1Po<{R9Y zseZ1N6=i!N@6de~NwS<7Uq?$98gJW@N#%e1=rz~=d`RDVuSM+R{V+|7=vyOfS?X2n zkGo zR>te4nlkD_x+vxPLWa3k4aNFq;gOWT3eG*#0Tp#U4qL0Ct6}ZgADh258jy+Z{hln~ z`@~ZypB@XeTsES{gd`%+DaT9Q^q>0z#Q%0?^teZi;S z+3$8Ro|#^U5?DHfDi2<5^+0eQaW648dlqj2qIdwLTx6=-?yD=C{*bAbxQfp{Yq9jr zJ-s6~lfMc`GM;&XH1Y&-XcgHckxYWVj<8IAN1pfQVp~-?FmV=?n^CP?VJezcE4xR^- zs~^y>3d7G9?}30V>EgO-6#sVPqekV6zW42jI?lr=Dkj}$#Uh1FE{L-jqNq8~+paKT zP@ISG7>sV3xId$|HIx0$P>PBI9mZ|TX7sHXN|H!HB$q^kcFL&v(yZ&ss$yv8*cJAA zols@&+EHbIw5^>9Nq8ew#~NO<_eA_p#VP@5x_alim1djtio|g{C2v15oorP{BFF$; zem9!9*fJWeJ>i;qMU+XECDPk})iz&s38lXT`qSmAWmF+6}fPxWAF^w&wK5t5xY+ zuiwO2>M_Yf5obdOQ{#G#kCI33EOrs6kQVq*&!ro3-993UH94BFdj5GdzdybzR{S9# z^!UdZlV0sJN~O+1H4fYS4Q7eu%pi?svj8An|D)(e|L{DQ2?Gl&WH^n@@cV;ti`kh_ zqWXzaGd+UK-LHmgki&pRZq_gPECHxCy_0|s5Np#OmoDk=u$1tOD}o*Hy*)Xsi({-R zmYdE$UnOhQdqdIvEP_uY20ixLIxF2vr7#hC-g>Ia@jm8?J!02YiOhkSDRk>*au*a_ zSrmQ+RbE4{XfA(c2Bp3e3Mi=NyLm>bvU~OQ;>1IL5>_i#{4h>QOGDG2Ma@61Z~=(E z*nIquLi#bSGO+noyA`rXK64h@;pTY3q&ZE8s_rn3xhlXg2W⩔b?1ZS0Zb_+ZK2u?vD_$ZrbA8aQH@KnSxh%Sb)AdV z+${G5-jyzzCnT+3_FqA^q4!ujOcz-X+P^i>o0}pVsebvMNPkiGbEB0>>^!Ea@QvDh z#GkAkFpeGRzkr~ymp*{d+?s&dJo;Yg$eQO>ot-UgIag}NtX26z%$wywFjEZ_IAUOi zC<>^ruo(Q3tL#ss$6gd#LN86!>s)}BPEq&FdFhVZf1V%@&AVqG%T?iCf~dH!GHe6>b zI&<2g^?%Q31xfOVHC$Gz9@k^r>j)wNuD}8D1kN}-1 z(An(#NF0mN-2qQ;>UUFiFb{Icd+tuf|F{7Wk~EZ3OnYX7^St`0b+#efHDL<^z8NY_ zWb{Y(YMuJYQUkZ+&Ts@(oN$R>rK|zZD>I|cpQsnV&R2X4p+|5KNa)6s)`q&lb6gRS zF%r0*#?ZK2Thjo#%rB1yJdKYnn~@C`ebyt<1ZLFHZ>+ep)uE)T;igufqb;vTHz80J z&kGQ_aZ`Y9D9&Ec`jhxkVhLT!QGU-QsC!;ZB44-6N#kyf3CnNs=iV?!lqFR`xCb3}_X2@|cT zNQX06crhuFSfBv_1@z{{5OtnrYG0giCu}9UC5x!T(|xi-c1L&$VnR9qmqdl_&vTf*&Ge|$TWi)>a2JdlhaC?xCF!Mc+EaL)#c9`D|3KP` z&209kSB_Zb1%uX`Emi->{p$lG>kv=+1+ib`ZQ_S2vs>3cXLEC_JV+39&45}VHCwS} zCIEV$YZv?+nyo5S+Y7s$yrcXRQ@t}*Q1777Onx64Gy6B-AtM_At%B)pD)=N-a)6LI z-oj>O5Ki&TRplr44Fofp+X5*~=d=FZvk@B|LwKi5FF!KVk%|Mc@RcqA=s*HRU-mt3*f zTRy#FIplNvacb4s5ywMiSXjuB&&#-udZ&LBHuFp$oX^ZpO$gzi)WI2x-jA7vi%=v# z_-JWg7tFF|;kSn}n~iqM78c{2XAU`2S+72h{2VZjsnhOxx<{xR8VrWhHy7ZX#t=4B zuC%quP_-iWySW$li4hYMyEyZU%EU^lg@lFdX>4VQ4eBh5z#46S7I5Q|&lbvqfX{@Y zr>-AKRC~$4tlY6f-})f*)c;;(eE(~h!%igQdCRA-WR|-pu(OpYRe6R0FY42{YBd1z zet3m=HpgvXH#-#|Z0YEjZyThXbl_F>BCp1X67g=QuW&aTN>**?oUd=AbN|Z3I`dOa zwcSZL5JgNQ&d&;m{~P?K1k%bnl+UOV}c%z_=jKhD@0ZjebRNj(b#N7|Id6895akvOaz z&hR+yqjn{}-1O(gRCy1eKQ-HvA`{KWRf2vUV5ye!&(RO_emNontBmGN<-awvAxI^? zJlG|vE38ljN_)1qYlN&%gwJ`S#5eUeT1W3pk_$d;&}58qbfv+;7l}5JO>*liKx3^N zU?H-LNK8k~!$Lws9~PyEg(A){VT8&!)aBEryxSf|$gXwqDkvH?5&+Y5Z2CZ#iFxGg zM9x_wrf=O#bS_q+TS`S_J1=rJlmY=ZBzP1bWI6cyeF#>lTxXvvblx{O20yZ?-N#2q zn0);;OkmK)y7P(Gy-bR|R{dp#BN5`Wy<=pudn&)9hu?D8dRW)m&urfeFuKe*Zkycq zYIj7OmNPRRa?^(dlD?^Yx{u9Qshuy`a(6gDUqbY3wbBwIy1+sLk0{Wag`Bu4=0@!DVIN4qhv z-ZO&3YHjp9tMP)uJHnkb!v$(5O*X1N0)s3@IX`$UHWXvskMX@B!cwjARuFz9&^x}X z*2^r`tD`f>5E+;>LG^q}W#YDaaDIB4vThs8*{#~LY4<9bjktah`*$jP=O6(mybq@2{Qt zw3_UCnwXAJnuz(FdA`q$5-OE`k|5@>XVHT4XjXa3N^$5}RDBRUQzVnmQL~d0w-{qQ z1nzboBOEx%D#2B}=Eu5yV38Xk5~o-1T?#S2NsLmj)zahQV`F2JUu~1Of4jZyRKaM; z?Yae>`0Qw|vX6Hp0)sm5-oe#p6H|UO7`x?<<6>@u4n$F=8#ycBmzK>hW22VoXQ}10 zV|VxT;F_NaeDq4>ZnvCvJ;ncwc2nmn=)uLgfVQwJB)%p3uG_ zboh?#c_8^`0sm{&ApxHctLN2f_k`K!wnx^&-x)z=ZE|YmC=o(FH`cZ4qXgqWx9Er@ zW}Lmf(T_Jx6m-%gwpUSe^f{m1GU1x2Lj{A35wPg?%V?!!D8RNHQ~E8HpM+5RzjoJq ze%&f@AK3>#kYy`gknt5qSLm2IllE@SET2sJ%&aBz1gmTt^ZNx!swrQ1=z~Q-sM(W{l-#^xruw0l`9Pyw zMxj+_852Tw@<;6y0RqEpu10*!$W)O<-?7cJb;c@~0 zI<%oGHC&o-!N$9q#t=$3 zM9=GMjjL?GSDO?5|DKa(2>Y$Bf{y=T9g0|gzs*RQgTznk2@1kufFC^(YY=)?NT5P` z0L^OeO(?p*+;{KK`S}aI0J_zGZl)B3L>Fr)-L39*y0qfL0m`3Q9wfrrYJ- zSkdL}d;&6oK=Eyj=|CXEp%|}yM{@6T+aL=KP^EI)f>oE_locz8#M<1J3jBL{_ z-##uO(ecCjAeJ`bhp%K3_S^kvW%|TM%$j+UE^T8PU_uw!Tt3hM6vArE?oz|}slgp? zcT7Lr=keCKE121t=t#9hS?bL1rbnEGx@C*a_8&sfowgmH*-&)O8kQ(`5J%k`&b!{`ZcfeQ{#@I|)U;lGqB9rxxDNKKxLX1SrD+X`? zt{`S_|I5rp+VgKqGgFiF2BheBBrf`tYO6?_kZisGuU8jQ#)zMZ{Pw-Lmiq6>A%Sd7 z1YD?|=SKe0|M9scZ!Tq8yg=}uE_%J-&5KSVP_O*;^tVkyN)A}d`BZc#aA5zqgPHPe zSb97M_5XMu)K8XgdjqGEg~H!{Rs`0a7T5z?|8z+;{Nqyz2HpVJ1WF>Lz5Y4S4jEve z>jl{I&5Zwivv1#lHPI?Em3YaISC&{jVh< zkkSF5SgMY*^_^k*VzYyR(qpdc$zrieQS6Jm?O1`?|ISj!GYJVhOhgU=$c8btfhM5F zu0cVZk(LgUNFZQ>9LD%!SqVq5mYa;(My%cR!<~ny*KpQ2BUw|-d$7g0~j%SMTbX~vj+ zXDGB(5ISD+KuGa9M)A}(q*~cxaPo7}J2b;$O)mVy`!&(bz!D{TJG|+^lc@0cX`U8e zLTE%Bd*K$wJ#=u<4l!6AkTYCBi85H}7-_DNXM?B9!KQ=_rhV@RabS_3Q2<{8S*7;R z>Y2_3a6A~bTQWTO*)*sr>z{1Gsk#gB0guu^^Tk2q=FxfDT1zpT`*^wXyu}sbt3}W{ zek#Jupt#{_ZD)A12l!UsaQEY@Zr`Q%gnORrdy+xy4fWiwRBK4US#N4{X5-c2fF)?d z-l8nj#tS-mE*<}2>?KN`bBrI<@$_zhthj)HK zRa|F{fg#V}MMVQgiF}uI-ykXX`%a1;?Uxf)VDd{q2a?PmE`w+A{KY0hUzj=4^=aH` z`c$QrR@|yh10J%sC@{^U#7Ti7O>ED$24Fd6QW9GUCZ@y)!u2wL`T=%7 zw=#yB-?T^&r5C8B@z{j+T0C@g@Fj_3C6$%)U-g_1j&*mW#WFrB#^pzmBUJ%ol^<*AD znJ|v>m-B)0bgBB7@#EeJq1AX!z(YHRSE#>nq;Eg-=(wIc56qp^sVJVIasB4zxl8}O zzEg(8XGh}dXa!gbE5KFr=;+}=nT56P$HNPz=I2jnBZ*r_t!gGuRHyov_|LbZLtoRkyu?*%i42Fl{0LV2FWUL3+*D0@)0666%u zw9~Y0N?0r5{T4?az*#R#tj~gU+Fpx4r8l-~z&AOgRyY_hFJNzOOU9Tk|kL3@ST& zSs%~6B?t`(*(m-_wAyMS)C;IN*et%}@e%t6!Ax!Wy;|j&qBj5V2}53GCpjqb#J_w+ zE!hOOde0Djl`={0l<6G4>;m<*rE-0DBOM3(JG>8G{)az1TjxWGP&4o;iqmR2W+!;_ z@pSOnC+hUs(7V6q8Y*%U*igj2h5yF0h{}^n=c)Z@F+lBpNuN{}bynF}Q3gr$M*H zZFb3arKMwAy+Q|0fa`K6zY-O_8cesvMKXylNsf(RO3ue>DXH{xq-vj@rt2o<`V5Gy z()s9z9Zqcg;}cGa+K#Ki`8Kl+_xr0x1y&Oe>&aC46cX~!ch_{{$tJRLSvBP~Gm#ld zSG63#Vj0WmWM@1g<8VgDfHcna zGWw8SFx}|JI%pbZ98KgrKCD?2o`iKIbX3%LYlTfZu1PYPY_tAg{b0{BB`}`?n1-Z7 zQDHTKnv#-YlO?5W7Kemi@x01Kky0c&W;fH7}j4G~g^ZKe$$$x^|@kE0T& zNQx}~zEOgCCQDc@+ngl`#VTb96VBCoiZlf{C2q`eS@E_(!RDF!+6qp zuA~fpw|9}4aK9TJ*eA*~O{0l25rS4dhw7nS5yh`QvhAM7ay>yPsP%(;mOSTt%Dq-0jigax^ z4Pj@`x|#l!SrF5ZIBrW`Ts&Xv@J~h-XZX+WV4b*Z zJnz36lNwNWEWA64e#dT(saUcX4W8spck}r!rmBNYpx(A=%sVp8gXLj$yOQeA`M#MS zoQT&+WalNxw$flweUJ_jZi4s}zhDiq$!Tu1n1H=!+g!l;)F}NYn`A)A&?QFuQD0ZvzRKAe`t zg}{nuOlQ`r?GJfCay9QJ(HB~pBF2Mme{n^tZ1>f0yM4aBGVYHT(5NlOi6v|aa3J>f z_p6@m+3mG6?)e%e#YTY#tpgWqQ*BfFBc6i#Q)mpsiwWub*K6unJGc-0#aI!xlIugs zgkOby`19$%W9!!)gkN!AZ7I(_eM(tF`9RG?_Au^pS4{E+v%fHq9wPBfktauC4kIs5*J6%ZQD)VHDBuC~eS}##*oN*Ryf^x%?oKz$XZtPPiG9kQg85>7wT-FDc zDwtt<6`Jz8@V&D>NPYblS3!CGCi9-2jic+bfL0`4WFo0X2ykb@>c~7agIHdK4OyNb z631J}r!!DQz747P2<$j&&gTjpP5O)7`C+p8OY0h_=A4J>Q96y=G&@1A-asjK?Wx!W zw)0fto1zA`&oms8(c|;`IXYYwPj|SDttAb^MA1Zocxyh~5NS0GU3qb{$_mYDMi-k` zmg$lsnD2U7a+J=0Z{eO0v!A-Hjg-3i7!MDYCE@LFg$Jdp@<&~R`1O&DOd$x4##!8p zr}&#AYSNQ)KeU9(FS)3$WjfVFmz>#pC@}Wn2C=`aHM= z%Y+p(9qyu80GejIU%~B5C65tf9CV|^tdYohm?!+;#C30(<-Nuz( zX$>|1v{R$loP{LSK1%c1#m@_xp*d9{rq9bTo*x`C%*Q0~-TF{Qt(qQqhR%T0ZVVm)K zYHcpdPKH2UkOE^o;c0Y6xA(zobfcIatB{fzE9O*jOTQEAW}TLc3`K$pbdp#5vA0~m zFm^9MD7Y{lu9o8`GZMzl+Wh<>_NaLM@wi$!5jVSP>QwZ1l}-!ZtFkjGsruKF&zyqC zQRyigq63odHaGA-)*$&X2FaU2q+!G3a=LVjy|sCt6=Ais%gSi+!fLKHS3hClT3i-7 zHBcKG_VvlVrPy+vzvT1}- zaMq1|p)#Rrl#P-}L+RTeG$;6ouQ}#-MIgv_vXYIIMHup;6Un5~33rtR_@hee9@w2+ zOpS`*EUEQJj8;|0jCTJkdNoIBlF_nvQxMbCPA6LPw=XdAVZ{Vhiho94_{M7M>?SI! z2DuckE#4{47YgaiMmSmy)i21)%*0*`$a)EpsLBpB;zJX$CzG&-23UlILLKqq5m3GQ z+Ljn7Pvn!0_>en~`~a6SBof9hKRh=-GR4%Ec4$v+w{gP9=N{Nya({{$CE4uf=(-C-l?aug$miR0Ea8eVI46RyAZdVM4a|QtwbjPg~l!_ z4QjjwQF@?3=#KLOwn3(EHIK9-j#b=Uo)y`M-EpsfD;nbA6`z}|mr(Y}(WneRD5#ol zfA+)SU9c@_)nOyukms&IuZTeo+Y(C`h-G~typ9p?{X=N^Q_$${Big>v?*auuU4Wk- zYt)n2?h(zP?vp4n8nR?S9j)>R(!iXX;(3{ohx5N3^VrLMy5=a!!8Wygbxjo}! zk9u)$@gEE0j-vA?v9YvwbbnGBD#jH~@BggQd4IFReDQb2b$Nf}^gf_Vn1@kKeBbBF z#3uV5ML{djfU`yMJOa5nyhOj<*XKE`Z@XwfQ)I7(S*a%U(8uaIS0d`Wiu*{iBv{g= z+nNt{`Cxr_Ksueh0D~a%W2WSxjD-@P7vm0}Gp@S)L&wLSZ)=f%x+BOZV37v{C6+!K zu0$x7HF;~rHAr8;?=KcJYGGT$uJvn&pQ}oW%`%*u5s1zsS@XBymp0gbIPyQfyFGN_ z7|KAo9)+-aU(xC1gq^d>KGJs62J?CPHs*1>VqM$rT_JweX*x$u!G*yht3_4)&rHj! zyPMm9^0@S0OPCsU{_V1pbC(MlV%(BKg6a;VH$t}n3~A+Kg7pQ@vk zXi5z53bITz$7nKpfBga#N82c-nj#`t30-Mqx04n!7ZQwFj1DS~!Zaaj`{Q92L1)rM1` zfK#dYE4-GY>qq-*-XsovRgRu8nA}o%kRytL$XaTo*BoPftblXRa>&UVb%O4oIf#W0 zRQ=E}im5ThE2GHUOF4k^ZnpZjAtKmry8jm^`SLsZZvR2yrQBa2i=l<+AV$efQ<{Bv zqNh58PHzSsoJj7~U(bfQhAh_etUEGz0Y7IZjn=iBT+ZT%TSBq~<#TM7{%F+gRzFjb8>7Y`+^{Ii#IF%)E*Sm*;7t?qoP}Zh}gwZ$Ho1{2u0l)aNUFCAmTo zh=GB}@>Q8@d_6~Dc~SKK7cJX?MXdm9Wn+ zwn_E?14>`C0$=sl+AT?!ok?fxdAlN$58<_>m2hx~NpPHJLx5h+lMzR9D03G|l+ecl z>kuV6eG|-2NodT|wsB5jOeAK#Gdw~uOuR|i|E3uS_I z*+_k_-U}q3{V%!x3~EEqd?ie%KbA*hmb7x*4JK;2K8mw$G8%zgw!pPYmss21o7SjG z{$X@hVJNbbO>E_1OyNFw4Ay6=+FEjc<>vo7CdbzxX=vtg^}1}GW=e!;pS@F3rqxqx zB3E;I<^6R2BsmXCTliEgbTlqu{-vm^uzB^nFEA;_QnEUg3%x(r4Pjbb&$?UmzR2Vfmcnigymrm8SOYq{mKCw-DQ_87q0m=T=0u#snypKt9tA1q_2jH zCj5mqDsdKMr)cMHmGjSxJsJ>styjaNi9R2pRs^1l!0i&f@encls`=!gu)~Tqq0NjI zt9R45EGT$}8j`QKpRLp@jn+WS)59XiGQ!pxdMh?evWBMHdlkHu$#mD9C@M}r>^Jym zbep4@#N3GWRcUteL`IZoT-8*)aXuU17^Cl2HvqSyS^5#_xxd+>oChv`HqY+eA zwNivHd)*D1OV;X2Ml@JlhIyke3?|F9ty*TC?dFRd&abDe?8*7yzY2lE!_}Vhp6!n$ zb3D!WqR30;jBk_biWQsL-oHK)4{kHZ5obNi=+@YqKo&26Wv9dP`#?d}KyCDv+v_03 zDbFqIk1&bTdXd3Tjaf@ty2P~0sbX5LsKn$uNTzpw>h6F|;Xk?C)7Rto>TpmTaHGn= zV%hu|?Yl_~1b!|EY}dnT4F#rX8Fj{-usmMtJV71J89q5UDI`-&x>3+bc==nf!`APv zQb(&f5HoPWqWN=phNf`!&LvF7e==ybSYt{wzgJq=tCFfot~pJwN2fC@-)dCo?r1$8 z-qsu&bNb~oiZiVa9q0l@uCl69A5992(*D+9+`nDqu)7vuDmD359t!q2SR2ZiCHdtu zhv{f)to1_8L5_*p*7`h>S%RG@Y3k^j8(t{GUaIXOUS5Y*sEhe0mXYw84i1~N#}Cnv zVh1mFxa1-sD>Ns^ohd4yaj?Nz4y&;;2Pj!?$)K5;{zNzHAQM7$VKx&JeBpq3JfkO* zGpmXjksPDZMzB&ImGN>WDF&;oKX0-8QH+TTVeX|JR;iM~QCKKT#1&zynH-gWl$;c_w345cNeqs-Q4o<|j z;@n4n#_eyBv#);4HnUOiN{wgTEdixf7omf$(%nv6aC2k}RvVvKj9(7k4WASi=C>d> zz>xFrDpb4_(;dWpV{R{B%5^Y|emC2&(`~Dx5d$;Iuw4Ptn9CbN3%SOxwamF~J1Ldi zS^RW@AN=dn5_vyjfab5~4rptXrNG0wpBvkwGUkbGg1OufF*WVcIiLJYFzsDM+QvWO zLUZStlPq_6Twn4NFaLhI^0H@m_E>I4k0tTjiLHHQX@=Q$jtH@p((W4lA#?Msc++c% z#aXj6h`l(2fABu3)QFOoUW{@TPi(z-S{pwn@T?3eE9^RY33bS%LUl?@wNEe=vM(jW z;k|z(P<)A2GBAa2j6{opDgA}mlUTLu(FIcNbebW4dEuc;W3Vc>av{zYm7)|BDu{Q9 z**rLG9i2Ro7B^(6Vnw_QtkHbUnx#igb8@s=G+Pr`U|DxJ$9cMM7p`(SQbY7`%jW`5 zJ(bRKKjl512O$Z+KCxGnV8vv4UP=|639t}EPkgb7n@gPQc}LAg-P6tZWa zy=xaQAAQfZBdckZb-d;U?tHz`*NBrw{jiM3O)`Z5s_aPPk1uteZ5YM-MK}erP^mw~ zIM>+n7c;Q0@@TeGUDrh#OLX{r9KR)FH9DAvT{a=|YxD~b{n3C02n4dEGGNs^ z90yace8~$0oZi?f%a@@CMELg^F{D9GTYnB<|M+dO{G{JnuWVgUq(7(X6~PX_Z?3hg@iFZjey4GT^8C;?S_IY+XN=p+mdar} z0p9+R_)r;3e!n*Nfjq4=514z zQDSvyZDBa0 z{r|A{mTh%x>(*}Y0R#)~7J|D=&>+Fxo#5{7mf$3~ySuwP1P$))?*0~QzkANP<~jf1 zoKNJsGOCY~UcKsRqdo0T3c8U#7idh$t(-At6JOJ_2xdC-dd856+Y6XHNcdT6@y;oL zwx93yB|NxH`AZyb;MItL`s>x*;UAhUibU*mnUsu`fG3tB0(jBetKMnxj|XI!=U7ld z(;-ogX7dZ0cOLdcO*_9cH&NQ;I|omAvsbIalKRrd4RSXWcV zDy0bejqx}%ykblv_AOTDr|irLc}&|K3MYYfb+Vz@%_eo*z&^pi3-pfC=#3(2pYn1} z$*Z`qyB{(ihIX&BIB<};_7d?UBYa2 z5h!-rHm5uYlGk*#>7l8TJM;*c#~R*Y$CHi{?U6q-(klT-k%;R+BNjXhz5eglJKva5 z0p*zqEp4iYwIY2-lWQupEs51UC^)u!=Ngy2La@~57nh^iHa9N}VI*NB(Bo3b{;A%E z3rB@-t;{PdmCY^^)zbt|v8a6PL#7d1oOHTN#Nwi5{9w`&Tz=EE^2Ttvlte;Rb3o->d2)m890s)ShSz zC#ZrsFEARGUgO_#3h@j1SbZ5af>d2Gqu`j?Wi-axGy7HtyqG5vua~^EiZQq2_h}8x zE9e}${oC%`3_#BPdOzS zNsii`6AOAKK83T+VBxZ-w7s3sZfgLn%{arWiDdyfgOQ)H8pOkvULb*4@tok+ndNrY z!vzJ8cGjcUiG@Tk-}r5blIlQl@-*0Q)IL}1cHS~6la16D_IC5BUgl`ood8%880GKl z`ru7SJF+^Gt0J}&MssDt&j!6}sivia4(&XLo{vDRq(8M#z>Gu3|F zEx1RydVEFT`)%;v7bq0;8q_03lY1XF+Yggb1{%E!*^$VlS0#*#LkM4p5H*D z?|5K@Wv`F$EZ#tu65WgHG@{E+WWJ8-f~McWOR|X(M-ta!%uKO@rq|-y_dy@qc-M$G z{GBMJ0!8Z7ExOAE_j|m4-3U?=62+wspc3n!l}_sjOkO{OZE5{tdJo4Y)EN`mW~bI$ zbu2ubuaRXEO>*A0+wsR$cJa>0k3cc~=cB9jPGMfSzoZUNK(dniiUL{cZd2lJu&S7= zc61Ml&1t#!#zIT>|WApz1V9RBB5 z`~1rpRS>H8+V5)6N!Yx`&{zPI4>l83|Rl|oK*m~54UtmX9mND1!I{pZ(b zcAwM4VBJY(+dU?{Sqdw zFzz|>@H+em9Z8t49iqI%C=t=>Z@fmvL@70y5~<6+7ihAj%Y(eiWOTwVrVE1gs_qN9G;Hbj^ot(?g|4goXOPDNk)V z`mRUe_Yk59+Ac?wB7T@z5;s~#8T2ZINw=FcwUUl#Y1WGv>?Br&_hPkczDe;sHRhN` zqbXp}bnKo_w)HMHB0+ZT5!`sHR^E=qJM0U=)Ay;>8(U#Q^?j1y}=HhpTkMXUqoU*gpafAhN^lv)rVBe zDHO$FgQ>J&$=GcEVfC;(M0>~?_3zqeS|j6IEa5L;LJP@PmO-`F?AjnS(laXzYB<{o z#!~Y1P5W#!otFa-rUF@jhcip4V)*g7^TS$^kZJ58rD3yrk;7<7;*C;Roz-$tI5-Zp zhunfeIH7%Wwc;NsT^oHPH?2mim1fy6$$T&pS&Qs~(7Gf18be`{lFBMTD3}dLGC6D9Gw?SplN5EOa>h<=sxWObG z21S8?7`DlX4AA0R>)WB7Tu(lxU@dBt;>X=Ze!J6@Tw0ueKjU!Nw&{u&>lL>h%nU6u z0(7ui5c2q{ih##@WgqohK>W59)KY@>rN+$SP7N2^iy8d9Y90omLdQG!3_f@l;1k=? z;R|s-dJr#5%EVnJEL^A-5HvJ<`(By=R(N7hR#Qq#we@xFLz&K#5EQizzb+0A^bdEr z`P257i2kq}>(8zuyas4-sY{VK2cB5PHzyAAf&BOe;ant^M03VFxPr^YWzOGUL-NFjqU#XoP#BrL)Wnou+N$uMI9yWkniArXR z!re7yps^=VLxc7Trx_72l-B-(+7O=jxy>L#?ZM@#k#$e?5`0E`By3(Otz&P3^n6A` zC?ziT3t--AN$0OSn!hFjG7^KC^JR*aex%JXLw&NDe-x!4JwpTk#I~lc8A&3uuOG4( zY#oA?`(7UOkrX`T^Bq?==eMvXzvFns!)RKLmtsWuQ5@F~jj?+CgRxw+Me!l67|o+% zYoXL=sq;1Xr-UnEQ6+^AL`?OXsV0J>*)G*PdRt@YKA(`Z1e%th*6gnxz-h(be2;^H z?+~DxaDV7UX_gg)34wY5Q>1|Ar=KK{JZbzHcE|TjPej_oTJQN{tOBHrZ-AUn_cc_{ z#nX6kUSqHi)M4BMXbCuDMwH*P7oGL=`)qPfSKz42Q-$J59;4VcNIYUSqE{(vkM_O# zKFw(d#m!xcx&CEt?6?F021(c1FK!(CBqq2Qw%>P6H08?XOXHWQ*NSse3vV2t z@X<7ki+iSZ`;pc%e0zWm&}s1yJ(eK*f8Fh+Py7&ij|$U_l$VybnM20iwV7*9q;zmq zzt1IJd%m;buM=YVs7LaIJ()6R%=}akt9>Ss$Nw4(W_Y-^PM*|cBJlHXGpRKxu)p=TS+_9FB-G*C#>04eb^Yq8+J-lKOet z-@Ir4<7{mY573Nr@(?_)iD^RQmW%R&=D>Z6*>twWs|!~$)uYDlQH zt1P03?yfj6PZh0raSrZa^C*AWSzF#5uG*gWIhTSRpTWPD4v;&#pM(pF|LwrNP3lO! z$6Gqo{`&ZrAM^iqbSyaoUa2MLEjz)#cH=*8<+a2&gYU>8Nh@Id`=@{THr1!+|FJ<2 zSroHIUBeGxCZ8~aC2}^5b!1u56Aj*mn)N zs)`MtGa^Nkg8=3c3(9s_y{AQ*S0EU9LbEjAt^# z>Sy|?cKsM4y-p)Ex6`q};D=t>dZ+M@g#N|Qvw;>peZLpcM0?BhT4838RGxo1{52`p zIzODbY_4_nhTc65TUT24OKUdTDN)bfU+na`cf*hh*_`1|HQQZN0j(lo01cefI!)7g zNm>wHNG7Fgc6DsLcPNhT2i*41M@8Wad=SmuwzDOkY-X-4fA7O?VEexO_2Hj@_Se+a zzR?r^9E(*wikogF^BIHlD)hByPrh*fsPN>UW`DkEkj!q42lKtd{TVT!EIM3yR#@#bNp^i$wVb*UL|PLjsLC zng($+t3i#I-V^;Z<;Lk=AHoSuef)zXb-Q{fISPzeU0xm>%`-K25p9 zcX;b+z+klm>c;hSCvmSUaVv4P+x=qS8+eRHD-UW#gDvK%CJcaU6|MEuxhuxphg5s$$_l+5PU<3{!9 z1;E7TYK;ja(m8%8o0Y^=RM@ZzF>5U&;a*|6EW0_mV?W*yI?VHF^BLa?Omqcq)X1o0 zkCA(4YueS3Xr@wozqA~*?FHxy!yR6>f7X95oywct_X2J-mKrG(97&?H!n3zf_<1PC zO)TE_rq=ou>}v1wQ$%1I|0v%4x!b8q42qg!eJ>j&Dz66;(9~4WtXSrH91hoOR-F97kuUa3+3{EvIT_4U~SMtj0ywa*g5aum%32mo|bo9|` z#DaSZHd9nCN##COge!-hp{7+vkG&hE2u*#)ImTZ6M>f-P{;RI%3y^PEz!7+OcRbcEmn#pY zSguZ~9BZLgrCTo__4;C6$c9L#-2&MZCQBgib%e`i**Q>2I1ez4Y)~#So{IXP7mAlV z3~m5Pq)R&}gk>8b4}2eSa4%u$py=G6IxM|5zg|Et1;J1W1)Za{7^PEf20@3$V%9B{ zT|d6Q2!Q2k^4;GVG?cvmbkjW);Q44VIlG{{cI1;SD`)JQHU+>2vssYrM-D~EINM$( z7;;D?D_3ARtBoV4UVQ3B_iU4H1`lf4gZ3W%`Zj$1gKG|d#ol|xiew*jPjpMwa%eQA zVy`Ex$TM5>z21H9>b71;JR`_DQh}x(PT@6p^tKh8-ey$8UjN*UUD+3oV@6u>s?mM) z8cA39m~4Ui^Tj`iQi%J{X#6c3LfT2Hj(9=QKiXckrHKYrsHyVP;`;NSm?I58e4rv1 z*-Nq~uW~>2(h8Tk$@hAO@=D~!PhF^=K8LF{SDtdcHke$()8bm`Xvqgy<%;J9XVFm&9x2UL@U1t4njUaX2z#WT)d5+~TAD zO%cEQAnsVFVqn_iROd@~URjgx&%^B4VszloXO}S}kcsCbS~6Jg)HkL2-Cq*7y)at! zUiG^Xa-`WeC;eWNe7Ak<`y3;1Mmd=;lOMVJ)FNj6dsab)WrBvopFdL%JDmYK@V3i% z)3O?pMczd7lSl|AZAt%QVUk|PIKpW~lJ7#1(T6bK=M!!SR&VCvt)gd3ZYR@jejOx7 zq|v(*2`z{7s@Sq3v8KOXAfv)S9~*7?*}}*vhOq+@62X5<95eq;RfUMlcMm|}6>de% zJmm-!mrssZJ|8~y-p2;%=;)jYonKx~%c#0~tc|0jDvMyFDpu?x>c?~liO*NT#o7=O z5+1{KIrIt14vyt&3T>dytcOSx&z8-~T&Ov1RS2jPv%yiIFRM(yhhEO3Xue%Q)A2;U zRBE#@wQ=$dAwoaD9x4sEP|Xg$zWJV{l*v+{q0w*3Vrd~^jFZkrE@Hwl(c7=a;8>Qz z$)eMg>2ZI3yMP{IsiP!QQ7Xv$9?!^|qUDp^Q15^kzYpJJo>ca7eUTzn`7af2tZ#WG zn{hIGlqrq69xz%q911;~cMrP1=Z$KNfMgUS3qo6)f~Q;1#67Y9_K1wEMI^3vufUl%~Jn*cT; zL8!i7@%vrQ?dDkfpA*un`Mj*z!`M;jMGz9w?7pIWcn&Pwq+_?z5x+caxX&%$gCzTG zvnXbaGVd~)z1PDwv$C%sY46ryKmt9LTR1^kxe=~c|A8a1Q#=XX?bl2@(M&Lbuc!|0 zk~=(Hn{@41gy;P_uLsIu)QTQbIO)zftje1m>f~a-No32knvJBK8fDli;IziVCXC_8 z>%8;Q9L>u-sY2VmnrciPqq$i%OOKPEGl&)&_Ss_5?0|G7!>~%~laT9`iWPsBm62Ne zdRoeVEc1k5tM1c=qPHZ@%ReN&@2+MWjy%|pMVmEXE(`C}J-6v8yxmp0$Xl6X&A;Tw zFC37((rM?_E0$4SO|a=dEjkxn@9r0M6RAKa%G>P@nT9M+$I0lzv|W~YZHy7^^v75? zpx7fe{TMGD^av~dNoiasMNs=2nnv+N3=ihea412}cy}O8$-4e{DYu^6`IK=wYM3dm zS?){W(v;K>L3ZjU!9;VTY;(R`&tR~AG~RuuT1d+H(%duCCZ(_{bE(a?b&}br|o;H{|E&13&LbfT64IcYL__dk^8R3Md;r6>@ z{fj|*$<#(AUS^+;LuFR#SN+o?35$&hHY~p|{ar7WliQ3LA&Lb748X+i)L4y&(%;Ay zvdu-`6WpX}L**NOwaVS(6>=K|Uh*!gI z0J-ngbnuC9#VvZbvahlj(1zMi%uC5V{HdeYQ5J~Yz~;2W9`{nWAUQ>SP2=* zoANz&Qb0ntjyF60Yk|+U(2&-FY*^^mXhEoGRtclu5!(eEcK7o=O*oZw=;pko`IAqx ze4&nKb;V9@H~0myE~vO?HlB;`VHPth5F!%w`=%q3J`TVeTP^^E?bWrGO=1J(Jcnft(Jk{S% z=-z|YoK35On+4`}H^AuL0V7(h>}?h*hDh21BifV`KK@ztG_~Ur!WEVZY=N zAbjr~R^Gcqgg)iVSN3LdTxfj?!Esl^FBsF^8RNyVECOW9|Gl5Z@*q#<$=Vt{lCBVB#LDL)9A3=OUvACk+*54!Qt^J*+HLB! zP7c9Y{W(0ANh(@%sOVg4K{+O~eM!hV+{yIv;x+Th;CKqeppU?2Sq5%^%WjrYz4uFo zJq=uZkbtKn&&S>UXs~Iy*Xgj2C4x$eQh*!Mkb3f5cA)j*)!r{E4eUwl3 zDDoDhbv`&HjM9C(Ln3Bls#Qsq_~|!g3QTajeLBZ9%VrWen%Ftx7+oX^3?&8w1(qz* zXUPrv-Y*310zp;>W<0BnmzYPsPkG*X2>o@gN5xF{-T}9Cr+V72%x1Bq(7PIeojWW+ z3GF(x=@N0I`hDUnEZSICR+?cYj)ppaUl}h*kC?}>Y=YhB?3_iY^oon{zCQM3e8bzw z0%z1M*J$jT$<9(KC%xb47Sf|q@@qhYJGO` zG9GS7;+0f5rAoM;8sl1vp`PS`^BidOFxwePOlgx#+ReNe=$PY1*TyZQi`1iJ#G3yxpvP{eLfNJRh=>KaTcofk5N#9uD-c#U=Xl)tm z>d|F*e;C9-4w{9weMs?3BG(dEGg8UD(d)>(eS_C#kP~@T#PrtcU|a}i`Up@c$~AnU zH6Xw%)crOCy(B{W-d=} zej@6TH5&Q19tj9iKFAFU;M%*&4*3vLqt}|B!(~Pd;wt}3LR;bnuZ*8* zbl#1Q`+ITzpRYKxc@y34pnjhG*A{^2CJglTKMJdl{xfR;*oG&iL#tHTVS9n~Ig0^_ zFx%l@ef%GB!97Hk9%@Wa2VG8vICS$*l=@EwO>*V_aY+AJax_06*OSDXd=&a?p8rQ= z&7lEyD5vC*HX9xMzXV0PH#zY`__zOEul`4p^^JMk<y6!= z?~SSgRPm^)vih*Ur_p~#YItkUm0O@0c z@}{0~0==ebx}3&PK)PaX`81H-nqs&KaFM=8L^vhI%j(OA(Pr9n>~Kfy3S4zo%GlBT zt#SCe3lj~7`Jc9>hp5*c8XVN@e_feHr&iG$nX|odYShWpS89;wKZZh`!?3Fvj7OKTVX{|n{SAkRno%<^#t!Z8 z2BJBpUmqR`6^rB{fS5L1^0Cx_9IRyS-O4yyyfObh|60B(`hV`!*ABiy902SNcSFeG zGpOSkj=5MXRb-S}ciFQk6%E)rwnkHjP1~A(z+=4lMQ4i`yso6)PywmE{k7(^NBy@3 z~2T=(0n@9fNm#rpg`NA-B70eK9a21+EWWl*j}{QU5H4|9e-&odNg1TS24YNaV= zC-I!XA>Q@JkMtA_jSAMyI zd&&Dr`r(60x%Tw40cX>7#VJ7d8#lZYW+%NVRu5ljvM#@ILjXVt)AvRem%H?T3_0w( z#@JWAtjpUy?%^EvA3{!q4lx(jZ~0zob`$Y|N`yv^2a%Ao=+~V?lXLp)Igot1__$iD z5vj^dmYJEkcW*`r5)b|~CZ8~uH(zT@j=0R3 zeEas1_f`J`htrAy51%pamw!n7YvGVTdpHuft$9Px=)wrf9fW2|)C?MHW$!gxjcB3Y z<3Z{W>a>hEq2C-YM((CrLr`7E04df$Mo~ZMw{DRd%SUh$64J99?(SNW{r6^jKTx?& zXG%4=nnlM}63*r^0d~=sPz)Jjo<8SvkuReuoIqRUjb))cT)yPJ$m~~Cv;e?JeeewQ zWM_CZR-K)ncLKOV2fh(ko&9sFwwE35SRF(JoS9iVM%r}C+*m>8I;--lKaK!J4a4ij z%f);g8Nc4<{sP?)*|Ej*Wg!a?dnT5TuQ0S7S?xUuEm)Z) zd@ebhU#YuJGLmwgJxX3LO6V>KI4gWh zB;NLWehn&l;nBzwGkHTj0Ayx6npC#$Td$@NmT zYRk7DpT}&2gRsV=oaw{`HyujGHfc5L^_L%zA3ZNd$5Cz`17?%)N-xIRU+2S%hKD%T z@6#;}oBOp-=D;*!H0BE?433pPFcT;zahNiRv~a`4QY#6z&+2qTJ0f{Ao=dp9|5uEU zHHpovm9WPGufYz@)&4Wp)AMuf#Ta?55Hyk?f5-6-3}t;Hucw5NJ8Tspx2CR5(D!n0Uyl53W!+*>UoELNUR zn|m!Wq5|@cLg~r9%EF1+lkC*CF2~5T05_a!%{WH>c&-Hb)crOY_mAM4)Rp zLOLt63mXPUYB#2{$DNXKQ!nF5L58gxX0vba#-w7h_GB5Qy91jTmSNKgyv5adqz`0sO-bsIc!yVdU~qFJX7v9Z~7K4tZYzVv|^z=C0TWm{P$PZuXOb= zu|LIue*3s7CPn>whj=S-<5&rCRn>$kyU`C)i^_!eR7zC)S%BVJp`!QXAFau0f3QqZ zOCr@5y=Xcx^y9%Cj13+1xiJf=}c3@o6G_00OF z5SM%WrtBLeH z)61%I6PVDH_7>Z)tl#^^BA+c!N}P)!~mMXZ)8us`hk zPc0ThE9BZd|}&yZF}SePVCO}KhIP=R0*WgmHHG>vUB zzNhQ<`Y}@iFtKXtj{TVPvy7hQ$DY&|?7)@f^8iYNx$o-=EiR0|$)6#<=|6nb_{W9& z7}s&!fBmDhaA=Cm-wF#~$1JkXY^P%Y=-psE>`!Mn*)Xxe=#Xw_wKbSA z%akj@IUEriM;>`RNbflaq)(w!pB7W(#GSLIrP>r6-?1GURqy)P;Q5r zzVKw(Vk3msu{HCM!6}$?52(VqAYX`;j^fWcoo^ASPj@mCFTGCZf5bSbb=q2*ki+r& z8M%u+w};S;2e}CR>4b`D!TKVg0J1{9UroJvJu%<#5z;U=glTIC7=s7i;p(t~@hxmQ z`KrGFW_=IbCO5@N1i8jcNt4G3Pveo5F_%EwKiB$P$PeWLYS2wK6Xg#D_PgdadZ8`pcLZE?=`7SD3OMPvuvU+$H; zR(S#qZA`V$lXYW(J}F}0J<2UG1Aq=usU+iYc-%20eLm-SdVY!}ymw`oS(`hTwH#1A zoKy;~vYDVCh@-_VQmkuypUAYWh|KgjOtix@^!!mOiO7TF#o*ZATAxf(&3F-@pXyER z{C3&P{Gj{JA;(TDv(l!1vS#1F%`s1}Tl9e6{3g0(j-2=FYPEB@mp6oei3IOe6rxdH zGyG~kgJs`{YJNU0)+%?0GinrVd7@Ght2^E+tu}aCu~M<|+U{Yn=sZ#J7{LNI7cWV+Rgfe7|u+t;53Q%%eSqHVMqb>qwm zN{aj!q>(%&n;17I%Rm^QI{Cqp{97{1|IW2E{}WvwXp6sp{Z%PsVe5#9R{cTJ4N}CZ{RoQIXXd z(%h^-0TqHyA-m+qE#L?3e5;??(9GStT-zXI0}hE)bLV;b+SP1*YDMpi+4_k)1uHaz`jW2aO?VFI+5iX!=MKaAT%!8Rk9|s`+t}St`2l(SF=b_ zsqSI!e@vl@P4ClUqj~`-M80rvzV9+caDRN~8Lu!;71N_)0%${x8KPfE0ty3AFT3vT zzc{43Ff~P+Nu1t2{5Hp7HKRd)t*!C6$LxFAO@ECH%T|FgU0~dc1zWKZxoNTwUI;M} zM}I94KkT(aDDd)Bp+v5AGN&%b{jn!AhhLys3(aaaxddsChV|L3L3J-a#+tQEZBePs z6=yY;WVXq%uon4cty->5Qy9C-{^rQFt9{F#fjLBEIF2sMrHcl3T{4-)T!2#H;kd5g zVIlk8Gf~HUSe<)}6f(_Hzh3;*`NG{5MOyKRV0{zpk}Z9g_d>nauow^(`teo;ARZLH z631KDoKgX6>zwvkhmYgGXXu<9;lF<=mCYF9PdJlF4!3)Og&ydCMAGF&mF;S{>cbensky#7 zW5LX*fcQx)Z|ZXuglQkBCd zsoW2y>tBNf&;f%5(7h~i|L^Skzq_!;0;;fCW6^;^3i#JxAT3}pkk(V9^S@3|hbgk| zdWUHn90BNGgCREogCRHF*`NON%mYX4Gey+VII20q+Y_^E<@%q|RRHxlD z_wIaSo)z)(!AS{jTQHr^JC8yxOLl9ZWgc}%N4VjXx8{y3$LF)%7DXzLV^WjT{>SF0 zi*5LO9&hQvUu5uw#2iaw{gH%}ZdZGVvzu^XK7vFom$gkYq7fr15gEOPq#wY2KOj~Z z=$7lgla-rU(F($;J(!##m!WM7?GC<~0AflhoQ_mQ)-&=b9_JR5>2!F^<+SNLTiUd- zEf;{?8Bz0@vTWhdIC($FbJOV|!v%}bOKO!e;1R~WpYA<68AS{Mm)!hF^l;QOX$^)Up%JQ2elacVyg8{K%9QupN~k#V=1h` z7<8I6yY(LT7xGh|!V3jFResrdbb{vm`E4#((`Fm(x3e-gT=U7!=5LOE-H`bcr5A5SvwL;U0Y*EshNz@KS|o z?&}&uX&|y}i}N0Oq1n1!6T*e>CJ*6oRu)*i4An$;-z_6@0P_Y65}XVVa`(<^so^W- zPxAaAgu`S9R7&}ezsxlo>a8VCSKI5wkU|2OjbAc9{Z^On3V_|)9#*;9ImY!)c03(V ztUI0in#g3Bv#70o^|Jih9}B#5+u55N%D?su{(8fBhujFU{BqE2@wyBede;6?L6N|1 z{HfLTIY=aeK%!iyHShk?NF!z!Qlw)ZrNMreE|yj&8oyl=O^$U0<1Yw{Ehzu9wVk&PeqfaVn ziqBYDwxz*#T9!5f28rOOfpA}bWID}7w2!e@P9|VS>yV&%!W}*MIs43`i*@WDj@=;7_)S3yBzovW^H4u%=0qO+P>Oc|F&_5JS4LzA`$4mPq6uX4i$5X#vl9+y2t+WWY z$I%$bvv@u@U?dF!@lRig3wp|XrG_^2*ZqX$s~PPdbYdvvND>LDm1)ct+aT7{G7I(e zA-g_+Yc)A#H_*}59G^Ww~&ZoS3?3o6^H+Gfd<*)pl);yj!VO~#+#I+U^ zl5mU*TpYosq)y!YKo4v_@J2dR)g6QwXS(xRkughm@cMkKqn4j$j<5A=KpdSKnKvXu ztJlM~t>(DEgO>?h56rh@!&wt34>t|w5&?;WAL8M5I@RFeX5SDTb>@Am+ikA9pg=Uav(+yi zN3dRD?ptZQcrb3_8H$KG1hr~2C^D&}FA=rwkt?5s@lzorsV<>Nk^Mjc@DMkE>1_gA zuM6Vg<~W(9vr~}3)bVfzSWUBK)7zeN6|Ey^M{r+~xKfLhO7)4cGH3$8J&tl?C>4IL zoCIk<19$CITI*whziP~va5Kt$t!7&xIK&@q@X#g@pWrhV>sz*>Um3ympnS)G*PN9A zQ#_XInZ51M_$vXPipql1>FVSqN?d@fLSu#GJ*HD3o3J?&Cn$VrBo%Pqfx$&o;{|Lt ze*EFn579uIckKi<-iI@m0Lq@``gArJz*5OTz#x#(sFzFt74`mYNNgZm4M8{_qvQj$ z8g;RhPODKV+Dm3bMR3KhcT|cEQJ;Go-;*L|f%uaVe>dJh2rQ?xdE8UE9s`;#i}5H= zq8321alMMX?KD=kF#eqs`*3|MH5h2ZtvgzW z+Cpp4+XHD@?1XSVbXv_4#5tC@?8{74C}XxD=$oI}_CBr0cAO$a6(H3zF^ENddSQ%lV<4t~Pywz9zQP@wBC^qttExjaoIjpOlU5m4~rIKSY zl)qR|fV&NbR(r4BgdbYcmdoKtkm0Mm)BVf&ZQAF@0+R_Hlimxms^uZ2mkf{lV-4Bv zeVs!EMIbJlN;4;6&XwcK_}O8!=xrL4YuwuIW>=(s$nKZMsB!Az$u-E&jtlQOrQMz0 z*`5joAz}(ja1O<@N16pizP&%ZLnZdGm;jJxu!O?MYq8I5-YwpXhY=+kJ2R821h5QB zerr5#e?WSS<6q($iJV9>*2|wn$y_auSAo(w*AKDngBu*zpJZ-pu;Yc+;1^EM6arp1 zi&DzqDPRWUo%m2Qfcx@y6b3?YP_bG48Fz6X#QXj!Yg)31;^S%nG(XBGR1bzkU2~=! zSa@#^k!~FlE6sMO_IoUScvKR5Zj0O>dcoi9Ywi~2QDi5aR@z@@IsO1hKJj1gOiti2 z2I%j>62(^A3^4ES?#6L5!yyH#%Lj10yA@%YAqlj&i?h(aK5+^X@_G=>0YAPw%$tZl z-&f~CE7o!vUVIU3Cznnm;b5;f*%tw&tm%BKOl@|8XAXK~(Vs(}4x1(&Az0#eiUWh~ z3&%FB#WQ9A&-}r^E*My##jWuB0bCCU-V12NaL}zks-9DtqaS(B*p`VVfpR?9qYG8r zeAWrYjOCFnaiPSPe3#cfJ}G1ajnD0r($WTx{GB%_(%a$n1;S4^(zf2Lu-)Am&`13e z*SEf+f|S7oDDgO$VD6j=Q~E)WFyPX(cpZJpV=_b(z8(IUi$OO@;804d@ECMIrC7}m z^7qHhF}sjSnoRm{Hv-d5zx?dEvDP}M{cRe?FGd>0QJKCvVyQ=80(#-hfJR70W-%q= z*o&jq`I_|S`AR{c3jzm{w?bHT&cDs;*{rz0iq|9Fi%K(QT-vr0%k_REmS$IH6@>M! z%L*#MmBxtx7Qp~`)O~AO;Y(o3lS&Q(a>7gFEhUB8Kf7NOLyGYKL9Mmdr|BRM^Z7*- zMPg`&N{Id5{B-3RBU^~UPaeQ@U)gOBO$Nfp12jlOd5*Kl+e(hrN>6yHl)bOtADA@~ zK9*ucL!gg|hTwgD0LxsT%oPvGfH9?cP=;-5J1x-L%$s~Q9X-NUmIBk0hJ5o4hGTIi zo$$F3m7|1LI2|YTJx2XNe5rOuw5d>|8wEkazMVhFL40*n$X}v~Ot~gLHXf2G9E^$F zlxl?-9|u9{K|c^C)EVT|0YbXHME<8cMO{oN$o{WUO>vmO{R)Br@&7_BCGG;hJ(yoU zu=buQX}qwCAmAjz4?zKMa9VzFf<1#p{S=euGOkRO_HwcP%8gj3yFfJZzM^Bb$&eJC zMwOqAXEJZhLS@xN1D1nJU^+~z+X*N1nUUlj_(;N2HY_UnFi7mX^o>Mx`ZM*AYZ!mR z7osAYqkcLZNIi`fq|59=Z>hLT(O$lyNqu5PmV#Z(1P zJ2Jq?>Ss##_NRc%!~p(6Tu<6gr*YQwF(kPV~TfFK9W(es@F!%XBm!E|s)& zR|@>af>+VxbV$5v>9a@q{kvZQ?F%fr6H)*?6F6}gf4TN@?z7eY>X1aR&KHso#o+sA z$sEzEqkfYS-I_$|H6}GC=kmcKu)IO8NGSo{beaiqbQ)IjI`_v!SRLiPp^00CHeu|6 z3U(J;gIiroIX*iZ=K%QL!nFr4K+6x6TzbL}z;(J^7X?)G64CM%7magmq>|MJZsx?T z4XEWyd+xtN8e4$mK$jq2sw5kSB{IoqG=a^b9#DWV5SQ)oMHSB*gF6uUNlzZs&Udu( z*bh7!_YCKIQoxvbkFa{sd=WalLGHYs4>TdD+!1I8ttU&-MiOowlL}ux`CYuD7wCz2 z$4f+kk%2sZakMZ^ElCu#uM7BGOgmdRFieC_yNUso65EY{Zk}oC;*t zqV%{n&Bj#_O1SQj!_FVZU-#KLAk}y4&Gwk>y3nlV%JHA~648l|GgOy4dIV4;L!rM@ zh=$ZWh1E1WpMK4k3XtmkY(98fTFz3~M2ANsU>1)&8g*x~Us#{+>dzKFhV484h5z*a z`+36aC8NHQ~I7X%*&-WP_Jf>lggL^45R;GcX6z3zy` z7G}(mbWpn58XI~lU4F<%IShp}fa5?4hLoc${ORAt?)^~NNen6uF2sRjxn#KSJ`vD0 z0U-Am+j@tyrEG0b=)@A|l^B_jdBQesY&vs$;SfCV8sCmQdLju|bG{SsC8IGkyHGnl7Q5)vLFnJ1{d@G`RdM&5qUM()4SkF@evXBJ0!KaKYJxyMIH z_yii_k@XTa`z_o7q?6Un(?@hcFtd}NC?H12E-sy5EZHR#Qe>b;`LIp|M26SDX(iGj z?<8#iWCg`$yuo({Oe`*ksIL|dimZ$R`h)wkd-8D3{{X(p{g$9~wIW&*crQnzl!sKRUP2@VWS|Cz75#c$;L!`# z>1=1{h`rH|fXg<~4rmVu2hBk)t4qXyVu@@6-R1=pKV)La>0Wp1!4>bYBcoBi+khYk z-4Q)jTeIKqPtG9_3}r=pR?Jq`8cLn`!f{dzSO+-*~9*9b;MgY7aiBBbwg-HZo$Lp0=pYSX|s#zU#zju2hfme z9d9JP5Nzf=VJ>n8a5vE0aCT?!tc&#Spi0z1<4oq@uTOgjjQpkmjxujlt4ZyTtA2CiB?y$|EO|n|UPWUM(CDo+ zi_~kGSS=?LUVu8U>&rJJ%cYvYw+gr3TDvyir6QgZ;JpYxmMPue9OD9S5$ZRI9MNPL z1041{gl3Z^WMq;FKH6;_!T?FTlB*Ap1fr8Qs108Jp$Zt_qyuw{h@rub4d)}TP@QIW z=Y|3O{P65e-R}EsylrYx*Om*y*u8iX;jL1^L`I{d^cYf6C*T*0B<3aFNMbx<!!A|>L;vea-SYZEqT z1ZOJ{`*g~;jN|=D!z4cIx?UG}3>yABKy3ys0`X#s}OQDhPr zTwYhmz&c87f;E=*IbQ?u$3&KJu8|b(@Dq@|HdV1sT4)$*#A30ho7ne?1?GPu`98(8E0ki`0Nx1}=M#@6$Hh?b7z#S4SAwLo zeA=`t`d5G#r#M>+HeDc>3fM~M@41wUV*ZrN&`->kW7Mxx=x`2gDp9=>KYE;RKuwov z=$sJIY4fCKi6C4~)kDRlAiIGe*>yxC@DhP`ua7ZaU-u4Yg&?VYi2Y&Y+dN#irGzqP z6)Bo<*sa8B&8Ks;ns1}>cNLp}_OLgsH8c_qQF{6gC|5jo=uT6?uQSZlTW*@_!6@Qs zghe!}92-mmGKEWh0?=*tu9Mk6r2~=%_UQ&-W_c42AF}0ze+Bwh`fxM?at0vi#Vjq> z=iVGIjCTqJn~tWqR*BJ6k`Zz{S-3_(aZ84h2#0)Ow_b^Zzw-iE;H62dCMPXWHSS~D zmeqZix2LNL^B3s3g{VXND2#d=-*A{58h~UG9L`g;8XHy7L=<7os3G1ed+Nz;7LxZ6pTAZ^w zjaV%=NdXApSV7mIt9{e8jLfa)tsh62ewOC-PH%+`xQu{L&{_EahRni{=xYLi1U-?6 zCKb+mxIP#<|APV!F@jPOP3o}?ip=JlGgB#73(gYxAx^DAlL<)1YPit(%-~>xL=E>i z{#;a&-lr?A60SYf7(CF#IumW4FyGVxgfnc*p8{LvbB_pN7sUTV+gpZ38E)<0!~lbU zFw#gj2-4Cm-AZ>#iGT=#bT>#ycZW!W(jC&>og&@Md$Hqr_Op-w*Y|^8n2}?+XXd`v zI@j+!m*>rOIOqimLv~1yQqHlSRJ%Bsqqa6BeITS>bqS87T8G;5kG_mIgQVcWXXc`@ za(!56U8y^t?2USnP9amFsxG{JxzZlUZeT&aODYxrV*%YlXT-}}9G#He>}@=rRJk(i zdY?hhaG`@n;^*ZS(V4B@nEds#XqvwDGXTScIVN-2(u^%OopEIl?~7lG6P&o7@0hS7 zn{-{oG}nHJaU8#kym9b>Un`UNU^Dmp$(`=FI-B^>v)u5#cxH_eyR#jxWXGQ(9im>d z-?^9HTHR8#FAr#J{F0sfydZ_(oh|!T$`z|3uykM!mgjqXQRmjln7A=glHY(Af@q>L z^#ucC*`#YPs4{8gvDd3i`NQkxdOPE3Zyh{so{ zOM*)&ef`fC-Gh2!;-rab&CU!1p|>wE2Pk%*VUh@m1wXr^cZ16sK{2w*km9yoi0);@ zRaQNXr>t_EbNX_)>^)y7Qa8CmSdV?L5|xfNmKXlF=cc@NT2NC$j3=W%ksa3m&BrHC zK8^m4T`@&~Cbd|n>Qgol|6w)mi3ITeaPonMR&YKB@VikHj(<=C)z|_fYc7c}+8~D+ z%O|02w~AM!?)SU3*3N&9&3+{#3sXtVF8yFmtNNBM4-HLEYa^citda;=3zz9Mlhr{$Y$Yg);c4-m1K(rVxDZ|n2sdEtcH`|<0c zjBuz<2y1megGer66(O_b^)zXf=0|O(Y|$=%0Uu zNICP9?>UeF^%~30YK2_l_qu9fmgT3^M%RnyZ1O47wkrrXM%GpDuVi<2MpBR02Nf(1 zz|=zfg_XSROi{S*d_2y1XW3tW%QsC89!^nRVoOTnIl28YUx`_omg0o*cx)FNzR@&? z*r>QN7`6GGI;{9rIKx)5zj8k)RXr@Xl4C!O4L2XjY90ok2g``E1&`Bhwp_*3--{F< ze?h%k;9|i}3f4 zpUy@nwQY&a%<;ZB%ZS5o*#Z$rFd?k(tOrIBd@egUXejE<7=vX2A{Iy0SI)BWFD2|( zJ2HW^JQg$WLF)@Px_;{v@;H3k)tZii!<-b#i2-o6%7#}Dfm@uhA2XJI$L5I#M)WxO z%JSIx)edxbO@)bQzJ6@KbiBk64Swv}R&5-Vv_%||=E|90J8f^!U{>873joa6Z`PQF`~|x2w5Rs zis`)KW*w$bpT_dW(}86=)s}N!9%#f6`SWDL))B7SKkHD#S~wx3G6{7bQIR`z;$IUO z&$;C?9skCKN9U89<}0LYT_a|urtmsQGgIA0j%Vr3H9da%N^Rs)sS@C%C)EB-M*QPS z-JkVo%#@UnNL!n%od$(JULxL|pcVC;kJ->zrpybq8!JhmzPw@W1Nq8!0V+ow&;H2|JGDk zE6=eQ?-<%$U?3Nl>9ED3IX)pLqO%190|B$b`2jv*!bC^_|&d!Ec&GrxCOytDhNa7Q%}} zp_oYE?M047{qwL`w`p#Ykx=&)J~&WQ68TuHlO} zF#Xt?$DVzmwi(^CU^KJbcCJf75o^KqG|$K1BMMWsgFq@4TSAk58Ob=v!^cS}l}Btr zpst72T9e0wA2#v^ed$)7=p%@;EyM4+K^e_aWX7KaaV@zY1iz@1@ST}f7I4_#^0-9! z4~A@nT5k69QX;eyGO8AkyWa;$X%RP>vq56n2VV%G%qPy zNXO>pbbzUmt*|(73Q)zS{ac?87O^=Q%M!bcW&fc8u$IV&d2vB<49OvoYFEROl*T__ zn%|DAxnVd6B_J2Dc5-zhAmniR1d;kJaXGrW4Oqin@dgSt2f;`%}?5gAoqL){_6xq&XVd{dd%z2$_dY7wWej)b# zPzU_>q!lh~#>=f^m9N~nz}f)I&!#0`DP5L>%v$FNYFlvjyydL<;ht{K^^DKY@@qxl zYoTlB!{tofWqJc>;}<)p!PMFeXJOHI+RaYi^kInsW#P~(=oqz)a1V~kS9+j&lb{!4 zxDAPTCJ0nt1n}%%bPhu8e5mlTsWK#5a!>FoE^La5^i#G~djHWzQcE#q!SYm5F*d{7 zzV}Q`X*8g$NS!db*o4xRZ1@_Cn)Oh&7f{}^IguVki%xUhNZlXdI^(TiSD!>u2|N`F zIKW;ELihJw%<5_`w;fB}u>s66GA|7{*jH)T&wp|KIMIzIhyd7Bia+_aL zEVyN!hle2)X#pk1ueT-i5BA9L^YxXA82erQyjR1d5_o%FilOam7ZZJ>-`wI^H<&q$ zMxj9Tc>3#g6*klG!@X-VCoF-}z9B4~RDVC_fa;c<^Mb5P3Y!OEC4!uSmTg*gvp6>sU7$O~#B74MsR~8aRf2>Vg z$(h`B1l0diQJ^FRILPaUgOr1We`_WGLSP;V7DPQwFc$w$CdLaEU11vp7q2^B2!&sy46-Oz`gKunfaBf|pKS%n zO8#+%u-r2j#a8o~thtAqQEDAO-~aS8&ewZ4_ob03TLT85SHTyO8vS*|DrdXzG28yc<7@b3(h$Td)|wADx^wEgx-q@sT9A&Am$v;+6E#=f$o(o-XGD_LqKuJKB%XG?r__746L=v zeM(ZVJYq~^Ikhi{X9j4j%n_5W-dlT5#j>6$T4$GqOsffNl_E|;dWXrB@oF8+Ht~R9 z7*qyQHeDMiYbImL^v;$<%?Rx1j-~_kyP8bvmj*XMpP^ft#t z>g3t|BULJ|z?P5qLSL4pZ>0)Q_HV#EJ(lCM$L|DQelDx|usnry%g_AyEP7dho1W7X z;(U_E|5<<0Bt-QOy-vjZc%_Cx-RFa*iA=Yf^YBBr-v$}1xqV;#qiK~CuxS-1nQd7K zERNS1o;7p+VIL`_=^M@z4|ME9Rx`E}DxDx`XE(5&E|XaZIMOWjpxT+U^*Wkuj$ttn z>NCFs9m(yBcTwin7)C0h)XV-w;mI{+zwp|wjgF#oD~K<@1;fhl;cwI^&T9x>Z#+U7 zGSSpE76WrPsCqx0M<@m9v0I$?N%*i6UcK!KQ%{ym+~aq8lP6F3rzRf1(eq+4XBWus zmeYq(u~e>3i3e*YcSZmue9m~`O7lCOg=HX>BWAI|L!3$~GIy#}f9%$!Qek!jO_IC{ zMxe|49OGF!h0c!=!z24x&;(GptR}A{XR9HqdUW)RR4jsZ!HBA4--J9a>AQ6szo=d3 zO2cBD%EtcZvWL0e)STWo{vpoQ7*wBKrf9dg4$~UXXnLVmaL5l3<4{r|H|dK59Rr@^ zTCGM&lAWbeOfK8W4uL+)ADQF5F?8Hq@0HWZ3B-Q+y6Tp@1@9A-2Lby8$=U9T=uD+q z_Gg5@N@T)+;?7do)dmk*$mKTsS9vRZt)@zp_&>kTZO9rblQl{#bzjnAWc*>!Y;^Hd zS)_?W#~NyFBep&N%K^VL7`s~=#`piwjQPaJ+oErc0NyCEhWc}PTdVemTZlTWJAM`l| zbBY_njHA^r@GVaA9PioL7+G}7q$6B5Pn!#X)>L5Be$DB&Z8%$J!!O#`)dA%}Lzrgs zGK8D;Y_DsvK5*V!SzC0!?(HPRGaT+0a}c&ZC!PVDFY+FX1^LTsP{Dlsz+=CPUyykPu7nwiw$*%Dja*)Ln3C;oEhwTx*hf2VWtB2e^xht0u!S+E18W{-be;N#VAToXbI9B-7WhmwWa5m0}arFlARMo~%S8nS|- zCW&W7$!es5VZxzAe4*YkcO+Y~=Xw})#BNe8xDE&XS{_fHhLN}%d7Pi#%AGfZrdjJ? za&PV<3c{MG9=(RhWyIlK+j~SvC#);{M#InFE!k;*-C*hdPB8<@P6vh8Y|toH_n)n? zR4|i`&vL)P#PZqcX<-iKMY zy{r5uijly5#K@udb9EiDw$QeU*X_etM$c?J$4mPn`^y8N;`in#?UE+lkIZ)EQb7gasuOQAt1vW#DA%q12d+8 zKI!36O>Wk>^o-2AK zT!8cvOm@sR-GXk3PGM-=Lf%fo1S-ua3<& zR{21l{^)C*+0Xs)!=^r?xk_}+k*|Y2s4vBD^)uO9t12AmKK}+p$Jx$8JZm~ovSmG* z!OQ%5_7n{s3AEkEwGEP(fD0SId`_lBEhLuveU zF^4)nc0o&I@;#^9BWvC4yv@@z>JZ&0kK$`Y8=5~?Q^0VZp7pXW;N6V&UUzcISa@jzb?z-dSWl?p%fbXT#2z#%V!Wv5+9 zIgehoc?pSa`8j+llAT*qdQ=wB$W(Q*>DLQ4%WL5ga{tD!D;Xc!WzF=e$_KvgqrH8U zppg4HCxPo`9wEFtq)gl^a`L@2Dl*oSsz2Av5F4wb)lQ4O9oz~P3tgjP4DXg~Yy4jv zwQ9RWaoDq*Op}=)Fk0HU{E%3{!b?}c92?Cj3 zskq*|f4La$dpcP)-S%N>UQ}r>diwbpr6x*GxD*e>kS`}Cf&$tyB%&9eX2;WU6htfk z8Hb_{Z8LD8{^Ct(+CrgajAWJF$TK4TcQ~z4IB2}PINDSe%gV@NyiVjIpTRg?;8Hz- zl5tdrz_2_pu!*v|J^zPEM^L6BgG30)5Wcu^9}zx?3doGSa}|uNpFFh;32< zNnTmkZJ6GE>3H(o6dk$%1={96zdCvN>^{)#n~MCEovn?x>+{{UqYrXTj;5w46=pN4 z#UTpj4$EgDasm+%cwB51W0A@00J!Dt1IS+Mdt6#Y%|oLtnaFWnL92CEOAiP(3lD31 zSoOPbE8|+~%yRAT2P$FX`!&BWl?(%1BP`aL+!m^2OaK<>hk8rG@5uG5W{a|->w*5_ zV&TDewf4E@3LFAwY;GXteq{pME+Kxr5s+z@ACz>Yto)T{a2R6b>J@B0yI$;1>0i74 za6_Y%h%hzH1ANRIu&s}F_kGQZ)qK+OM!@KNiw5sB zW478#cJ33C)e-q1y?o%U!|p9PUUdlAsFvj9HXf1sn_IX|AXimpOv2BF(6tni@yiBy*@|I+PZ!4VGJBB`lkJf078szDbNodg1Gq zb_F$J|HX*+D)Qc~`+QN^;>dA-o|k#@`JFEEEdTq3k5&WAYirrbwziZWF9iLLpLsA8 ztCvPiM34zHxWNC7K|cGd0!$I6d=DlN9|ZV(yr=IMGLixh%Z91AM%ui~?6U*DNQ7A; zXU?>@ErVGhI|7V-d_6qqHQ>>n_aS!!-`Ov-9=7QeV;vI6bCZ8lMih`T9k>&UglB~$ zz8S|HKkZNA8bW!>O~zipqR>pkBhB(=gn9hM5(a7BkBKhB+_1jF<@pUGB*h$Khk?r5&u==~i^?mN%PFw2UNlBT_@)Pg25v?aLdP?JEb$*@95zJqnFeuAU-VX~j) z^X)rVVg8fpRqh|TIWjS_*l!ZFT2;{@VHp`#!x?Ec776^ALvAG=rIWWPr%g<|@4aJ{ zOEkLDs)mg76_1c6smRTK2mJlG;bF4ULQ4KCYyqm7zPI~<^7ILWLGCZh$BFk1e*5-qsj;v9aFaaa_H5kZW%6j>@QGWK_MYN8c) zbAydkomiW&plf&q0%FL~2j8eR#j`@JH-=jwfyPOkh1Kn=4wB>@SYCEssR+2K^sFAp zc0PBaw~Gf0^>j#fuMyjQ`tZn#lLcu~jEpoe70!NtE6!nE+0oW8dz&4A5f&9>l^_u; z`cs;xd=-%cdHToHcdgXPk~tA!RJze6j~C*@bxSy6p(RK}5N94Z>)ypwzYJoOGOqz4 zytucoE#&(h{X4`aN!vajO-{vEm<>>-(u^R>2fQChWGxoOAx)SODJa&;<`c#CcvF=x zZ8>85u2QqyhC2!)|aPUb;`lde6>K_p}thGy>#&U71O;DN?WxD6~j9E_L2tQ z>*9tmv4Y=UDFiLI4gk{k8%el7DB~(%3& zN`#ld(^GPrmuKX{s>6j)pyzOwUN*U&?Mj+-MZ`Oh$iIR;aUAV98qc-n&R+KK#3lZM zt5&N1tt>-Qn7ld^{;kT{r1>SSNTzX#?fA*TXCF5#hy;(hud2?m*loKawi{X13AbIK zuQLZsgLK7A)MF`FSw7hOX;p*85KxStprrEEVN)8f)aFAkFf+Nnt2nRx?TI=~U%G7c zF;s*itpeb91BG8RhzALq|Eo62dS9|e5@m7fFn~W2c<%bz-aV-5?Gd#BnoQSL_u-zO zv)I_)^8Wc4-tckLt5opDTSH$z!>(}@SGKNh%CG-_P+lt!Q9|!O8EL#)thknxo?(!{ zK^LTLVWy-eP3c0&_78kZ!6~~r*WrZmCLfD2#q zY4h_4Oj{omQY-tCof&>}N}@5)dgPvu-`+s-F4L+)a3+KzfPV^hvcMg)1TX&sl$6w4 z&19apDmwF-4iDEgXZ+>0RE<~99gQOV%#k3fdneH5gP-9qSNVc|5+KL@;zbk&a#etv=xU*cx zu1~941qz=|=d0DZeoc2<&+Jj)XNroZX3)z2+>H5AgUQqqLnBwD@MM^m_B!JkIxAeY zbp{D~7ZI@k&EchkS-xn?7#FvrHUqJ+ut#|@9|Uo{Ja8Fm4Q7JnMdmL##WlU0Fj)H9 zC<))Oe1R;o zY|OltX#J)0d&S*79ou_Jy%W6`6KRVaPnx9?==~u`TPJ*!N7aJXsNZp8gLMp>bQ+h! zJok9D7n^H%Sd6K#=vOjcms9#-Vgs$$j|`H8uON5HgT3PY-i8gCPC5at72T}P*(Ea# zZdv}~*>t&KHzf4Sr7i1Bq{!==)C8)vup#SB>l52xKzVG5kOM2`pgo$v6QJ4#Z9`j# zX;|kjU?j~!A0KY{QFWF$Jq?Wh-mH!UxI<#KqMA??r<;YF5Fei`0xdJd@^0D{nP+@Z zM1a?)SKT;xfg`f@k`Ue-kZ`vg|0xdtQz{OA{JHr5k3Sd5Sah)ams61U`WdGOH^Rd; z0sNMP*VU{`^K(){DlBDS4nzzgo-%;|@f(=&$O%6E9gkqoku$;nsQ>w(kDC!io#*hM zc0i2D+9c3cjqJ$9@78I(1URk=L@czlonH4<;N71wy0+vnFmxM!O-_Av-5!XELnRqT z-l*4b(z12sAe+Xd^|j`&feH8)#2qNLK5qw44e}Go=nwgC|6hx!xMOo@5@o68(VIFK zZy8PkbWWFJoZpH3!nrc>!5++7U|l)i_qp{BnKacZO__xupei{2&A1bEz4*t-tkHbP z?NgsUSdNqQNCo+priBJJ@TT&fM6QE1ky5|E120tJsqMxe08MvgEi>mh?oEp4g_Ck9 zPJ-Z@pmKD@2x4BwF*0F~_t78rR#$W7>P*;KJOProGh80}-W5pIn=_z`HU-%&lv~l` zmK<+>9e>*fQPf>-joAGtJ04wkyrQMhd^QAl0Bi#YrVij)XMcs2k0obM6E|#|%}bk3 zuLqLhBDHj24N)wf6f=>p7Sc|D&!^8mQMhuc>8!dSV(S!GG&W9z)soOvGnR)qo#vgd z51JpCeiuPJYFpD^h+16BgnFltren`YlZQJ&PmX? zD@>-gv0=M!iiP&<$_EBLfO?fu5XhgW$N~!%`{GN|AHIR;@ot%Z-XHYpdr&@Yy(F!A z|A$hmMsDiEBSECyV3*c#etsV3C{TBiBaU)T^!er<7xUxc^w!Y%k>(kZBlQwD5O{fL zPp~hZrgo;3#juTel>2ss)vD)T->%Gar z;Ytm%n2(k32H3_6b(YE;rap)R(i{VbtgM^OU+%Q?O7try69TX)-V;Pq%b8(;Y?!qu>4624+!r%zI)79^s`0N@Lei@~Sj;+mTh8ic+aavcGW~uO-PjVP)}B z)h1Q7PcOAC9@Y<*`u;|g2iwd!0f6YMrr-0itpNl2G4uWuvjkr28UbL9ieV$W4LDam z0K94VpJn&M6VUmaCJnDYy3iJsi{yE=YE)u21AzEPG?>P3YcWw*!W8PnYQUt^G&&t| z4%uL%({1(;j|1C}vtP|QNAM!MD<$yEB%A{gCy}-)y^C`%I5IL)G=Z#_dR{IvEzcpT zhUuRti9YE>jg4*UDvhk15YFb$*&XC^`uJv7W+gP6MmLYOnI~!DY13RWXB`oV>!OF5 z*Wka(r%nW~aCagbgrdL+nfJh|il4R#gqCgJUeE}eaW%?i; zj~!vtS&rombT!=FJniAM*sh|++c{?UY-;E-ohHO%()1cmXMWa(+=%^7y+pfum^oys zg-*f63H{B#vjCD_b4OwmKHk^{k|r3^mn);PyL#pVoD871ek7t68`b0AEln6T&bcj( zp&Y4SemE9rS2NC1KH4ma*xU-{ubUljYGl&t5`;|RcC-ISP~D8Q95a9BEMO$oOw#h{ z&zQ>n?bU17YobA*`u;vHQn2p+X}sT>@KUetJ(G_fAOzWm7hEsGEN8J}BLS@XDj3lF ztlljEFh=F~0q9fBVM*%;QU50!xrym@iB5a`!}Bg^9Ut6C8P)2jS7x*-o%F7+&t9y! zJSFX*Gan>A)+%wnE}y0m_;Q*sjd9DAZ1rf=)2r3Dy0o4s5_RamVkCo%$+n_Yc2kvLc_0b|2n{JfE_%{v%6wQ@q zbE&~Gbh#K&q6N&yELSh>AL$GR-4-Qip1`uGTP*g&_rx*gS%20F5Y}$_WIDS(P#})! z=!MNv(^wYMGog?sQD&X$%=7)!g7?2dAPy2ber%ud^1`qd|4ukmtEHR}dfWMWja?H@ zhuWR*&a8_)95mMkqkL3&wh6(ZQzSOp(tPvETLKY}0+7{N-^J*s>78IzHG@Uznr)bH ze)JOB*GjMTL|pG-*M1`Gz%};}0$73;sGC&Ar^4uuY)gQ&`DNE)+?o_QQUrDv{bbgw z@!@m)#I0Y9&D0|mDk14FtCsVQRlmWRF%2oYiA^i&z{>kI7}uM_7!B16jTkQ0UZC#ILlg3RghZaz2c_pGO*;j7P_jwvCn zUzN=1tBNrlifXQ#)Y{HZtHiR{k|yTep=_BCmz#9={&h_bGWCljHR>++XsjDGhxh5E z9%866-mog7hz-q^jTQuPO$Y!hRKa%gKul^zcYDU3)!d4e(T9jf$OZOjvTMj=X{AZ8SLqduluagcl! zo|gbB;}&FuY1lSZ3L>l(MIgV{+KV>B5~m3={-fl%+I{vjK%~#^D3Z#yJfcT=* zh8j^;Pw|LvZEza7-4x$397Y7VZY{^5EY%DM~*SC_S(_ zpJ{lN&B6hTq!EEPYqJB~uF+F3EUTbP_{uEd?ub_&G-Ycx^9A1HwAYsc`AZ!LLRu9$ zU2cxis;@P?kk?zh#dBz>19O5{J_7<_G1@&>XjIZsAjB_uf^ccO_SP=X(osmjpIZC*F)-FYDiN0A|v3 zfC<&?+n>xL{AnsYN~?U4M1B#iR#1F$Lza_KP8M2yuhGo3pRDbrVu#wgjejsGQe`kB z-M##VA$%;_)4uNuWv9gbXwn@P`{&C;Y}->6L=c@@pwY2ZRom3WF8q#3I28V*?a`V4 zPReFp`z>GwH)kIGSgw$iFI&q;RJOkwgHVdEEl46KYJL5YQt$%NNnUahAo)>M#4Ue^ zRHg$q?G1o2sR3g3)uk=w z%+c{m%K0rV=p2Tn;PSjCYtB~QLtWi3i4En19d}ee*1{v)fC=-TS3x8#P3~W0uu0Y6 z8uTjPIX+W~fHq)oK%i%Re!b0&2%9j) zo@upY4)e72ZZMu0xeSh>erV0HxC|OOo|#i;`psg{M0thNirCK(31yD>+5*Ovl(0_e z0PaQ4WpZF5Z~8(_85k8LCJBOtV;xWs$i24y{<^+`RXH3;>j~EN6-~nzuFZ8V&agJC4g0!}&9^Uh6Gjf7`LBgzIm7@{WmG(4y# zJ_uuAT&bRH_VMXn7Z`XHJhZfSoyuT$_XSBv{{$A&l%I>imzxQiWp}6!v>ifS)p2f} zeQJj6_Ijt5idJq>R9cIHE?R&mykCb2kDb12Qloya@JR{jX^{K=aR0oe}0*(NK;VP3ol z6u?v@DFpg4pjXq~!S{E2TOS5o#L%5e91`~jpW=Y&aR1owQ~jh%nb){&C$qw0T466j zuy%|jzjGl5@M5QpF^WZd?vY2P@6}+f-j;>5*i+D(%g4t7dWu-!V~YBZ1!-sIdck!{ zYS$_4i2@)Zoa11AA{6k-giX^JlPRSMNHpa}iH9Q-sed3(p<3Q`BN<%T0(LeZB>2t0 zhTh+vn?c%e5kx78dH5;q=m!($h92PZ5LjivTl>GcN&js1-$G*TmdAxq^2D8P870F& zlih{e_SDA-ZR9BKCo(v4|HpRs=ux|eT{cVU@ib~o!zv816q|LMnEww&=1nd^K97bX z8iL2&^?&{+P(Vx`y?xyVFPDh^>$fgLAA^&8@PLcr|Gv65w#N{pHI{hW$0F)KAG3o1 zhK*P>w2%Ei^0^*g>k7cV{Mk3+i~QCMke9UmfS|zF=5gk>wn6|djot0WaS0oOXqZTb zA7TT^FP(g+SPDI&`hbmpVrrz+w$eiX8yaM)u_UUu4cu+(C%3#bN^wjD$(!c{-pUf> zoL77Wmpg!FWB0RP@5!v>y*^r#3LzEzQs5Ce&kMjx@CUOr?`&O;r`LzzvGNNqFlww|D-)UE%lkk0-P8gyK z6S1JgN^&EKy|@GY6`$vdaYmBc!@Z#6_OyS6X&=pI$Z7Ngh)F1zqrLpH)-@2F&Q^*; zf5{O}!o6s!-`c?D)RnYJco>_uV7-^*d3HIxkP*a( z0A9z9M7gPr)u7o$e8Z#kHCPb}8vb{q$y2r#fXF_*p0G|59(p4p2{JpahxwO2<8Zu_ z?=>)J@?cxEKQfkJ5cPp{S*Osag}^uhWcXeR0SQ5FqS&{4e*CH4x|DpWv1J*&NW}G`>^@Nx8HGX@Zd+rJDYVq{5H) zwO5uGy<_>D4(G&0Z|m_M$!En>KJhg=`Dh2p?x%ZZGXU2f0qM4umxmPWcc7^*lpLu0 zw0UYDLdHT0lqXCs%gJca;)sC5GYf#Z38$_2AijfwJZBZ|?b;*>@ega;RN`O9gmuoJ zxN9C1cJELX8Z$xB&u-H{EsP-(oy-3I%C$K2%K#s8tSyigmNy)wHrLYQr%1$O-Z7gl zvPj4g_^FQ`0vXO!rq1PWT&7WWUf1mv8eSojcpAC7l2iy#9m zs>ant$TB+rcEHyhXXPT&MGV@@*z1f#i-25?NM4H1tWVh|_~mvw!PbMaDjfOHFn>^Q zpc1`DTo@7gDch#eAmgcn3a^I=C2(zK`gt`96p^GgI=(-u#wyx4aRWh40ZG+*7*vws zQ472C)pN?fWHEX*mJhPKBc9izD6R5_)aX=yu>SewEa^K+8bT=r=8%U`n>kjVKmZ|! zIic(~2D-Vnlgs%lw62;eIzVH^o&h1oQr-i&nB=cl%AUeZn)FywB zG?w=*ZIN0fGH|pe#+7O^oRr&W*0@PL!}jWz`^HG4kie4|0E)~LcmJBhiznZD4yf zt5dzttH0P)AoClrkD?UrtC;m1e&+)k^w-u?CHWwi-LjH-5SvDpdY2+wE9WMj_46Xa z@%CcO=0GY<_oji?pb10_8{VO5@));kB3py3w|!cr0Pu1OSJp(qf&A{5ha<+*Qy=Ct zaE#;81^$CFGZkoi$zwT6-DRyqS3LDS_j4Olg<3HloF zinSdP@Y3~h(%%?y0r44FhG-lL!jI{;ytA5dq9jHXO6#7#|D2)k&9=J=TAt8 z6nXj5eqBocP*{>2oqW0KokYtZNR2w>XvrzFIwL|gU#N4GH*9n}`p$VEy1KvMx;im^Oi!&Y2Juk+}z6Bw9l%@$-?5a}%Ra8AfFaa0U-Ai+T+4uD9Z|DDBVtor`i6X=JmTdFFS9#{a$& zr}cQaI897P81C$h)rg^$5nEJBvk28r5#RnZU!9y{zBA##wTVf>F8zbV5Fre6d-cjQ z>ZzXVg2mxiJ9E)G&GwB5MUMgB_iLa-%ALbN| z(V~DCmN_QAR5Q0?<26#fs6?5#x(aGEwG2~CbC5LT%lcvyGq5kksejjH#%IUkIfeSb z`C*1V*A9_Mduy5szG!a?qiIfycw+Gkm?S4lb(m9g%gTI)5O#^dQt9L}VyiGT4un=O zaDl={7rL#J7XML0vZ5x8iLPPcJmF?vC&dVhr z^WAY?Nq~PAOxBZ36$*{1UYf>faPHR-{j;~E%S3nGxxgFe0ltxgr>Cc&(F!Eix-r;SfBRC zy>~Gb3TTO{k%Dch^lzPye^0+G*XD$q>=l;0^Nb7sBCmHqum>Oms-nZNGp zwF0M=Xx0=mEiKI_LxjKEi~4x{p8E~@_JL!DjPMkXnQ^%yU)jWwGu;>V?D%y-y2azk zVr^h_jZ#7TH93+c5&Gg*Gl2uC9xWTv9+h+U=zK;p`#S8>hu{k}sba{Rlb%XE-$> z*LP=`YpH}Me_#+hWr3B;V(5@4JTms)K3uUF!CtzX)rW&qS+6$im zJ>x&LVD{&2Mz;nlT@rwhEb&zT1tI;p3A|<{RF9=DU;5EBYx$ZF?;2~|NWoC?kOgA& zLD`}I>IC0ZJ53;t*0+a;-t7th=cA0r5K3QHel7L8`cGFC0u%d+ZBj5lflF$35L|}= zpTEcpMNiPo@eO$ToPy@kKNL?6WqGwT6m?JJ4dcTCrqSV7qv)ffCLdtOMpM2uoyG~3 zmK;~{<4S%t@z#n!VjW@S`)%hc-ZBnGH%*77q$dH{?n@v$%&>^{h!C6Zt(%Vo(ZcJ* zKMD{yus@=F?~N(jjgbeZpvgNcBU*2Ll?c7dyx};{Xyk5XV+0>4VZmWoZW)cPbj7}b zB2U}uF%EA6Q7Lvk%R>YFgU}rrozYgLNiu;fFH2gYR~#V21;ucV_YMl7w8?`R6Vf78 z`a4jIK~0D=Br$f9$Jfw614mw@`zu>xa+xiY2W!M92QK0ms%dvT&aeXpN`9IHsM-yg=zhX0K~dSchY?L!uOUN5 z5Ft3r^kPxUE&=wk?ZzRyI|#1WvQQIJHSamb5(r%zcx~LlyRtki#nzpgd3_ z3^qDo_15hg_chRi&<9S3YgMsZqlRTD#I-wk?M^=hIlMI|cnvkwS)P%cNfPT~nsoF$ zl3r@Mtz#}JQ}EJZ*Mc{(KaoFjw%U=ya(}VWb+lZC`Ci4g4vW4l_emX)VRSWt6Gwka zovk8qfD2D_N8$UlnBGeWWY zHH8yHhIlfORza|7nTQ+wehNl8!03c`vUXeCrG*hb9upjpeitcM*V(OH z+0P|98UhDDLZ!tW)=~c0F$QSurq+MXW$|Gj^L3;=57!H^r3#vZ zoFLdjdzH4+!=i7X%6FEL9Om-bcqQ|uRn$3aybVqRve~xL=f9qalKCw_VNI%O#h#CF z-feg_@-2GNQ~`5k*<_0%)h3d|XnR`&x&a45X1EBop9aP@`Mlt@a;@Utu7_O3)WQwt zq642o>fuADwa86PE$MsIN4|w@_xDRp9sDayGNEXrE|1$IDI4{&RK$;`sGcYZR2KMr z!#;*lrDxv`ZOQGtI@%YgE2ogsXTIFB_f-4P!P?va@;@!I3Srdmdaf++np7nDq* zp%)uW-w)e-h%KF?5_l=tGp-z-CEaed4bhjdzNn#t@c3aSJTP#WOav}EsBK(*?$}?a zI~rBTZRu#P055@u3ZdJ-jW?UM8!guE-<-lq_3ruC(67gO)@s?$Izy)4rt|@AM1-Dv zS>8<$U;rWMa*J3|sYk|7xjB&GjUT?`>@^pCG+ge4+y;J@S@2p4z+!j(1uIHvhOnvo<> zIR5f)ku^%Cs_W+fx8JnmjeV9B%|A?;7TW)t3k^(#6p_3aJpheB=$~p0XRRH2 z#zRQ}#}0)RtOeN}p8eIhJ|7_0_a`mLoW1P=UvMd~kC1w*uA0H$IlR6SOMeaMFrUl` zpGHyvg|{~C^S`i>sf929%#`Lo-8=!|SoI&g(AW@QaNb{+|CXVfSbT>Ub0u9rlhC~H zL_U~CG*dS{h!8Dg3J&{hWJ|(o^=QYVRO17>iQt`nIXh^>r=}H^)90H#(;wrNf#6v) zbNZP4ED`#x>hfo`rETR){mt>8mS~I`P0rQ_R5bD^4|9=GvUfpRDCha3YFx^_Q> zmA=dIdf^8WV828N5wSaR1h%K|_YuWvrMSZ@$Z~=LYq*h?kwEUImYRg~CINP`t?<9e z?|DDI9nyFIv)FiZ|H}1W^a%08j|Eqz-^TiHF1YuHP=Rv?GTwb>;_mI(1yrQ!8Bpsf z9~P}4wfMd;&lX+fFzw?aViGzZ22{tN+K;#ZV89*)-Uxb`=#FT6kDz;B!3p0+`SF4g zKs=5;G<=m59{caO6hjPeErFmi-fTejJDQinW~uyt z{c$cIbxOJk!@M{ibak1+o)Fr-K(HoneGAA6#8V;!A4==*xcwRxGDtB>}G` zkYyfiV~-3;FJMS`feR5tFozjM4+3eUl2WZV z6gVTgb6NqCvxYuh$`sj$%w$PqLwtvHhOTTkM^w+zQ(Z)q`)O?5YAj%ibHS z>4L)VCjG-HhG7KVAh0H^aLG?kEi*mzw<{OzV-B=Dl=Cs`CT_3LB(!wo52$z1Nf*Az zroB|lOEChuLD3pRsm2KP?dC$AbqJ|4)$HIb_^lupO-@8j0*Ansx=jX!(^%jxMKmG+!E<@*k# zNC6v!1wO2RCBiwxqleps(yqoBV{{V?Bi`=%`t$qb{k?$$HQ=k&y*>Nj?CPqaD43FT zEY;fRaymgbtE=4QjFPJ9%t~^1_q&QO&E^gC1NqVbk6A*k&gxQ9 zkW-nOHQyb#g}=?^>^3@P&|AiCQ2W32^cA z33@)dCn-_?Sp`GvJ^!2KxDBzPh1Szi+WB9XE1XU4yB3bo_a#4Cyh%mrE*lzXp_piF zvlREY)B}BW$81BlvnXPW5wcc z=(W>vdtM@v^Zj`7=DpmLyS&cgf?|=`EBTO@Gu8K27q0muf_Ec^9%SRXIev&3rS%VW zE~n&|V~a|So))u}BLhvyl z5dHaLs2-?Y3`DI85uM1}KwhetZ_n{73j#ZT+?;l>uT(HO1Plsuzf3a$DIny=Vfk%5 zufglCntMb%R@#m*qQQyj?PCDQ)M*-shS}l{OenB$J7=c3oMu;RgAhL-I;gC`QJ@~A zR2fJdD>66c%sxe%`@V4Xm{zLB5tp429x8C<+8- zDlG;N&%dZu03=@+;ZkONho!qKbTKEyi|_<^!<4@gsI^{ZPp;?7Ukx)GFBVOXp;vg} zAj7Lb>b2GSXfSF7ENTe&Vj@pky7NT1o*a;0-&gx7!lcYklq~+qfVNq2V=AbQVH0`t zxKI`$5k)N?I)er%9^{oU_=qRgH69G6ZFLNi7kg^g({=!luc+Ab?0|(9=4eTTjzx_V zrh$`1nq#40>H;<+RE8?ZYh|mQ3O6s~u9wpxFnu4Nkn>922Yl9MwQ^^Y+hsjy9L~?g zAp{?O60rgkAzed*ThVndm|UV*+Izb_hI?*@pN{F^*Om*3cer+rqW0Y{ZAO>fx8Glg zKs`Qc#rFZ*e_UgTJ2OPliZUk6oon+H)samte7D_jRq5#;fEA5sDdwq;iY$b87f zY?+Jy_Nr%8W>v(_SflrKzsdeDWV5tzRFPqoB2|gz_px0)g8S+$2$$%bH}74vHyG$N zSTn$aK6WJ_Fa+EcvZ?+W^V#@~zT}UXU11q=qyeL##WY!Lx)=HNgd3f1N|)71gAj;m zf{U7d(K8o+qrMBAIw(xa-BOK&J1j{)XIdvo}qxeUjoK#)2OC`a$Y{=TKydP znNr1?h<)B@CFYx<#XfLTIw5|x+Kl=_H4-%H%qbV0JQN#Z%IIsvzcCRaIpAXzQYi=o z(Y?}B!@hr-+NBRPmCwoMV=a8gLMsL|{C)J&XD1nQsqZTDa(E>-r1ly#KF!f-Ub%W4 z;qyR(FZSlU=(nwx+lS4oPQuj6jk@>DUy&IiwXj`B_v2|6t1~;C2+40C_`$<`K^QNk z+*d;VIh0o9P%PX`2gJxVLksgWvVT(HVMavamT4a&%E>Zo$2N`f1yf ziB?QL@0XP$c1l+Ad*6wigXH6SlW`1a_!e4{k2oImehTeoP|vyD`}up#}HG zX4(S16BdXe2~36(HZZM9`1@g-_~c}v*sMZaAQ&0e6&m}vo##6TdyiuDuASQXZ&w$W zJQpOHkQ?{Y{u~ZgNEJUN>f#&fmcvG$uixa~j5}KbfHQ1O>mipJ2m~g@NS!QG7LLBh za8t=7&+Ub73Z8`_>yW0-c)v}12S{d+Nwe<*O~E+LS=jjoC_Zw(1@gIFFK;M`_@Zef zNku|8oX)*r2)TdOd9;ggW^z*Usq$JkN=DJ@MP=_pZ4l`<@#jbrOpUZo@Rf$i!T$ky z14TcT$;I99hwTXqq5rh1h_-A>AnH~*$kUW%ZlclH zO1=Cuj#h(x5XP*U8I}8~vy%=L4Pw7Ehz<352PJx0Wl1X(ggb88m`W^|o`8EK_Cj!H%g}z^icM}kH|hYy ziYV|$`gSX1pV)2O3Vf{{_Cgm#z!#u#J7Q z!k6!RJEtigJ9<}VNct&}{VM}BBdEyK>)e8VExIQ^m&(qoXoc^{9c5G`7^5$lC5h`r8l|&q^ST>86``0yzxi6!3q9$v z3Re|7))Wh(Y;lI^Cf7*_3$sO8>MT)8(X*e{r6juZ9s7g&29^9_XeJ#kC8sYF4MVh> zq;NjiqPe5b`gh3^lBtz{Udw~L8dBg(_0&;MBP3(s=t-bJ{fe|dWh9fs=DF~tTIToN za~5xLuKGDMEK>-B* zvUXf?1-h`&1SFuGnp&7N9nD<+NyK40W{JGg5whD$WQUw821c5Rn%A?R6gl#R5vFML z+F(O{YS%-42e?_|bl5L$CPmq~dyLN73UsK`V(W~)stSki$xdo7~(k`QsR z{2WOT21TYf%#GT8g6Lu5IATT!!v}6RYd$?cz3}*mNU|WA41!bW$`&ld^!v(s&ze6@ z&E?8R7Z-7YUiK)?_?csQ@?{P!HH$_|{;bie0c~EFA2^ZHDEH>-67p3FR*pQHvNm3) zhj=@7@5PPQ!i+YrpP4r_T%TCuI3zqj9)~qA{{G>ES@Q`$N9aBzCI>-G$eB0DjoFnH?iM2ef)lhRz8=@vgH zk+ zEgMfB5KF>Dc}g66l%DZ2n_g|4>~nDEpb{x!DOn&E)!UF1_8`|?J>@;3VN#sJXPMF+ z>mq{9V9bAaAZ$!Xi+1DI&^7!*=VO0X3mdeq@Uzo*%ok~t#0w+uzVE$;$w;i)8QYXb zw%0vS${tjgr};_-+)o`-Fgb&DYE*|h<_Jwyn*qvLpA0$MX&=l2RBa4$WkDA^*6rz% zYl*9+PsBr!FuGnor3zFCkh7L%6Wx-4xW_AUujmuL=nNFI^z`Xuq77Jv{?>kkIBrwu zkFx=~O0b3Yu&%swwX#gKL~j1q{ifEs=LU{~^lOLa%130Y5MVP8>RU_$j- zv@qnnKEr+~yZFvaSIHNXLS}?x++Zto?w3P?>zgvT50NlZe}3mv13x;}beVYeWEEN_ z{Li8Yupgj{56tI$vDbaYiIpk6v1%AAwo8gyaW3KmX)OsdzypW!4B(UN*{lv|&BpTys?0`1GtzmE zc75u;;BNI){?XP}8^agk79GkKi)N-$Dpm}CgY!`oRw|w(R%Wc{EF=SHv`cMeg^G&d zTG`?lAc4beK09^#GO75jL_X3;6{+5#;uF<_hyJy37SKRR5heOJVLQ%_9~DV>I8 zaIxiU9@OQ<2Kh`nc0NOHt)jmds76!B#4$7+PU@(`NIhfGx3AF=d@tZQcT3kelErld zpl>;Mb_30LFNan;2)$%(zTvxxe7uR6kUAU9_$01(JxMlN`N69LrY@!n+;h z4gFKs!<7%wdmCE6GT^@7bMdODwn)HoU@?lXO~3{9Wtr0(N2qp!uF%4>;VF$r2eompVEjber-xk^lFsfh$xBl(FOuaLu1BI@KTu`- zMx2I)&ctV?`jL3_nDrb9N(6IH>qzGp?avf61XEa~H+?c!s7M&R0?r=S@DsflyRPt# z25%{?e=;FJslfV-xbNo|u7H5?7EeiHek0*Wi{W=p|B1o;Pu4rMuNNzgEpNd)2HT&^ zf5nm#Kjo4myoMFxKzN-ajzsnU#gsoKY4Y+x$)WJ_8FYXAD_Ij<67~gjIzH_8QL?XC z(EoXde=eGm;zbFcl2W$z&vpMNdH(4_fEXu-;?rUJQ{cbT;k`ll3I8I;RO6s;bcwd8 z$Gxh&teW8c>1q_dAO8_4@ukBNm$AG)$@L_^mq-A0hss&p3V4>}m41Tw0bv*kjsOltwvom2X_8jST*{u$UaCjUWwM+y} zyzDJX5|x&JUnH5DM80wc>wK+EyQS*BGw_5D^*y<-vZ#05DAE~mJ>PltDV{sZcCAmc zTRt;w#ISetXI5{~NCXVYkq#Py#O3)G<0_?VC^xVviW&^nzg+~B+zLqlp|Z=ew$|CK zKDAmt{&=C%WY3`6GvVxy8QJ&)d+$NrPlnmn zx@qa1WjapWJoK0|B!Ekl4q7z17)u_6K?G6#GF61Z*Ol}wPdKIW$NTu?`a#kHsWpCt zt8jiV5tO^wnAuhu8SE*_erC&MLKTL^j=4+t=^^P^Z9Q z*dO;nhh>Cjh$O5L3%K{_49`{_3TZsdNtljgW=kd~pRHWKC}8jqw*sER_ID|sxdvSq zSTsgb>RGsCAXqG?l6s#Tz>!%r@B+kSXh(zY_Xc(4XZ>u8vgtyId^A2>P6zV6$`;#o zJ4fLD#)VIUa8(d2V}wPcECsnbLg%@@(qw{;btIQcl+vhq5SB|9Vyx>aMnjl})$a%u z?oVK=%3n+7aeb1BV6m7f(p;um4;5ROk7t*9IMr(ucgZBnc8{!6x?X%?24@E3Mrwkc)Y@d(+!_STv z{nGYFLwYk+uBdc(!c?$XlI1!B_|sG>$%hE-Kl&3`%854INf3$59a#ah8Rp|GSLr74?yE7^eT;1kxxa_!Ho5Z?S1cE@;asvAvW@|T z_Xwg;*z5kuQYW?Am{~F;H zb>wvS)TInWLv<|7aLe9Wb7F?drqc;}0J+>v;+%NZp3!3$37quo= zs~#ex_qo3|L47K%*XF;>zUA%Pm;50N4h2^NQ2A6OgalDkB+k!B2t=98FEmlu9M33q zKBi(VEyktFjiwvo?aUXzuXiW)s>vQ>LX#x(*Bmr%f!~zb;6@P%ZzAWhEDATuw@y1` zXZ%T!2K(L1V)oJKb?xDBUV!T_?M6<&Be!01AbxulzsoIjohugCeqJJCN45b8Ip=C~;9tuwLSE#uD*5Kl=!Wiqkne zU14-Iac6hAz{6X~qVPnloNu!0!XGQT_sWD%=m#*ALZ19PD%a{l%XJEGKYlSXFZ9YR zvq;o$oNJZws|GZ{V=j$wzP|i})LlG_l`w_kq5g8);%IFOx$Fe3+SXt{-gL9Xy2P1a z&`71pu>Q_?MZJZ*UpVKSD?q4nw6DtCuj_5Scx)yudFon+nzS_HdFL-HI+bw2u`Gb( z4Twu4oqViiL7#x`rB;3Gi{R$umK>?7Ucf4N$Xz0f@50vEr2ukfdfsE$5QZ+;*FFmd zPa4llGj3Y9_(Wb3=gDV(wj+#%S^xnFhBx@#-C-5R-#BplP|pLs9VfFDoDS#5r7f*s zxOi$lM0*LeJonbk!B34rL4}=D+|u&WY;uch82d&pGd^S(jLkTOya9!+ht1{~ISrmHJD$ zgt5M~y`itG*q*qrm-``byb6vjbQ(49V!5KI6u#+duc@>O8BQr5 z&Nq?{DRZ5go|7XJ+aqsyCl6J65v5MPx0RXjJ$+LHK|o2D{|vFBOPo5`~49~t&w5r|#`sOO00 zX%YOsXGf%zTk&JI=I7Hn?+`?~XjYy7ycZkZbXNtlLCViCEqD3;K_-rzA9&>TX*Azm zQScO6`UFL&t}oT!UElpo@_Pj>(BjVzf=Z{?|Ac4wqei+juP9`SS=MeR4Vi`5+EeS= zR;V%bSYQ1?FA$G5@@zaT;>1KF@=@v|X0Y*Es{P!(l8(=Se2usbD-w*~PVU83ZM9w} zuyL~!)rs_qmLKq%KDbWLcOeksF2(URW$ZuaEbfCM5g+4jv0iRFZmQq5)AfA#+#CIs z;s@Ldsn|lf3>4Hl{gGG7>bSu9ro_^RWW4!d#wb&({4=6~ zj^k4;8TRbJjvJ=5_r)+l>t!G4CIVgK>;ILSVRw2#7+z>j$@={^MDMWYF|x2&!*`INdd_s-8w_6IcZ znQz#e%{06lw;GBK)AJ~WjN4{x!ffzfG-(zFUzkB#zLOzk!V z?-K8pZciB-mg5lAyM!#I-q+0bfB55pJj$tBuUiJ8&j*Ni4JhG<$1c%WEw6l2H3h*8 zxbOS-qt-4W`}9`oXV8YvNDp|kJM@yWb#kQXsuXQhUe#3H_z&|I(BsyhN?phDSe1U+ z?L2Q*FgOp%1iFYaJWiY(spOX1L6H0j8BAE+X%Gmw8M8zXgPd?PjYi$KkfTXV4_sr< z+>ow2YTe4#gj&o!NR)3znKOIzyo_gE9{oN`G>t2lnpM%wf3nv3)6_qQ$nggpg!BuA zK${RE8*{Yq*>`pcR3eo&`Oh~b%Q4LF3em3QVIOvHIT?0>y1TW_K-pSuK4Ap!9bTMC z0_#KL{dm(PqOb}W#%Ab0YUX;2Ih1o%mE7%_G3YdI@H}^)MfmndxH2-y3A!)BhMasc zW8tIa*00w`8?lY#y<Xj=hZl-9O*1hZeiI~ZTVXW&PH=^?%+4-BxRx6t+imS zJ^%35btUD}O*hB5M0154_D%pe3ra2fTq8cd1>L3}KD4b)K%yNL3p(fpWgAP<7}$3M z+bN;@0^16%J)d6Uh64@8t~X4=k(&RsrGNFPSBbEdYQs5kj42=3#5MlAtwr!AmH{|A zvFEo;ufP5Tw6ROQXlUz8z1GqH>w5e@D2RJL{@@dq=LcQxU+;Bfh!ALdWXLZ0{{LR+ z<1+#G|9AoX-xm`9!9-gp{*&^SDiyS~f4$)q6D;&ulJER2x5@R6!`ft6YIhBKQ(t$~ z|9W>X9Ryf04ln(oA_F{$sJC$%E)eayP94S9>n(Jet#60Gpd-Z!wgsG=m`Lb{L4+-Rs)ZfQd;Z2t1-`=5#QV1E^fl+!@q~ zDrj%$G;3plf5=CgX!>F8hI{i_M^Lg;YMq>|4Ng1l4MnS0ebIX*g7VjW1kk}7?`Kr- zDFyju`d#T@GgN=qY&@tvQ|a;o_#BP*%s0aWEe7x`p|`Q8gadMrY{2jt9izhZtYHsd#^Ny!w=K)K_nCzjE-%xZxHa)!MD}ZY6!mF_;Ug4k`{Qv0MKqAx zU|*9WkIxP~fE!R%X)!s>WU%o?GOewnb`RkeD8fh|zJ7m0XJ+qr-$WxjRlXGSb?@@# z+`O3Le`4sl%#@2uYUtauDbV4Eg0o+#va(!D&H z*xmoKM*oz;`29YG$5lG3H$lgiA8p{$@bJBoCgRHG2ek^7eqo^8SsKPo%BEx)pCAF3UIEv><)Q@|nlY zlt;PZM2@1la^pdX!Kk=tfI}6|{ZljkyDbo3f$rZ(*vKvaO)~34S$Z1+*SJaVm}9{xH-@1!F)K>x04ntm^=jUry8TzCXgXff{e7Y z*>oLMlI}zfQl(tgD7>@$faFb@U_ut3)wg$`t1tzjCW;STw(OrdY_GhAbxQ#DWbx<< z|J0*S6l-)LZzJzc$bwo*9u&f+_x>Sgm!V0Db@I=FEtmoDYsKPGuMEq(Ua;Rk$SC9w zIyz$igM@;G?&|ZaSgj)ZaXIW)k#=1$_`XK;wSsIcw(W*t=%l}tVoiv8T@IUKY?8rX z)%)*l4Q!QYnNhxg%z}a2Qffh;16iulROUJO;pTE`M$wZ5jL>ga-pLReT5ps@!T5b# zRgF74Lf|jfOo6n65OV(Eu{(bfK~ZbM*whzwxmcJi!rtpGP^ikUt*sY(zdyRWxq0;S zTwkBAX93V;jnI{6Qg6Hbm54R-#9$IvPN;jN@7eY!htf+lo1L|wOsS}?rL=ZG#EDM{ zRhf;hjYxCyuEoUhkQk|k=|JI!G`^g zH?07McBvPS~o(2hr=%B!J? zArctVg*mmcd?hZk@oWND1K6d>=XO*g_9;Q-#drOt`g~T&+|0%oFU^L_ksXu@Fg}5q z9#G1O#_oOh6a#RHbLLwCKXzuP0Vx5PhAMRyL?7)dka?&|1;%*XnLu1!&L`ywhrBm< z8!O{=#^to#84g-MzGr9w^6qyyVyHxkWvHr8t`qls3%iNnlCF7 z0V;V?o_56_r}O-jsaUI7+9para}#Tec_~2q4v^-|DUut4*QGGY#g2wE{~WzN*Mj+G z6yJW_-E~aZ!)@Y@t(Xp8uBlT8(e&s((lzH%Zyyb6BFPMtu= z!jMA%K+k~X`+RI-twagg4tsgK6qIjyb zk)&TmmH+o-2Z}^D1&o})goE~WxkKsiRrmCSqDg~78_mNf3-Vv@DA1w>CQG%F5^aco zf9?PG`u{W+|L^|%^Cu25ve~;mVe+WKnvw zWj5?j7tn7v056Fri;0W`HW(d_TK7$q81(Zl&!<9Ea`K?k^AT!RpOeQQ+75)2zWTA8|nAC0b?{ZgpU8P{9nc)nco zdfdg98x0BruD4^i2=rh3%91%T8fz&v2S|$kwYDw(6kh?%&~3^=kfm`=B$VT_YQG$5lu|i?b6Z@zfFi zry284aS^4WNW5qhZjMW$iy)V&S}A+<3dph@$4w-xWwsod^Wi4s@%XYmmG@#2?|@PT z41zf5GS$*QtF@yZ&}%o8_J|VQF?+n-ygY2qsf-v`4k1*HuQBN_{2?<6g761$#_qLGk1DL1}H&zPwQZ*p=B9f+D_iS^EtSAY39TXTs0 zkkxZ%BKPg>ro-vRC(_`Bk2e`2VO3bvn!(qrb6dZ-IN2_C%A6KG0^DqPU(>5~su4bJ zdTK@k0+MPxt9iGB7hf>}U2#v?>DSW$bGX96M2>=t9_7DIfsfJuGUSWvBj$%>luOhk zpJ4E63-u>eib)jib1y$+yy%`s)#!(r>ZDJ~OVysu@D4jug!T4oe;5}a&iHEizy zuaFFe-UTPs8B%df^1uee1*<)9Nqo$^G;}Xg;@D5K);cJR5nAqvl#hg!2gh?e{gCU={WSUOHQNnX4L~IvF=q+L z0An&U#cVttkF!{~m)LzD+x-zG8TC7t?w@Fm$>^jaWX6UYE!jll)|=>wbQ;yRY7u1O zpfr%q2BRyyG{Ef=JbEnCRHIg@e);L%g)=2!((J55d4rNQmO?(qk`hq?~T@};AO!50RFfq$FnsHZVi_Aa0gn-A(p+X6f(|_ zOKF19oDO){jjMF()n#klDqiiY@XJFHp~KJbGAM2~iZuM2T|`;5Ir@N%33(F?3spHm z&G*PBlhArlO$zV%Wv1_VED@HJ5b}Lk68R%4!wK(`62m|dx=S+p{ieZbSVp)na38Tr zTqYHX^ToTRB!$FQ5nG+Ha$yAL>-xcd*+U#T=?&}!@}^ohu$DRM02^9$JdVJ^qtCFK zfa7Yv<{iFHoF7gZ11Gel?lHhW7Kr%tel9nI0@<2 zmHwq?E-+$p53WW4y3KinT*tRUfh_^xk!MZ>n~#{xSR%|B(q>Pb zuj>x744cz??S|hZ{g^o!LO}wX14(uto-;v0GK50E{61DBMfc3V#sWdWHNJ9HI-{E+)We8MfjDGrk@m<-hK8zSy4~M4nmU z!&*JsAo5FMF~FrkG6A7-207CSm#dv2?Hc2!U#@^k0)y_xfp=X2DbQ1xZDu$S_04aFnly9%;RvdrEq9Kb%)8bf-=fb$Flrj{_8L7%f+Z#mi5Zp1FFMR7!DNHA4C;DuM zAA7CJtGeeeokY&UmRP1A&Ld~{acfp;+GMWwICJBjH3T82Qr)$6x}V-qIfDkDqFfy3 z8zCuYy>|}lG9(8}z$r6Qo?Yd=jsO5^j3SBw(WlG@9ecAg>CU3PmYT2UdNVpiu~A)# zHGo<+H&A19ij|1t^*@ErKUHyyr!X-;Dg4$~krHd5y+K6gpxd1&$}#L!8L92cpad>r zirG?BxSU^LdE8GTzhPY``vSdpNX@E8{2z^AGWMTRac;_-zg#biONi4112)i+SkR+X z*xJ4c9|T&W(b%Eaw}K6*?Z9DoEE_cAGsvrb=sf*EMYckG-Q3$X%VCC<87u!jn(jU% ztm~fHJBENI$G~zbql?fD{_Ie4f3|8_S2cZ1&dRkFdhwWArYG{d#Y6wLzURfB%U-4D z;(O(xi(C%W(vmK2)Cv<;YrC92&SGVsROhW>7@iVXo}-xqqHhDJ9p7x3RWe?Gec>r{ zrIO97n4zpiXlQ#m7rUF*YqC)=KR2(xP8(Qpk^>as+4tOWCBiJr39sq21vtjHq5}V| z-2j9P^8_L9vfZ^hd|7CqN|Vcc-gqsG!usXTH-<&n7aMfI#r00RZLR4dA#J^egn;ov zk!^NL>>P@78xgOXkXVXE-KUd1cx3u{@}c%DZ$Q)v91sH$$;TnS5BE1mCpr0CiAVR| zwPrrdWWZspfhQF=8{}}TAdXq97K^Ym{PY%YicQ6*i5~pD1adWg#M87a>X^*uKL^1^ zkueCu?aUezw29ICsN|>yZ&^;X8gHCt%Iw}a?2mwQORVK1quPZx(>wVB=3(UX#^=m; z{XgGZsK1BlL6`otAq{MST-(XOGhaVP*};?rc;?KaA1)Zny%C!%W*stt@~3QD8Mcaf zFRsGaB}hri-#?cF!;Fj!pHA~^I|0jD=E26#ph*5;h%1bfi{`_Km+r`W5oU%~>`k0U zo?lMu>sor8%3^WRBvzDC!is`A{W8l;UR-_t_=wRJA^B2S4Ab|(?-u{%WKe;fK?xy9 z279N`r0#7K*<8d$?#e~OC(FM^)UD%f(4kt9lK?^ma+R z9rmfWA;56K)_Gj8f4fC1N2ys4lN5FAd2jR%Vt1flch|-lgDGTJXVz`Kgo13h6v81- zTVVP+>m@dR3(PfX;p_2;!MPtO(L=!bwwn^BC(!T{qxWpMZr-P>-E!UU;(f8Oq2kTC zDHrX2I(sz2ZDPd^vlsr&@|LtuY|ja?4C4~sM7XK=L4uxO@m2LPlZfY-oy(!!lYs$O zJ1~Zvfn^=#MH`h2I6!Hr=c%&USxp%DIgI{f0`uo`m_g$l-HNUN*Ic9;9tzz+PYD*m z>fKuu@&KBw(ADm+zt=Gn6b}gp7SeSG3kDmL5d8;824frAromw|-tgGkjFVaAGNG`% z<6&&7xj7TeqqP}|_v^DiU%JBiHvAP4(?3Zw zYC4*#4fuTfo;oOPTo7;;pe_|<@2b$-B>#E|`Hr~z6heK9f^mJ~4|CUgth>|?6oTr!yspB zQ}446xwzv6=;Yz>@Q4TNs3^f%!T%x%Ein|SQNA)-TRKfIf3B*C5hvn?w{VALNN{zO zr-b0K$P&HW#rz@$dwBwnHzY6nFRrr0M?y>r|A^?=jYaB7EAEa6ML|yQr9$kbIdG+bvt7v@iho~s6$S{f53A&5|M|oJDX91ZNx_^j zkSj}`)?E7_B6fl5l+E}E;TL*K8T!VO`h}(@bh#Ma?gr;Y93x;*>p+Ai^!z#RM?{a z@qVAdb&e>;d$H2we9a`g>%+|pPwHxjN^J>9L^*hDRiq4T#U9r5Rm_f1A!EpDu^ z#-VR`=tpyp6>=wAmhF|?1CPvK%^G)p{hsW7q1pD~ps((Q4vl(V4fxFdX>YSR<&nJ?&?AN9qgIB%w?6-M-^oGLoz@oa2 zb=(c5-0te*vO1=RBe%Wnt0t)5p4jIaD--40nK4FtWHXRh#UqH`uxy2WW zmaQ-d)u2^=;)LSE{K(@T4Lqj}Lst&(+eIIjnbH%ltY)*==+w&mffLCg(JB^N7Mz#q7xOo^1Yi{#AT7JTLH?y+TvS4^m=*|~fPTV1JyEzQboJ9-HbU+6 z&2VI#-Tv#|(;g*Gj}@JIkB{BM^LBWuhA4f;zKu-J*B1C8Iy;bChi)X`8Y|s_^d|F9 zc1huEfr6Oh2fhzO;!Gzw-IWJneD_L*vv$qyn}o_O3k@#2C1+$mpB{3chC~<=IDI`| ziey+4??|x!+j8k{6X)J=d$9;o-Q}dYg&I?JTzK>kCm=#T!G32Yz6kn$3_V zIC}h;lxa1ZlXaPId5eO-vcJ#eM5X6*Wq&e;Gw|MNV2{~-0D6IlC6h>4f#|gD1)21o zYSXvVrDalFW_^&oyIo^8^FzPuTN>8)`m>vmQ~!z*zq>64BNI;u=!N!YoV*@xT;HVO z6R5M<#eeg35hRJ@I8?bw6C`psH3e>eg8=ugCboDXM~Gxa`Q;& zdctx}>Q!%4`p`}TJjin{M{M*wRF8WhVOUMq>*-=?(TjoI_0Y&>zW{DuJ!Bt_#auP! zP%!CovI+SWH5T&rVuXHI@HRbG^=8u;sxXygwiT6t-_e7F%-erZLLW}3&7vB9N*>6= zq{W;-*f^yi7;BjDmlWU03Lo}fXt{1LxYOUfw|Z};ngjP$MQ3o_$A31Puf~oPm*?#3 zi+mB|-&P3O95kMb{I{rt%8mQ$f{%g4?3%y*}RmwgGe znZFy$9n6*A$qdpIVhaV~$GKjl;is}Odouw#@YR$BRLI`cE+=K z(in|<>(@}f?)%?wBp@hH)2Y?@yRa6k^jgcL_Q1F1B1p9o^11W`H`B+a#IxoK@_mZ? z2rMdAhNFRJvYqe7Oh`hd-1y;mfy$1iXeK8M^{mp%w8H!2O;%3pEy{P&kzU{4TrZfs z)Qg0h?24<{q_R3lw7)&jqE^XgUmuL=P1I*{rRK2McR^T~{M5JE7*Aj2Pxl9L|mZS4o&n**bxk85d28V=_+l>$~@|Y|! z?K!Knzqor2beD{m`~l#&I<>uwPdPfJEyg@Znkv(m?_%v&n*KT5@KrJrSe zQvQv%I;$yv^<;U5+mjwaC30J0(UM?$`t%L6{v27pQWZwn#dPv4bDsQ2c60S;sMhV+ zIkvyVD|(6b-pJ4Ek1c7Wn#G>diRrxaTqBV~pp@z`Aq=VJ1X**1l3kqB~&x6Qq!1dOFd(U5G2*cV+LrSO6FK1d!a~{7I&RG1-kp?S!)_dh=nDY`BZb35U(k_OBYJ(d~ zk#t(l&nb7$U z2{U+?9#@1RH&tTB_E&ulnbF5>>gJ~J@2r_v?cA@T{WZAe%)Yj99 z*W32zuQOV<8u&#q7dR+sI6d0}P(|tD$8yMaaP2PZ0?P}vFX2N7SiK^;{?zdhQZ6Rn z69VgBbq^gL1yar%_pIJ`8c!!9zbx62TD{cpRbsO>RdcYh6k#u_9(V9yo(PGyZa?k> z$(c2ga<)K#;$el_x7+}Bi1ASD$JJLMzdurz?h##`!<4b|f*Q6Y*S# zDz+ojBi&_)&YzwrIorTs@VGe-C(CBQQYo&YxZE8sj_!qt7(Q@wxHOB5f6GUA3g~xy zO;eE+KuzH4lg!;Pl!~2N=EZ#0+;~Je_&tXHB|Y<&PCojxKZVyXqHy|L zb2>5D%ZdB^*FJN#4w6(x=!(8M)FDJHLS;n!DvAb>Fv#)j>ON&52Ysn%+z0oB*1a!# zT}mOPj&unlm53R+pT2kAr9T}D;{|FVcpV%{Jk}ngIiE7Hg6-#q5_xTyU-hH}MAPaR zI2|DMSu)%(<;t+#q*;?#uK9fUe3Pi-%q7xTtEXS#Kxdh!bde0yYi?r{ zVyAjfDvuK&TQM`$?9cT#BvXDhtuf=~A8O$@z?L9(PKSbtE80PtJzkjSlnqesFNDy{ zH*QGgifkM~HJbL+qGP?2_s7r%NyMxUqK(BF z!V|dNQE2*svu}l~Wg51wpQ-iY!C1D7K{A{7HQQh^-{vn;8OoU2#nz*{4oOAzGJ<@F zTN-n^9)is!?s5x}=Z^(|JLWo8^^E;;A)UFKT(rOLVKK{3=BvtA((gbf{|N}rSwRZM zW%|&{vWXM-WgRk?o#2_C-YYGUwMYi&aZZ!=UJ1}jry)d9=|*AC`FbsMN3l|vc#ez!wBX9QHWRRvXzH44*YMJpo2jF zkQ)zbJoM>lc~7Ij$xUR9+$B$15|Q1%aXT27!eOzuD7P-KiqB-XAaKBKvY1cha5$w* z1s8O8rYV#&b{}?`Q?Uux6W~e_FaKb}+jmIwKJf0JLehuQdoUlIGlV>&%_WPOkj96V zP8f*#aE4E^9wn@XQWz_Qc36z@1q5)9kVc8l@C8V`+%j-$-g#0A94m(L;QldbNK4z0 zjt>l5cq^8V)$KwCy={5#N>RVP0BMLkPNB0SSC$=8@b4soR1V`IcNgQ@D0HIaoa9el zd6m~37DDbA%XMiEl?-?*cZvd*49k>R`d;THcK~0=8H=l#&dDyBvF~q?wU||h>O&DbGg#gI-S9v1=N*GwXx{Y-X^*G)<%2N|WbzC^R>C5@JFGkptk z&wP&?QUqGsTp9(@b^0XHWNJ|=$=7591jtMD*F1@}(L*#rgrAskk5|A{S(LHzQ-K|cvqOz4k{G%%eEGbOFOuy`}nJ)E3g0SKaTzQ7XDW9|B&~%95Wm|>4 z($TPqL_HWB*jF!~#uee>tTJtt+fUMnJzU1OQaJyeh?p3oU_4;wtZ6Jel&Y#r^pipb z#VA^lYC_k?pZs@{xgX5Toh}(dhKXzwv5TGI<|&NRA3Fyt!uw4!9T=xEmdcMjB@2z7 zImwcnclK6f;)TVAN&zXx{W-on@hyQFo+UNp=ex2b4*Py`q;$mx?y&<)ygt7}2_i0a z@;9S0T))EP6P;G&&zE-Ok`#N+y^&Q+tBs~R_f0v(w2QwOsJL))3|V=gP8HFy)>10E zBISGYK0jP5@B<%;?y0XaIk|<366pXoN&B^c=(C=U1H0kG%z{n0k*dokrW}&nr>+M0 znLP;;!{3Uo+~^v&hut|}drgnaQA^)!&*0-l=zvh|EOx7O zBrxXlCXUPWA+Ilx_&y+HnGSM#jM4QS8`Up(}z_cqYGEM z*Q}l6u2hupn2YyTz~-B_P8DDbn5XO&;}BTt#T#@<8WF!$wwPW+!&8OY`24 z;6@=@C#HOXMNnwuBNq`noH3eEvw)IOkNLWX5Q{bzbF4_NLq)w{!cvi3lB@R3)a-2$ zy=MgsWXEP7OPQQ2Eb6DO%z|g4Dgo4bt76A|*q2hje%6|Kxh^`_}9C{r&cC+k6^^H8X3? zI?rQ2jy;BQxE2m@PKI3w5c{fkdo(&HcLOT6n{*Lh*b$vZW_}y4wByndTk?drXc$1@ z$8WUj&y1_ozwk7lIkfPvDoVeGtY&_`@?;FKiAB04nED9=f8oc|cJ_6c$zzaB0m7eS zHA-RkmFV18qduW`ZGhz?`gEty5sX}jgH|L(B}#URty-pVkU85kk&EZ4C@Ufe>nFh| zSRve!$|so^WXG^V3o7iOLAfe|!-q8~-5Qa?PucJ<4DXF$Mq z4}o9$9sJ#8ig1PGCAJQ;Ien4lVWYdbug}T}WF4bfkG|gG(ou(=|7hzVut*jnC!@ zCHbV6&`TW4H}8%vR659s&MU1jHs#t z6#Q3}pX7OlAWd)}i%vUI6k`MW{Nio)5BJaRrX5dt%HE;~;GLN%ZOEb6xoa#8{CtN* zqeL0Ka>C&-!OfY2RZm~eS!{kktm=p0)G(Hxr=E>pZ}WXc7n?#}5bqllYP|IKg|bRo zr3~`Uuc+OyriN$=<52K3pke$39z(K8KB;tIsmu8guLUwR7jT;oQ58?}S(AI#8{yeP0*T zZGnba6KXeWVl?pa{qpUZ1{7EP*plXZxYi8w)idK@u1Ri5+aKxD{gee4EJ#eX{q5uB zneNuOG6@_q{RsI#iI7*ujpMcNdfY8-Xfii2GOgkReB5|ru=?o-8EloFUuNPBl%ULg zt&6>-y1wtah3mui$tcCZw7yvQu}QG4QLbrXI$gyZzWZAZBVwn&^rEH2i~6k3+P-v? zexL?@EE)Om-tI;^)$4QFz<$)PZM7gHmQ)`0$<6OhBAQy6?4{0cK1QQK-Uz2Kz{A+A z8hILS4omrUUv=1_t=o|kLB86p@2VNO?SBhs7m-$cd|#>dK7WCSCLux#!!3RU4fjQrSIT(usJ}E673IP>8o|Rg? zCU}Ir(Y`qt0S&hUT+Lx}K`T0#wZSYb$Fj9r^yqr0Nknc6rytg7!|;B?M97B$zAUak z6%*6fp>)~t9k=M$L{q8Z*j6378L;#el=OG4ozku_jdx|RnEEIs=DEhrfLIQz7X%x< z#M(d}LqWYwtl8@$B3XZ)NMprpjq`=YD42QJF8u65PX*CunLSHKhV>}nJ6a~G1*e_< zR8dGYt#-(Ievq$4!|6Loe?N)=#__UlzI8X_asx#bo78lyF_8B7n_%;Iho2NI=?7Xx zyC=FY!a1mrXSLtgwfQ|ae5)!ZZda7Q{w*Cbz7MYLli~^NGcM0tAI!NKgrvGQ_%F|i zIL~1mPh^Q*k+9kp*e5%`zG3wJN$L?CvIQ!gueKv zgI~`8GINN%`|Ry$wf@dNc^c0RQh^&IZyFnU+Wgq+{`k&AIu=BF;e0MudE-1#zTxH- zUQf>oub);+CbvbexBl4TFiRwh9jW?v(hU6`-99xN;x48Si;DP9f8rLJ>)}0%7Qg+F zKDIPtf3Z-P#R+d6o-rJJ{ILWL>w5hl_$M+vU zcaaAzVE4^LF2q{qIE+IR^Jepnyh_|`4WM&NS=<~(>Kp7rEf>z&wBomVwrXzKYu|U* zZW!J4@vCY*r}p&WV2qDIXwxKU<;FO|8(j_w7?25IDdAc3b`gvxM`U7~6+*Iip8kg( zbx#HTU9mJceAxMkgm

3x?%^X_H>>&s}ECSr4u7xY-(sV#MsVSAs)aPaBWXPNu& z*FBs@nY)h;cwHHXqwx7GHMM;@l{h!$1h41?`fG;aMxlz9p`qvwy_Y{yZ6EKjftoq z>MA{MCRJC^=rAvBs_3ahHCeo$sd9v@Q#|YUfZ=l-LSF6dr_o|qq;?hgIk;1cMKG-b%LPLbRPtW{%UBw>JRUcSZFW~PW%w%WZ9^FbZ@%r!`&TxOJn@(u%k=vDrRD=@Vl3~< z;NSE);!r&QH3quKZyiLe^0}5bHD|t>4X*ZSvx`^`tjE^b^}onsDCu35=H;ZWso8jA z3AN_UjH5;Li;f;<@(F3{464%sf<@S;Bw_v%Hj&9D3%r`{`=EfeNImEP^iWqP3#+88 z_8XWKgnJL(`UYX#ej|b3I9Ca2m^CqF7m~K;MU^}k>!-KJb3;dh3`4}9s2;&L5wbBv zK(;VGdSi#!J0@6~G73Lr6IAA;Ljk+`6eMqR$ll(e09%}xC(7)f-v_U`KK=0hgpRCF z#4hWU+XaXg|i4RsUH;UphRf zQD031J>x&U_y2q}9VYOuE$fwZ4jBLb11V_D$Lszo`D+3LzRP7I9L(m+p2iE8q2uwL zO%UZ_$q)i|h;iT<0<1a?l%3&-|NWi)e#8+W6RbT6764i>a;4x;mN8Q&5Ed*x1K|io zL=ev?fz4`}0i=%dJIXW^AQ{7QoPr*j$L&0M!OK~OAwvWyWc4!NxmPq}~l zF%?v23oyAe({iP9p`W#60uue@F;W;J5Q}EcZ_m|QO$w*nShtnK%Nzc631F~%o+O~_ z{iFOL5mt2)!36ODLi*jI7fQpH^KNo?&qF=xUv&HiGD5{>LtIX~b!s*0H?Dxf#2?AD z9EQgfCLTv)g|m~g=XH<{{p{-y5SL8nn^q?(f^q&HuJJ6eARKv+PPXT22C5Pii2KPp zO{2|1^NHaD@XnY$dl(7xEKX}acERE9aB#gjFXpkLVgue9zKlof{n-Dep7VPPbDFi%$Uu!k2V#_uM1k4(OAwdIm)N?b@~CWKd&_2ZH9G(9 z2Hxp-(Q9jLYt;Us1!JSt`AEKyevivON8ocn#Vjxja9)Ku>)dRoYo)D#FGLa!!;=6L z_>2ux`xov7S0K=VAuoAmL51v>I(!-=&vKX~XNO-+tj(vWcj?u)32@<`$DmtXbN5CW(l9 z&kXM?M|8+v|Ks^2VhPl*ZI)Jr(uF(cR;>VOO^hsun*rRGed|y{#ON0xuY&$&4Y9w5 zoq)6f-A>+rJUO=~YuSKQAp*>0+BUu`71qq97s@qb0qNBj?4#M5hrJyfp+ICGU=ZB4 zKV4~*iNjH$u=?wp$$7F1v-OZHJU%ZsV2MNkCcSjFcwDB2Df1KC?J-PXd(TIO3-+(? zz-*(Uu{jO6b3|M{bV{T#ZkN#;y}!0u>H0odfWqT+551Qy>25g!Q7WVWcFlhEaQ_wq zWtx_qX9KYS=I=9UVkNw>r@ov9y@_9dlZ6llt>&`+VS#cre1Foz%W2yL?lmJI7ke7> zP)^Ne&t<<`6m?eZ5XTfv6My(do4ZV#fUzMMd%Ms!fv#SqZ!dWMCci5Td1u;9@PNRCJMfEF?RI9hOT`m|-s zTXTm=ZyosT%$m4>!MrDxDE!%)-|kFx(x^hHGf}YN$>|o;*X?1W2MUGU@>U)8ojTL3 zF44vZ@3v&TTLPI>?lt{v>O-4HecCN#A}t&VkF(<*QUs<4>a9SnX6>H+%EtXx5>VzE z^HJWkOC99F&Flv3yvm4%=bgB9U_Wu}gec*3jh<9YtNlOxBOG?yG26l3-q-2l!S`k~ zn>X*Dkq4{ep+~ti+VAJNUSiALULL_33Pe18!Ce=J3+x4(F7a?Xv!%yNznayV6fh{HyT+%q08C_{F2*$2A~cgg&!q)HmsZialv!37Tx-fIqz!HaX@p0whpO!P|KcD=8y zk-nAUaGng$`V?tn=sO>RGNBi&Ff+k$hR_-f1Qu$?ST1uU;ycl$D#eGU<~&l{T=AI5 z3k$F0@q@W;ySN?w6l9f+;?cHGr=X2G1O(M`P0lvc)3dWzp9~~J7LGS`Ak@3OJaUo|s+r2K1;Vt>)52zr8EB1B|k*K^kn6NTt_U@Z0U&C zK&G@O*G@$9`HD{Sz9*8TQfgrTs4%<`U5Jf_28-2Pbv!=oLxYbgTL!MkSf=JQqcShB z|1Rfb#Rtu*Bp})o0`lZ^${4Bm$zdRKG?5Si{azjNrhsEO8D%fUs}j^;?+F!&vW zR$lXXore&)pY*e}5K&?z&w}9V!!rd@ic}k<{nc9>1`|-{pfPKwzKlm_36(rwb zP6BjZyQ?|_HCxkoWD>E_Ziftf?SfL#RgP&1U>|XIAbMB zpH(7F7BW~K&3NMOIw1TJ+vA1DZ%~!f3Ko`LK~`T-&83rZWo)AZLDpghnNwHDG#ZI5 zn&sv{8YAp95ayCb9tbiRXjcb~M&AN{FE{C}>!}Zu42wJM7H`iG1jz`e4G8@zPY@7R+WJ zZIYCQB<^iiO#tTZdwz>b95|?_`|(S79*t(z6lECBzjjU(*alz^l?a3SF8-wj8w~;n zh#Y)$-DYZWUxS~`%oM?w&yy9X4wiZG6JF46qc4kfk5hK1W^Wp4o#7I`gbM=o?o_3D z{NiAKql=v^-P0cdR}csy#{-}iwigU6-esW*ULD1JEF$)ciLKIIXa_Ma`F3%Z&!bj* zqEO6Hu&FNlGX6^$OE9Oc;p5Lh6w`i}8gu z45c3u(s7v0=c3WeUXyq(SxFOE#j6W2w96Mp02#oEayZ(jzDt_M@AsEFE#~FwXL+o5 z`NDN&t28i~yvVo#NUI`$VSqM2e)91^8*|ycjMsY&=ie=HXY1}_J`GY$8sA1^GF$v4 z$Dr^!7CtyT+hLn3A_k@P9M?_Hq!AJ+A3aa7ULw&Gd_{y__m3W4@p+db^c>R?)3ceA zZ+Y4U1OQt^gw1j;3yheN*U9T%Qy&xnuAnMTtNT^3+ahjfr=PlHNhkM9jiYlAzk8 zNU?P+Nt{56@>Rsh7sB#U!|-o%oHr9Ro@ra+Sixs@en{KC2VRx?8KViQ>4M zR-rlxrRI?9IJS{Z$A6I5|8{?$K?wrc=+A>1f@1#}5Y|3z1~?~fnt+}gXO&}cPPP3H?p0O7Kun6n;x6PUfB5x$3U`g}u`TzkQF z?J{FeDZ9i4iDL_I9kG7XlX4md%H&stDuvtB`MN7k8v|PE4dxt=CZO^;k|^d7nrDCG zNH18JUq%0bL8|BEIMJ{jHX{FpR_Ik47ut%g0{S}pP&Re$Aa$TrEdYcTNWA&_MZ47# zH)1uU4upSZ(ayWdZdWbJSD=~X@wdGj19G8uzA`6(o^pouyM_T#cV2vEb~Y&tk4IQm za}LF6MCU(VekrTGZf{#B2DN=G7n*t`OSKyf2WR7Ed%PYVbv8Qt%l|>TkNcF*5 zDQy%jZH*c!67BeuQd=&zNbR$ZxeGZim4RG@^Q%9=s!Kq$m7~^lG<8VTaWLRdya|Yl`8mG&NHtP7#$L&%UZSXLs0{;clpid>cqrjsR^|>ogy&`l)uH& zB=-J*x=Xev*YEgsk>!-;IU~GFEt$yO8MIh+^&%f+`JN+K7!Oxe@A6smuawV#=vd9O z;?o^IQD6!|3DCOT@$&ld8gl<6Y1dNNTpLAIwgS=Q1_h-`T6D9MFohm6cPw0gc zbhYcnS=4GRn?oB`K$0)Q)lz#nP8d)+pcQqwp&q_HTX;Lq;96qOLF09MgAeJFN^+=$ zJ*#Kp$6WGBA@1;5JjGg7)s^aE^FeeT1yDZPM4yN~O8 zEe0X4^Hz5#MMFXOE zkB-+}cJgeM30XYBghzhZSiXV;*->H3-F_H@H$MNdNTiwb{jH1yunPkfiJ)O5uhKQr zTWTckL+>i*`ULZbBeHzO`T`s_E{q5sH_sUD=9_{M6go?87vNvcGUo|$IYkOoN=X;J zS_NjDHi>1~xt|$?;IOFoTF%vd_gjD+58{o6|CQ#27h7e5Kb{icXfgYXa|J^W9h00c z$!jVDRi7Mi`8qtm9}Wprf@peWqKT~bUIz9aH-Jlv0X0}N^4yD z4z~CCW3lFufmF(f24S0@Fw;qrm&TE=Jk0{pC{<+}99=$mGUMb1t2ryPrya_0W@yd)a_H?(HtB{ba$TBM~6UA*V;8&~A#4 zx(y<;-M#}ps>x-qwnlsP!-d=nYQu17Q$eIZpPCv_M;a$G7+E+m*2i`pno32?J^Y>Dm-@` zjD2x#_(pkwP&K9&7)D@mxV$qT$E7+(jz3UsyE(i$(91Dk?K{DedpKV_ZYGzni!ge# zfyu>cy{v8J=r_~?gG3zR_0DF~|1+0lspCVF2*|H2HqdJI?5Bug``?N|j7$w0voID* z(E*D{#V4_~^aTD7eeiYm`y*1vYyDJPOj!z1IHd{iV4IQ@;xg39`br3ZHD!G!`hJul z4$8}(1f~E_knw6Oa-H{l>R!O6LR<(W1@Fts&-I>-^N~|Wbnwocix<)>6psjR{p5mT zD!P^7+tU`RbGvj!qgB^j?1>ak;PKk`?P?66gZ=^|>iwUGZz%r|<%{}MS8?Tdbi1hv z4TGh_-$cGKZkH&kJ*hNF?uYlPS|+8(?;R32%xaiTky(#u)iUbJ6|Bsou05opIL#d< zT&iIDExS3Wnl$zQQwabH|opYF_Ra8_9rw$Uk<(#XD_?`&=n z`Wm|zuV!X>B&{YM$_gs&KnK>K^~bR>AqZ>}BpA(2jy`;7aur7R%2%T_D5&_PZ6MqlP%9 zlO+_u=_c}L7h%2om2D2+28>9Eua$jvB&GyAH4@Hg?A7mviS>XTyf_`tJ0+=RStrGt zI371v4=YmX#E;PhtC6G*SP85i{-ct90wGquiQ*QlxnkdW$@!3Xnf1EOVmJig&&XKT zwI@Q1(-?d}T2XOd%0&Xm&;E11VMw4VAuf~4rFP|R&+gaZ5D**?aL-FPt$vD)d4_C7 zD9klorRCcdhJLmyEw`Fe&7O5)d`&?ke~MT<_=aw6OkbMc7MK`ojsE!l2JzRga5IoW zU&Sq`zA$eur(#~~mmfapj$2}p#L8*UF5KzG z#G=W+CnB`DTo6|7{B|KOkc*VWmlktfx!v85-_e$~KhVQKT7;tazf>T*{(_(1ev~qm zZ;i1UfaHDlls}n6j-rpccQ?y9$7#@w?II7U_2@|#g@Qi^Lt~sB+g-aev2=3Bk(v!1 zQui9jj8zg#;p{Vq@E~4HmIfp`9po@p+t7=(Uw$<5`MZ`A)~)2QCP`vQ3y-xevurm^ zxVZWV$3Ql6NhH?Sk>{C$C122yaz1yz`rsw8Qi5~adKP*|2=$uazmGs3m?B$b&mOkm zy>8+()I0L`b`oLu?CnR{Xn$S=UJd&kdU%Ed)*rD9;m@}4PenOQr0axe`LL{b3emUx z*Z&ass7iZ71gP=(S=0VpGO(mV3xTK8MJsUr_42?qm^_@0W2jFjV*g!rAD3SOS@1MW zFmdnSAN&7*=>OlI5I9$8vcY7yYt{;%KbPB@4o;S%F78x~?awj=s#F*hD9RTC%x}&A zs=L6`V8eh?LA<}(NOPM`6Wu{XK!^eHBEpGbaL^JHq<3tT?U30z2|#;kk*I$olm%|?tZIu zvVueFx)@e_Naw7X9$vRcM(i!X&pz7y>Sa&Tyjz$5dS&Qe-(&H0ApU&n!|5oZOTP98 zm*t@3U{Ut=z$R()(HmcVFWUszvw3oPB@3dT);ZCj~`mT1bD zVA3fTzeD0VjmVNo+b@CK(lNQ1J$R`-%ulearsyR)9B09~N+mMqpKecU1*6g0*9zU| zURgG1s_V?L+q^5jP~VigvhPJnXgMk)mWk(&t)hBkf=Bwl+mgUXmjrn)<#U*o{kJT! z$X^*>(t3LPlOQL9Dv>H0wUOBpZy-mgWS@TgsI59sZ7!9o43kV_P)MwG7IMO%RLLg= zl47T+xjLHmUe6Xl&>=_3L zmt~%=^_KDx4vxN)0nw|gJ8O%gf_oxbH%K@Uq*6M{{>G^B@6Ejo`xA*$`s%!2E)P!t zd&u6e<7Q^5pyBXq(d(hug}W8BO(;;wq#V1Rc+A(E)keKq8EW}Jc4fT$$prvvqqt4b zzHU(Pb8Q8cCEr^sYv2V^QpB=yyYZVyn&YiV)r*yo3*EMjmnn6)Ky{ULw$mD?qFz1B zX*!;dv9ER1HBCW)#YXtxC*D-^l|-af$up+ncyja@LdMq_F<+re^tnNDs#RVLTYur+tv{)S2KfW~H}BgP+LyXXGy zMOI0I`^S%P35UbPn70$i6pCf(_ItbtcFyjarg)+@LU83 z(zxgz-ezaotoV;)x~bFy1G3W{Hrdhca15PDzDUgjyA24*Gx*iG0DMnX^w>~g4)2~Y z3x%YbJ%h&lLha@_V(nY!kwPb(Zer7!Q%p{^t^v`naY;Qu(QcpNcD~C&Gka&YGhs2Z z*Pm*Lu?Dv_+L)Uq5hs0ry+h}6+(V`arD_%Zgi2P_bK&GG_qDNlCUtcrFenxUre;~X zZP2z`BL+cvcZ7s0S;r^it010lBwydHCxt7H@rzXJo=lwOyp#(@cD>w$bP8{AaL4Qi zfAMeczGNxpM>1MmlT~=$?bS@N{^m3}R#ft*Pu--eQ!LO_swc!-@eaBb%*0X5KW;j= z=ptiEv2pDPc}Wax_#oWCOaJ=SI#>b%7W8Z3R-6xq2u<-V-+mbOe>_IBYQ z9Kbtfy|?lc2cd-qadNW$uW(C_P;4fCJzSbyrH-nv#NnpK(C|n+bXQk${&ijHp;VYV zBxSWsMtIk;*zTW}X2*-wbS0r%#Hr;^+syE)cL|-X^=d^J*#TSut<72Nl*a8! zImy#{od*Hhm78>zK2xtcZs)^^djP2mwk*;@KCV@D>zUW7ZOvMxcM&0eL__jmh4@On z-Vb&JmcyNJH&+`xi3^ve#__l>R;Ko{^6%t5RAMj+xCTiiVi4HE0)M!D>%En85 z{Q+86&%JiATCcqe+g6K9h*M%4u%za<;hSwn&weL+@=*MHwvCH4${B>D*D()xY5Y?gz2-wq^No0v3r zZtpG@S#WgdSIQ$PmEv1Pk~PjxK-R@D`7Wa~yrb_>$#SRFY`=atOvU6ba=iq1tqlx5 zyXVc0H=+X7r8lt*UQHz`pQk&8$<*EbjrIlQ7+c44uo-Dv4etlC&sO;~rPkX@IpQ`_ z+tuR5yq;DNZTN)Lvr8jl(C%{*N#pRYCzA2I^2Pg9k!FM6os&Uu>74qQmQpm5xcn0&`Y0?YlV+QZY5F9&Z(LslvC1nU}+EJ>q;%JBqj~w=L;Mz*7F4Om84cW=D~AFCZu}C&+a*k%ZgEpP_iu z`SC>(et3Rmc*W?FFKvOzW0;Iar=~8I8G)e}?`*}`@_eyKm0~)izJK)U*tGm`a`RSL z#79l49zLdX7g10(#Xs3}}!-J7R;30jXsK@78hOx54tW|9pDv7m%wBY z`G}OgHl9#DlsXyXmpo-RAO%h2h823lKIhOvxfP){c!#|i76ZWUlAmHH2A;si@na_Vi%8*P(c?)VU`+Z^HUdN+fsIiKdCWMSYJ;Hk3&by;-{K<_x_iESdFNv_f z1yhWI-^2hKmmnAx$i{tmzbd95{4|Pq?5ZrOM?R`6^z;-DO}5a`fo20lB2?&aNIUFG z8AtnJcbPwjKzC@|P$?A=SkvOMZ_>CFOPTGVI)5}993Kxv)OO6&&05FjE#bJwNAP!rxyY#a z*zl$6=x`WGLv7D`rF$a#+tTZu)ntT>+r@7#Z7PV~xjLMgs?3zjhX>`^#7_)zk4j(I zayz|cuIyIH4Z~GDQmwa*;fOtD+Zt*qrhIwSpD&zh`zyJN$@mai%S1#Ibo2Ux-j{Oq z6eKC6SXYt#EJnENl)y099CBLw(e^nJ5q|vFH@)0rXEQ!WSYYl+(p5jrQG=eN^z zS|+|J_2Z3MDJySA%LPj=bqa@vFgkM5d2!p*j`;)V-y3~ma=__NzUi`(5pW}#yD_9W zX6{Y~U2!OuTQ1#OPy><@SqivBn%<-MJT|6gny}9Aookd(>pXA|NNx$;K+>ABOR1nQHHKvg<< zYh8mFDzK9GnIau5$aNSDBr1gOm`~P(O4BN2ISV?ynj8I%7glTv`RGVkmWiD3Ba5P% zA*gNNhfQGq&?@j3caKJD;LoMoxq1nZ*u;En)sY4DIS9K}I#+RUa4pSsM;%0DcZld{(X)s^Q{( zv9n|W581@R$TYnbsws9rAwJA2~jI^$W6d$wX!$sSNik`7E|pt0h|Y zpe?5;48C;#l&FIB3n=bRm9*w+eDNqcz9)mnnb)rZ^xWOy(4qoixKLZrp018wkhq-( z1_v_FMcHLx-nYX9;fK&&%Kkx!1gt2Rz_VHwR`-6{#1L(_>@}%)9-l6&@!{;u*zpF_ z!0*C4_5rlQ-`}R&MHHX+dEYX83nu4l|MSYnW<(BbN1&RVBn#_XNN7nbH2P=Vw?Z{O ze|0y0VE7n64EU7(#QdvRdE7yf-C$)o0(ta1WB&|yz%PzJg=;@<}$2gsZuIPS}zpZ4QB z?h^8^5uAshYl#2rAPnN5D-*J8@cZBI=my(?q^RJ8!%q8@ca{=6{FZ3LiaWhCC(ubM znD0KKN$CNzCyvA-z2GCqzy6UQnt`8yfa}G4jtA8j;adWUd+VNQY$S+OP$vEyQPGb> zspIw2gOBM~4{1VpT#k~Ov?@bRmhJwbV2)6iZbgPI`q!sh!uK(z0^SY6cgPtKenF-8 zYH#bWDhMnT4X#5H0`;^45to;znfLwFdrj99q{@7q;{@c|4=+gvLZ)9OwEFq z=Bp9>q~k6DL3nG0ez$ar;KkOw!ukF@S)w`E*fBysw@JF^nY30*(<+MV( zAvYkcZLM>=n#5@VdgO7aeRI&tvY6A)CtA#T#>}k|mYXHiJ5llQ11c1o!?Y?kfN?KZ zZpBdA!)&5tl<`}sZH#Hf(#aR;IY5EV24(2+nu-qBTI)4J-nPMy{NZ6g%;Ib2w1G&O5?mTZ2)m+s3nd@N32&+N6%5{@3tQ- zw~sN!3u|6S4D#m+iEkoCI26P%PZDN|e&E9Y>>^T`ue@w}aR2?TCzhqK%4}9xK3kmR z<(s+Gh{Yxs1=fc$p$7XMTHuxu4WcL@`M_^QY&33dkMKjb=(qRs1&U%R#6Z(7uyb`X z*dEEcmpRfhZ>GN9aJJYRO@VKwnInmr4Pw`?9}#S<1$qGB3lN@RGsgn+SusHVCt0-( zlrH=Jia#u|XZx`^?;Ann=9sV07)z~PF-A^t!puvlqXJC98wS((N~h0vuas^+1K=eq z4gZFkAgZ57(v@HnH5$(8>b!BhARgq*iGft74EPa1$6hTgjVp9f?oS+_kJ@QogYyJG z%WoieTolwpS)p>ayr5Mx_Mf3WsDu|iokTog%;y`VuE@lrWz&b|n`{&s9QXF4TY(vE z{-1&KySo>!VnJIDV$fdc@>SUoht3%1+IdTUDs0RzoU~`D)?f6y@3U;PI5r>^@`^+v zlRQ=Dd;0g1m?1)-o{K)M5M+*;$a_-P>`)^0m4_DJ>UPwH*D5*s?i^(^3@~WdMiWLI`i@t7vKwJvAmPcA7~D5aOXF4HHogF71LG7) zTrNeO#UVLGaQ!*r;J8i^DUrylQQi2DZ5_Ly_#v^JGv2D%A3sd}dtX7WDlfc$3 zrcyZJkoj9%2S(vI(E={zRoX!AwCBy9Q%}qe1qfDnVvr4@zK+4|Xi@H39Snt6X6m&+ zvF%kS=ydc%sQa8kDsfExeI7&YkqDm4DvFEKCD%xahe%3YE01%;pt%h9HwE;k#1)2e zNvsxctUo7nxGD2pFC9G_Eq&}?0*7Vh_3iO8tqc&ZQE2;Q0`w=@*7e*v&R2k;Cxp`L zJ~!KLlFeZ$m&;!a3j{3!97M+>0$Ywaamm z1-O&Yxqw$Qk{=Z8rarY<>(v~!{|+#@JA$r32Y~p4|EmEe%SbYb98?vjP9jZT0#{K} zM<{}Ly;on0SpC6Cgk~bK} z9K2O6U;jW);>*tvUvpF=hVm2ay!5oZ&(6=w!UHHINGk1g4UU#c7@blR*tKyek$9ao zD9Vq%X^~-{5J*~X3|uKjg}uE>b!TSiqj-Z@ZyhL=#+y=7)fgAUqrvKf`*61NeSjD} ztmZJlda>Sm@I+@}CyS3+(TSOR2LZ1Bot6E5&fS8k zJhhMz__uQ`RZb;g~$)Ellu;l1{N6S1~GSKGvB1AHcLbfJBQk zfnG0?SUOsj?93X{;<=YtvOPQQMlL-VLrpkdpoD%&{tD-`W~@*}WVthxjPGDtA1#V8 z40D)wB|4c_OC48417BJQI7sGlU#+4>4w1vLgEAcSFd1@nLY|1nRYg|Jz##EBzQ9S~@%2=;Dx*N^FPPk0U8Myr2L4pw zsr^zKJ(2Ihj^*x%y1Qg~wlK8z{2D~czQ$Qo%llR7dW}MQ&))f>;%ees68qgRE$I~Y zcPwU#7?-$9qH;JI@IH1-<|~A$XC~?RI_z)8w8Z0FrI&PZ(epL)BPEmBg~;xH1@tga z30S)hjfLm-4mmAMC?sON&9y9Ax&UJZ@i6@ugYc)@Nwll6E&AtbV4=Y(0%%{;ec99%Sr=lF5BE&ue z=c5KMJ+0tR8%>1>WKmn%YsQzAg53NtXH)8I)4fv+~(o6 zx`+I;7EG;(|8{?c|1=)|s!|{M%#!}aU$yLGeQys3qsb9y`8odIQ&-WY1>`b_z?_=` zj~U5d@AEj_Wd&%%*Fta#?@zNEJY6RO(8e6mVNvW~8<97RZzw<;y4WZ+0sne>|8Wj@ z0n1(ZuY>#O5&^U^CG5BNcOggM<0j}0Y2$$&H2u4n5G9xbAPKexMbGUI=mYij5*|vr zt#$a1O7(wzKJc-Ro{pziZVdF+noabEX}PVgZjGv&H;CZJ6^wd)jaG?E=;3 zaqQ$fh#@?+HZ&y5VRQV+@909I!>aX;wqK*^9hc*N@R6lgt!wCn1CWWmE&YoAAMzf+ zan%0@TnzhC=}K!|gl1|z)!;Fl)$ydFAnx)z9ycl)u)L#uDP)CGGkL$j;+bvL3MF6T zPU6Bmzan3y3b%E5jV(v<(`;s*?}7Tl^j#%D4;0!NRrao#UngriyZ9x44UI2%uI*@NrFd>JTNel^=z}elt7|ZAWDWkU55UFJstYUn5|0 zYC7)C(bU_m!A~ZJr8F4xIqogs(sCE*UxG@YDj5(tzxh5per(x(n!7}5kLh{w-z(rV z1u8#C8%L!Ay~hOi)}g5cg(88DUvt3Eq3=>JuR6$4eyxN9!cCg|a(%_|ypvg$ZFVrD z`xMu)?Fk@{dT@GG+(;@^Dr>1VP`jckrJ-L2CO=F=b66jyKaydRc|E#BAXBJk2Ye5v zY4ifOu2*cou{`f@T^=*ox&r((;V>{Z10H$2Y>f_ZzqppCYXGmx6ra2O^U+lRjfO#I zaLi*pC>F6)v$g~p|DP6a2dnVYJl<&c1aH69MuZ!-1?WRZ46KCq+fxF7x1d8uhMp?VKJaK?9njz}P%b3Y?+#Nz{W?)v1JuAJzv^rbBmm}v z$)QA+)N})*)J9YgKtGNPF9}N+|AcU_NX^c^6aPq+LZ^OPqyORK*^Ii(pti>9zlcdGs+B0JCJCg7r@1pA$m>TuiW%rsStSmmrvRckiOC`1x4FYrb*t-{q z#=J%68GObvH8?URe64qu)m?fSVXSVR>GlFYUQpdFeQtT><&>q_>Z!G&8|>@8GwDTF zT4kDP0w$qkpkSN43~U+6l^}ciqIoU)v%P*-3mvReoK%t*a&$n$u;m3W;XDw~WqSlj zbH0fwREz>+r|nM(z5ZD^ z3@XhZ2bO#aS?Rk|&155|fNKL;$FG4^_C+O?++^V)^~JW5g0X;>3@H{93r}r+7L}FC z(;zlZIoKQ1dLIcUO`^dXVZ#ZPpJJiAaNa2uL?<>|G$Jydt!y3Jx46san44Ugw{6JZ zWV>=wG`gJR)Z|Lk?%Uq~6f&_yi&5y>!a_Z-4R{svP6%EMq+Ba8r~yM>ive7fDx_es z6=8&I$Gt=l{he%W@(35vpJugxdWp&sty2g{TKi}tRboU}+dFHFKKCu?@Si20i1g-nk5c@gW~7x*{PC0Yv+_)-l3 zc-n6tJD~n5f6oJCWU|VSctgEuPBNXngatA+UAa+(DQB_+IMDrWDdOl*7raXzg;wm| z&lcLp3lEZi-EjHuuPyWHcXhLDf;n`m+@GB`Pzoj_^ir$0x75Klh1;R);Fe4##gS0v zlkMrIoJwLCs0%Hkn#Ah8f2Z?Co)3DGS1kZuZCtUESdkY1b#QQIH%>u6fF~STtmP3U|DfTX{Ae zpUS?aZ8*P%+MY0Is10+=icjmzr5ccKCx!af^}U3*Lj}D@jH$` zw!1l5RmGlGncHn&YiKN9uWzX#mUdXACN)18592(-wHR;dCRb$G`@a}KAAbz- zG5b-kl!)eA8Qe(HXRsfMS1ogPuD6lVCMd58DFXhjSBKxf&N`m1|LaeK+3wb zmQ3iG=gK7^)A#s?G9F>y5_a{&WeMi=wgd8PRGH;|w%t4ox(*Jtvi?{x-yP%cpWTTN zG!Z~g;I$=bH0zvBYmvsu69#_p0I9{PntE1GHXt@0r%<(){t=h_S4o#9$>pguKZrCC5ac9<(yatj*%au=W zX{+kGVkH{qWkw~^D!?(;qJ(S8ZeLI2bbRYoz*!IPTxI3%k>4Qab=PEPcNB%Ta?ROV z7KGRC8sv!Cj-p@QL!9lj6N;?law#F-shHD}#n&Ix8&21x!VB)Wfh#iVu1}r2CC0*96dgI4Zi&ma)k{pDEQ4(%p8cza55#7P)mCTNUI z%N0*Vt77vOkEebEd6k=VY$Mk2BqOl~g^LqaI^1+Yj2V9@9SGOUy13zYtQYu&%?1AE zckDaKiC2B-l!n>rft3JyG8%ue>F!zh(al`}7+6i%)6Xza6oBCNa7gc49=xZFR(u9v z9tl&pau$6?>b}j&>W@?+Muj_F3vaN_aNQrXjoS)X_fv;}{I+RiW(;A+*o2Pm>JHlu zld2<|VgNw{tFOu(i3~oO0lTSzI2<&d5*U@42i@{yCfmPb%Es z8~u+s#qkO4?|PxfU`@_G3p%rXTlR5@RwnkTL%O*1 zl6`Cnk$K%m_qA9hR{vu}A{qwP?;_VzH_Hqch%?GEmJ@K&Vbf84i@|k)x#e^#0DCa* z(7qYZ8j=ZY=00o1)bzPfM=OW>lk}4f-sSTP+u98iM#=CLZn@wKWf4bL4*R{NLN%aW zq=>Ak-=^i-6GIVGjR2hKKEOS{CSitv@=g640c1=)vw-fmvS1LhXLdr+vK=K zc=qiZBk#xvJN;_wA516PXly`pR`0yQ9HhH&qvJNsS($4=miJq?SM}e!h-_zXLZVJA z`cgZ^pvV85RV$-Wi}1xBSJ0xvjIzi>O`_V%diB~gkB4vZckoo>Xy*=c-tUoTO(Y*~ zO;z{YbjN?BXl8^QR6?D|SKtEq*DVq5y9qPQ*OI0uWl)t~iw<;;enC1GkKZ!M4xYLR=(VT?{Ohg*;$dwn%_s!lv1bY-5tyr13d-N;4 z?_DGP$}HqsJ1D!;=40G$5wqUnkXIOxNE$ir2g>JUXa6img7FeRAs)8U3jMD!=v|TT&t{jLGKZKSP9Rn1W3#NRZ>X(A>gKkCz z7J;9BMlu&W)Z>;JwF6E4(@z~v!);+vITfz5?8Z&$wj)AQbNZv&cCTQJPkz$!IrC^-odp3pgf+ZoZw>tvpE7`guUB8@gZ;r52^WZTa24l4; zo(|rTwJ@iAzj<`b#S=@!687JJ(u=jkmxc%Qxk9J;y4ikvR~@!R_oX#3$wH5rUnZ`` z{Kf;@UElrX_!Z+PeDad~87-p#L5mJ8&diP*i3Y9F3{zEBD41~cq3A%mSoQ!5mDjG| zaE;nZQNr;?h~F%*g6FmNsT6DcpoxJlcSK_a4Ph3AnTtH0z0!t(-5dOz&xD=H;_r^+ z$+~4LPA2;td(FtbZ=4o%wr%E=k$q}nw#3e9U8q_8X0g8S-8(jfs-N?Azu4~z94bDC zk5-%~@%5+>)H(56L*f$HGk}Pz7ipHS1bi&8=t?#tN_8OeLHis3%R16>@F%Cs5alV%5$mC9V+576tz$i&JSK_fBF8?fq zPCw{s}( zq|>3q@0oA3+^%;?iE$SNo`k#N+#ow-CURl^6-Rq7xpAk3ca6hE2Ki$H#YagTZ06lC zilxI!^nWg{g>({jDbr)6c#83v`lIj3iaF5vg1TF%b&H<7+2;zt@BQH0kE980@lIw; zklZ2Ff%z0t^T^XGE}N`|F1X3=NTTc3trk6P;tDq>w=DNiz~`TJ)1I+qSC{X|bfO!tjSnATc&LVAb&Eu?&EZV-l7P{jV_qX_Q{|%1L%6+TbCFb>{ zk25i^7O-6oFRvY?r|AA%a=0NTNS~>0Z9dXh__qwP75Kn zqV&H%oSg;Y@bP{r4KfT59C^$=6={G@GqHbQ(_>kMXK>(&8?Ps6=gs`Qaf56}PFhOi z=joR8>{8nAiD#AvP}jZo_ob>%J#>(ZLf<$#3ECAHoTwTTeV3ry0q>F{5NIhDmuJY! zftI8{*RunW2Xabqv&bz=w>-C<)3k0YsJM9^8HcK#RgmzHTc?3ykSo~MtS2%#9wh~V z=(Xo`quW@WT7?T2>Wry0IX4T>6w|)*xW1IyM||OB3RgTc@4+{GD)Fbavte#7Cp)Hn z{##H;uhji+>c}2I*`I_l2qO!P$!9~QDXV)mi+oFkRFD-DZAhubciwM$&Tdm(770B< zh8f@FCjKLdU=pJo*sB9`lP}Dco*xoIe$Rc+kyBp=un)KG-YAQ;G0Y-mv`n3&_Y^(j zlU3=rDvv2`rOMO2)nu?g?csSIC2R`A!4H;<6=I4;;?p3s+duDTo@us<$$I?iz?o{k zel?MbT=NzLzgF7^pW#0`W$=9_qU>HYFZ^4xYPjQDjg4*yjyM_-uOhR-d~fBc2JbAB zoAbS(b}Q#apKz5~r#uw{2$g;rr3_^;4P1W?z13Qu4<06HM%o9KRB9&P3w2ZusV*n#$;YZ2PmIM)S56`4==NZtD#kc z={!9$q|85nup@_n-OK^qTyBS(nrRHP zIYH@5W<&h2a_sK z!0GL7SVbbj{j$yXD{Nq{4a7xEH4Sr9^)KE}J>k}r!lps z>~Gf@Fe7g+7o1`Rek$~?PvN`+sf8x7>A%S0@|8XXwG_z`*P@S|3GWtda1FI=+U3fV z=Qw8>CGaWSLFrdYoT|o=WN|j%;JVy+)A5fq8(-J0WZ9w%X|kD}b0uR@dfezoG`j{l+NKO*jC!IOZfB7H!Uz^eWtqxeEg{p?oualCQc?W;S>qJ4>34Hci9bMr_kGyFK1XC4ARnGL)WQ4@wo1*X zDx}0N&r9sd7>i4SfHDaaF-u}les7@HJgiphe9dJb)^s9sVDx{N2|(D6F5}i|@U&U! z{`eR7b+DnDOAU8y0WY7$oju-gi~Uj)={UK$Fkthwewx$&F#jn}tG8v}RgV2u{W0IE zzH@W-0GPcsu?E!a`r({FxFyrLlB-o)p=no>aXTDSYhMrtjzaKl?(uMqErKwVed;3) zjfu6N$kABK7BPc>`&+bIn`J_h;&X(y zG9Fael-RfR!bSuh+3-f!3b8HY_hYmq!Om~#pm5`_5W^7N_wweeya7-s5hLCJDK%_R zT8eHC@vm@4_H@iZo6qz_EYWVgqpNio;wGdPT=#14H!`Q zg?mULC98z39N$_W)18Nv&@$L87Kp#*#Qm&n!gm5D2|X58`GY0$c6z;MUg zbU`MY#a?vYIvT4Bo;KOs9e#zNnFn!>sTh)M$JPc+sng_4NNYs*?P+d{^&J<}qt?Xc zx?}V}cZfFUJSEQsq+>$F>i6Q`$y5Yi@XMLn2BkD#dr2<>6XDl!hioll!t@^kkqD(5 zx{ZFNWfj^eeuEUnZ)8mXeB(*uCOABNF!-9801BP#g>(wpb*t(xuFmIOR)D7ahUs}E z*}Yi98^G~?E2Ohf@B1v&tc&ruakp;F-v)P(pE*=TT(w zn#E0c+DZQBQDW*E+7?3+%4m?ul5EF!ZqWDb%~$8kk8IL zuBfz}RekHq&}A#yH@sko-QWo^k|6m)c`Y%jOZxVbKFo(p|F!6@v{IwY^&CoQbEB=a zgY*q}kM)qX_O)Bp_5Veuq&^yj&YV2diH86-rOK~Q?<+OQ17||oW#wAzkq*yu{oXX@ zy1aG*II8Gm>C&On`T;yD{QF6Rbr%8UB!ENZ7dWzau)T_RyaP}7R&1Kh5uUq$BRFQM zubim}BDq?yyTS5JIUfzQ3b4 zC|d4G*pNUV602Qs0z_LhP&kONZ)k$+l>xLcdgv-(`5TR;mhd6HWp4$}(l%whVS{(^ zrPrCqMvlwRr(Li$ikPFitq+@{S|aN&yC!asK7`t|!g|#ad4xb_&>VR@P_@OwbWQ|) zDt2abk7~<+iob*di0FKY`9$EE`1=MouqjHVfA3}!8Bzpn0+Nd4p3J;gq zlE!@VQ}C6wnzv+A2{4Vd?z469sW3`pdRg?cU79M-K9~fmD+MnLef`>C$Y1}1xKF_I zu7!tRQ?I5t&tq5MDbIF-BTN{pTZl8f;qUv{{%kHUC@?{J6+Uc5{RSOIG39J&7MgHO zvgssEyVJev3O!wzPlPaTT)p6^;Lw{evj;yFM3;d0l&91v=)Z9Fi~vhoF?~+kJ1}{J zw$5f(&5-`$UkHkpisN)R(}Atp+tP|QQ^yf>EX9eiL=h zrLS`34-(B3QOlyFC+d2u0Mn^%yGz|aPh{SeLol^5l0TB{b(`1tNNl>+vElYGxm$^m zx%YiaY39qWpR;MkI857r$egnqSZjAj+U6n0wdd#F$joj8t`!`r8YuL!-l=Kj9U`-T z{`{BoNPRO!VGR1M;0F_+DFW;s$0kJXu+#s&R4vkyf=cFB;w?WZnWT?A9ocZX$2K5f#)?ymYMy#5ox?m%jq@hKcYgJho#4{ za#~;C&i|IgYAC;&Vo0NSunBxT78#cWymW(~_4q3Umu<_FN;zIBTp`uW>2{T$W14>H z$cnZHkENNckblNCs4a=--sN8fbdKhV&Em(pTrl*T1XLF-K^wajrGHSm7iTdW*v_{x{Ek{&=5z*(UcC_C|pz|?1C219+$$zETE6n zkud=iQ*t3~7(t+1mMC5JwR&i#Z(ca!9=-A5Cdc%aYLi*IY1=aURF%wD`gUgjrTBTN z=;UeWSJK#Y&kvv4-xMc^Ssk#P_On4gBi}2UDz6beu>9BR9YU4aVGygnaKA# z6z5FJc(p%;HRc}wJ7;(Y`eo>7xwdwBtpjv*(WZC2yopJl-~*e@B@OeJ1*pW;o=0C+l$zGd+9#DKaw;woU*7h0I+m??o|C1X&oL?$#&`~3I_ zLbM^-qIl%KS_}4y7p{?n9c4aC2M@hbv5tf7IraW@@ul>fu#>u5gepPx$0dZOSJ<(b zo)qaP8k=7$62H&32hM^0Lr*&BN@c#3ecMcW5%bI>D{iFtP1AnuBgkzy&NIQ)FQAHS z6v2p?3tUaVnDmjpnj4e-NHvlLtVbo39#_%{5)jL z^bO>a5^Cxa4OLAb0rtKI77wdp(=LJTxd&B*1R;zKvEgW-Xv7tZ5Oav}Qp{;*r}Qcky{669)0T!DQx*f?XtT0S5dAJL8<`pld1i<=CO{CJ z`MIx%8!3)n1p~(=taxNXWOa)D?}SlHg|j^=!1n*4l6P}p0XgL4MFF+1zdEH zg2BmI%%Sj~R*+utOA9w~OM3`1F;&t+)NIL39cKb3@}Y8d=W^Wqj?!#b;m7F*lh%sA zAOiWLi!z0S{3Qi4U?Ycix-EnxPF`v1{hiXq-GaG0&~DmHj#eH2oeU$%e1Do7+=tYH zXd$>;l|_$(+Z-ztdOZ<|2F5-gc%(O1d(J;gU{*$?cddwZHsbYx!Y0W_$7@L1CA*ia zFwMJA0=J|6d=$|Lq-rk}h=19!yZ*ZS`iM+YyOy%yP#f-#pN)sJRo%n(hk8__0C zGh%liYyDTwMM@HcT!`?|)Bamaltgbuz04VnKJ)LF|M%VhuY0lxA#htIHXFC_y#c=D Mo-0Y0Jv9mWKO9Z|x&QzG From 6da65849f10783d96e4f03cd3fb15e7b4f86ffc3 Mon Sep 17 00:00:00 2001 From: Chong Gao Date: Wed, 20 Sep 2023 09:03:32 +0800 Subject: [PATCH 13/46] Test Parquet double column stat without NaN [databricks] (#9256) * Test Parquet double column stat without NaN Signed-off-by: Chong Gao Signed-off-by: Chong Gao --------- Signed-off-by: Chong Gao Co-authored-by: Chong Gao --- .../spark/rapids/ParquetScaleTestSuite.scala | 110 +++++++++++++++++- .../spark/rapids/ParquetScaleTestUtils.scala | 53 +++++++++ 2 files changed, 162 insertions(+), 1 deletion(-) create mode 100644 tests/src/test/scala/com/nvidia/spark/rapids/ParquetScaleTestUtils.scala diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScaleTestSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScaleTestSuite.scala index 987fff1fb31..e8849b35941 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScaleTestSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScaleTestSuite.scala @@ -205,7 +205,7 @@ class ParquetScaleTestSuite extends SparkQueryCompareTestSuite with Logging { "date", "timestamp") - test("Statistics tests for Parquet files written by GPU, float/double") { + test("Statistics tests for Parquet files written by GPU, float/double, with NaN") { assume(false, "Blocked by https://github.com/rapidsai/cudf/issues/13948") assume(false, "Move to scale test") @@ -236,6 +236,114 @@ class ParquetScaleTestSuite extends SparkQueryCompareTestSuite with Logging { // checkStats(tab) } + test("Statistics tests for Parquet files written by GPU, float/double, without NaN") { + assume(true, "Move to scale test") + val schemaStr = + """ + struct< + c01: float, + c02: double + > + """ + val gen = DBGen() + val tab = gen.addTable("tab", schemaStr, rowsNum) + tab("c01").setValueGen(NonNaNFloatGenFunc()) + tab("c02").setValueGen(NonNaNDoubleGenFunc()) + + def genDf(tab: TableGen): SparkSession => DataFrame = spark => tab.toDF(spark) + + withTempPath { testDataFile => + // Write test data to a file on CPU + writeScaleTestDataOnCpu(testDataFile, genDf(tab)) + + // write data and get stats on CPU + val cpuStats = withCpuSparkSession(getStats(testDataFile), sparkConf) + val cpuFileSize = testDataFile.listFiles(f => f.getName.endsWith(".parquet"))(0).length() + + // write data and get stats on GPU + val gpuStats = withGpuSparkSession(getStats(testDataFile), sparkConf) + val gpuFileSize = testDataFile.listFiles(f => f.getName.endsWith(".parquet"))(0).length() + + // compare schema + assertResult(cpuStats.schema)(gpuStats.schema) + + // Check the Gpu file size is not too large. + assert(gpuFileSize < 2 * cpuFileSize) + + /** + * + * CPU stat: + * + * ParquetStat(WrappedArray([c01] optional float c01, [c02] optional double c02), + * WrappedArray(RowGroupStat(1000000, + * WrappedArray( + * ColumnChunkStat(optional float c01, + * min = 0.0, max = 0.0, + * hasNonNullValue = false, + * isNumNullsSet = true, + * numNulls = 500532), + * ColumnChunkStat(optional double c02, + * min = 0.0, + * max = 0.0, + * hasNonNullValue = false, + * isNumNullsSet = true, + * numNulls = 498986))))) + * + * GPU stat: + * + * ParquetStat(WrappedArray([c01] optional float c01, [c02] optional double c02), + * WrappedArray(RowGroupStat(1000000, + * WrappedArray( + * ColumnChunkStat(optional float c01, + * min = -3.4026107E38, + * max = 3.4015179E38, + * hasNonNullValue = true, + * isNumNullsSet = true, + * numNulls = 500532), + * ColumnChunkStat(optional double c02, + * min = -1.7xxxxxE308, + * max = 1.7xxxxE308, + * hasNonNullValue = true, + * isNumNullsSet = true, + * numNulls = 498986))))) + * + * There are differences between CPU and GPU: + * CPU hasNonNullValue is false, CPU min/max is 0.0 + */ + assert(cpuStats.rowGroupStats.length == gpuStats.rowGroupStats.length) + assert(cpuStats.rowGroupStats(0).columnStats(0).isNumNullsSet == + gpuStats.rowGroupStats(0).columnStats(0).isNumNullsSet) + assert(cpuStats.rowGroupStats(0).columnStats(1).isNumNullsSet == + gpuStats.rowGroupStats(0).columnStats(1).isNumNullsSet) + assert(cpuStats.rowGroupStats(0).columnStats(0).numNulls == + gpuStats.rowGroupStats(0).columnStats(0).numNulls) + assert(cpuStats.rowGroupStats(0).columnStats(1).numNulls == + gpuStats.rowGroupStats(0).columnStats(1).numNulls) + + // write by GPU, read min/max by CPU + val (floatMin, floatMax, doubleMin, doubleMax) = withTempPath { gpuFile => + withGpuSparkSession(spark => { + // Read from the testing Parquet file and then write to a Parquet file + spark.read.parquet(testDataFile.getAbsolutePath).coalesce(1) + .write.mode("overwrite").parquet(gpuFile.getAbsolutePath) + }) + + val rowArray = withCpuSparkSession(spark => { + // Read from the testing Parquet file and then write to a Parquet file + spark.read.parquet(gpuFile.getAbsolutePath) + .selectExpr("min(c01)", "max(c01)", "min(c02)", "max(c02)").collect() + }) + + (rowArray(0)(0), rowArray(0)(1), rowArray(0)(2), rowArray(0)(3)) + } + + assertResult(floatMin)(gpuStats.rowGroupStats(0).columnStats(0).min) + assertResult(floatMax)(gpuStats.rowGroupStats(0).columnStats(0).max) + assertResult(doubleMin)(gpuStats.rowGroupStats(0).columnStats(1).min) + assertResult(doubleMax)(gpuStats.rowGroupStats(0).columnStats(1).max) + } + } + test("Statistics tests for Parquet files written by GPU, basic types") { assume(false, "Move to scale test") val schema = diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScaleTestUtils.scala b/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScaleTestUtils.scala new file mode 100644 index 00000000000..fa6f6bb5516 --- /dev/null +++ b/tests/src/test/scala/com/nvidia/spark/rapids/ParquetScaleTestUtils.scala @@ -0,0 +1,53 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. All rights reserved. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 com.nvidia.spark.rapids + +import org.apache.spark.sql.tests.datagen.{DataGen, GeneratorFunction, LocationToSeedMapping, RowLocation} + +case class NonNaNFloatGenFunc(mapping: LocationToSeedMapping = null) extends GeneratorFunction { + override def apply(rowLoc: RowLocation): Any = { + val v = java.lang.Float.intBitsToFloat(DataGen.getRandomFor(mapping(rowLoc)).nextInt()) + if (v.isNaN) { + 1.toFloat // just use 1.0 + } else { + v + } + } + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): GeneratorFunction = + NonNaNFloatGenFunc(mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalStateException("value ranges are not supported for Float yet") +} + +case class NonNaNDoubleGenFunc(mapping: LocationToSeedMapping = null) extends GeneratorFunction { + override def apply(rowLoc: RowLocation): Any = { + val v = java.lang.Double.longBitsToDouble(DataGen.nextLong(rowLoc, mapping)) + if (v.isNaN) { + 1.toDouble // just use 1.0 + } else { + v + } + } + + override def withLocationToSeedMapping(mapping: LocationToSeedMapping): GeneratorFunction = + NonNaNDoubleGenFunc(mapping) + + override def withValueRange(min: Any, max: Any): GeneratorFunction = + throw new IllegalStateException("value ranges are not supported for Double yet") +} From 46a6905b05a25a09cd9e43e314321b05f14c361a Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Wed, 20 Sep 2023 10:19:11 -0500 Subject: [PATCH 14/46] Add Delta Lake support for Spark 3.4.1 and Delta Lake tests on Spark 3.4.x (#9262) Signed-off-by: Jason Lowe --- aggregator/pom.xml | 2 +- jenkins/spark-tests.sh | 5 +++++ pom.xml | 2 +- 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/aggregator/pom.xml b/aggregator/pom.xml index a001b0695d5..12cf62e6954 100644 --- a/aggregator/pom.xml +++ b/aggregator/pom.xml @@ -534,7 +534,7 @@ com.nvidia - rapids-4-spark-delta-stub_${scala.binary.version} + rapids-4-spark-delta-24x_${scala.binary.version} ${project.version} ${spark.version.classifier} diff --git a/jenkins/spark-tests.sh b/jenkins/spark-tests.sh index a71fac503d3..4a062f63871 100755 --- a/jenkins/spark-tests.sh +++ b/jenkins/spark-tests.sh @@ -186,6 +186,7 @@ run_delta_lake_tests() { echo "run_delta_lake_tests SPARK_VER = $SPARK_VER" SPARK_32X_PATTERN="(3\.2\.[0-9])" SPARK_33X_PATTERN="(3\.3\.[0-9])" + SPARK_34X_PATTERN="(3\.4\.[0-9])" if [[ $SPARK_VER =~ $SPARK_32X_PATTERN ]]; then # There are multiple versions of deltalake that support SPARK 3.2.X @@ -197,6 +198,10 @@ run_delta_lake_tests() { DELTA_LAKE_VERSIONS="2.1.1 2.2.0" fi + if [[ $SPARK_VER =~ $SPARK_34X_PATTERN ]]; then + DELTA_LAKE_VERSIONS="2.4.0" + fi + if [ -z "$DELTA_LAKE_VERSIONS" ]; then echo "Skipping Delta Lake tests. $SPARK_VER" else diff --git a/pom.xml b/pom.xml index e672d2a9e17..584a2a7f98e 100644 --- a/pom.xml +++ b/pom.xml @@ -416,7 +416,7 @@ 2.0.6 - delta-lake/delta-stub + delta-lake/delta-24x From 948f9751a1562d3f41dcc2377b9a23fd2ff2ea6e Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 20 Sep 2023 09:24:46 -0600 Subject: [PATCH 15/46] Specify correct dependency versions for 350 build (#9266) Signed-off-by: Andy Grove --- pom.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 584a2a7f98e..43fca2e44ad 100644 --- a/pom.xml +++ b/pom.xml @@ -534,9 +534,10 @@ 350 ${spark350.version} ${spark350.version} - 1.12.2 + 1.13.1 ${spark350.sources} ${spark330.iceberg.version} + 2.0.7 delta-lake/delta-stub From eb9aeeaf37eb172a5bcb7aca357e0b06ef768773 Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Wed, 20 Sep 2023 13:33:18 -0500 Subject: [PATCH 16/46] Roll our own BufferedIterator so we can close cleanly (#9267) * Roll our own BufferedIterator so we can close cleanly Signed-off-by: Alessandro Bellina * Add checks in ScalableTaskCompletion to throw if we are mutating the task completion callbacks from a callback * Fix test that relied on .buffered --------- Signed-off-by: Alessandro Bellina --- .../rapids/CloseableBufferedIterator.scala | 44 ++++++++++--- .../rapids/GpuShuffledHashJoinExec.scala | 4 +- .../spark/rapids/ScalableTaskCompletion.scala | 18 +++++ .../GpuBroadcastHashJoinExecBase.scala | 2 +- .../execution/GpuBroadcastHashJoinExec.scala | 2 +- .../rapids/GpuShuffledHashJoinExecSuite.scala | 13 ++-- .../rapids/ScalableTaskCompletionSuite.scala | 65 +++++++++++++++++++ 7 files changed, 128 insertions(+), 20 deletions(-) create mode 100644 tests/src/test/scala/com/nvidia/spark/rapids/ScalableTaskCompletionSuite.scala diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/CloseableBufferedIterator.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/CloseableBufferedIterator.scala index b258d892e36..5bd51bd9d33 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/CloseableBufferedIterator.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/CloseableBufferedIterator.scala @@ -21,14 +21,15 @@ import com.nvidia.spark.rapids.ScalableTaskCompletion.onTaskCompletion import org.apache.spark.TaskContext /** - * Helper iterator that wraps a BufferedIterator of AutoCloseable subclasses. + * Helper iterator that wraps an Iterator of AutoCloseable subclasses. * This iterator also implements AutoCloseable, so it can be closed in case - * of exceptions. + * of exceptions and when close is called on it, its buffered item will be + * closed as well. * - * @param wrapped the buffered iterator + * @param wrapped the iterator we are wrapping for buffering * @tparam T an AutoCloseable subclass */ -class CloseableBufferedIterator[T <: AutoCloseable](wrapped: BufferedIterator[T]) +class CloseableBufferedIterator[T <: AutoCloseable](wrapped: Iterator[T]) extends BufferedIterator[T] with AutoCloseable { // Don't install the callback if in a unit test Option(TaskContext.get()).foreach { tc => @@ -38,13 +39,38 @@ class CloseableBufferedIterator[T <: AutoCloseable](wrapped: BufferedIterator[T] } private[this] var isClosed = false - override def head: T = wrapped.head - override def headOption: Option[T] = wrapped.headOption - override def next: T = wrapped.next - override def hasNext: Boolean = wrapped.hasNext + + private var hd: Option[T] = None + + def head: T = { + if (hd.isEmpty) { + hd = Some(next()) + } + hd.get + } + + override def headOption: Option[T] = { + if (hasNext) { + Some(head) + } else { + None + } + } + + override def next: T = if (hd.isDefined) { + val res = hd.get + hd = None + res + } else { + wrapped.next + } + + override def hasNext: Boolean = !isClosed && (hd.isDefined || wrapped.hasNext) + override def close(): Unit = { if (!isClosed) { - headOption.foreach(_.close()) + hd.foreach(_.close()) // close a buffered head item + hd = None isClosed = true } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala index 1222422393b..3d904c421b1 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExec.scala @@ -253,7 +253,7 @@ object GpuShuffledHashJoinExec extends Logging { (Either[ColumnarBatch, Iterator[ColumnarBatch]], Iterator[ColumnarBatch]) = { val buildTime = coalesceMetrics(GpuMetric.BUILD_TIME) val buildTypes = buildOutput.map(_.dataType).toArray - closeOnExcept(new CloseableBufferedIterator(buildIter.buffered)) { bufBuildIter => + closeOnExcept(new CloseableBufferedIterator(buildIter)) { bufBuildIter => val startTime = System.nanoTime() // Batches type detection val isBuildSerialized = bufBuildIter.hasNext && isBatchSerialized(bufBuildIter.head) @@ -401,7 +401,7 @@ object GpuShuffledHashJoinExec extends Logging { // will grab the semaphore when putting the first stream batch on the GPU, and // then we bring the build batch to the GPU and return. withResource(hostConcatResult) { _ => - closeOnExcept(new CloseableBufferedIterator(streamIter.buffered)) { bufStreamIter => + closeOnExcept(new CloseableBufferedIterator(streamIter)) { bufStreamIter => withResource(new NvtxRange("first stream batch", NvtxColor.RED)) { _ => if (bufStreamIter.hasNext) { bufStreamIter.head diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ScalableTaskCompletion.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ScalableTaskCompletion.scala index b3fd9fb2ece..695f511493c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ScalableTaskCompletion.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ScalableTaskCompletion.scala @@ -87,11 +87,25 @@ object ScalableTaskCompletion { private class TopLevelTaskCompletion extends Function[TaskContext, Unit] { private val pending = new util.HashSet[UserTaskCompletion]() private var callbacksDone = false + private var invokingACallback: Boolean = false + + private def throwIfInCallback(): Unit = { + if (invokingACallback) { + throw new IllegalStateException( + s"Detected a task completion callback attempting " + + "to add/remove callbacks. This is not supported.") + } + } private def callAllCallbacks(tc: TaskContext): Throwable = synchronized { + throwIfInCallback() var error: Throwable = null pending.forEach { utc => try { + // this is true while we invoke the callback + // so we can throw a bette error/stack trace + // instead of a ConcurrentModificationException + invokingACallback = true if (tc == null) { utc(utc.tc) } else { @@ -104,6 +118,8 @@ object ScalableTaskCompletion { } else { error.addSuppressed(t) } + } finally { + invokingACallback = false } } pending.clear() @@ -129,6 +145,7 @@ object ScalableTaskCompletion { } def add(u: UserTaskCompletion): Unit = synchronized { + throwIfInCallback() if (callbacksDone) { // Added a callback after it was done calling them back already u(u.tc) @@ -138,6 +155,7 @@ object ScalableTaskCompletion { } def remove(u: UserTaskCompletion): Unit = synchronized { + throwIfInCallback() pending.remove(u) } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExecBase.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExecBase.scala index 195374997fe..1e12f0c6733 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExecBase.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExecBase.scala @@ -142,7 +142,7 @@ abstract class GpuBroadcastHashJoinExecBase( streamIter: Iterator[ColumnarBatch], coalesceMetricsMap: Map[String, GpuMetric]): (ColumnarBatch, Iterator[ColumnarBatch]) = { - val bufferedStreamIter = new CloseableBufferedIterator(streamIter.buffered) + val bufferedStreamIter = new CloseableBufferedIterator(streamIter) closeOnExcept(bufferedStreamIter) { _ => withResource(new NvtxRange("first stream batch", NvtxColor.RED)) { _ => if (bufferedStreamIter.hasNext) { diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala index a00861b8762..ca297369328 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/execution/GpuBroadcastHashJoinExec.scala @@ -131,7 +131,7 @@ case class GpuBroadcastHashJoinExec( val targetSize = RapidsConf.GPU_BATCH_SIZE_BYTES.get(conf) val metricsMap = allMetrics - val bufferedStreamIter = new CloseableBufferedIterator(streamIter.buffered) + val bufferedStreamIter = new CloseableBufferedIterator(streamIter) closeOnExcept(bufferedStreamIter) { _ => withResource(new NvtxRange("first stream batch", NvtxColor.RED)) { _ => if (bufferedStreamIter.hasNext) { diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExecSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExecSuite.scala index a9c93039cbf..b4f58a74015 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExecSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuShuffledHashJoinExecSuite.scala @@ -55,9 +55,7 @@ class GpuShuffledHashJoinExecSuite extends AnyFunSuite with MockitoSugar { optimalCase: Boolean = false) (verifyBuiltData: Either[ColumnarBatch, Iterator[ColumnarBatch]] => Unit): Unit = { val mockStreamIter = mock[Iterator[ColumnarBatch]] - val mockBufferedStreamIterator = mock[BufferedIterator[ColumnarBatch]] - when(mockStreamIter.buffered).thenReturn(mockBufferedStreamIterator) - when(mockBufferedStreamIterator.hasNext).thenReturn(true) + when(mockStreamIter.hasNext).thenReturn(true) val (builtData, _) = GpuShuffledHashJoinExec.prepareBuildBatchesForJoin( buildIter, mockStreamIter, @@ -68,11 +66,12 @@ class GpuShuffledHashJoinExecSuite extends AnyFunSuite with MockitoSugar { verifyBuiltData(builtData) // build iterator should be drained assertResult(expected = false)(buildIter.hasNext) - verify(mockStreamIter, times(0)).hasNext if (optimalCase) { - verify(mockStreamIter, times(1)).buffered - verify(mockBufferedStreamIterator, times(1)).hasNext - verify(mockBufferedStreamIterator, times(1)).head + verify(mockStreamIter, times(1)).hasNext + verify(mockStreamIter, times(1)).next + } else { + verify(mockStreamIter, times(0)).hasNext + verify(mockStreamIter, times(0)).next } } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/ScalableTaskCompletionSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/ScalableTaskCompletionSuite.scala new file mode 100644 index 00000000000..cf1dc7bb085 --- /dev/null +++ b/tests/src/test/scala/com/nvidia/spark/rapids/ScalableTaskCompletionSuite.scala @@ -0,0 +1,65 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 com.nvidia.spark.rapids + +import org.scalatest.funsuite.AnyFunSuite +import org.scalatestplus.mockito.MockitoSugar.mock + +import org.apache.spark.TaskContext + +class ScalableTaskCompletionSuite extends AnyFunSuite { + test("ScalableTaskCompletion prevents registering callbacks from a running callback") { + val tc = mock[TaskContext] + var doFail = true + ScalableTaskCompletion.onTaskCompletion(tc, _ => { + if (doFail) { + ScalableTaskCompletion.onTaskCompletion(tc, _ => { + }) + } + }) + assertThrows[IllegalStateException] { + ScalableTaskCompletion.reset() + } + doFail = false + ScalableTaskCompletion.reset() + } + + test("ScalableTaskCompletion prevents calling callbacks from a running callback") { + val tc = mock[TaskContext] + var doFail = true + ScalableTaskCompletion.onTaskCompletion(tc, _ => { + if (doFail) { + ScalableTaskCompletion.reset() + } + }) + assertThrows[IllegalStateException] { + ScalableTaskCompletion.reset() + } + doFail = false + ScalableTaskCompletion.reset() + } + + test("ScalableTaskCompletion succeeds in the good case") { + val tc = mock[TaskContext] + var called = false + ScalableTaskCompletion.onTaskCompletion(tc, _ => { + called = true + }) + ScalableTaskCompletion.reset() + assert(called) + } +} From a90242e6b2b34a05e96a5629e9c4633d828a7014 Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Wed, 20 Sep 2023 16:25:22 -0500 Subject: [PATCH 17/46] Remove GDS spilling (#9270) * Remove GDS spilling Signed-off-by: Alessandro Bellina * Update advanced_configs --------- Signed-off-by: Alessandro Bellina --- .../advanced_configs.md | 2 - docs/additional-functionality/gds-spilling.md | 49 --- pom.xml | 7 - .../rapids/DeviceMemoryEventHandler.scala | 8 +- .../nvidia/spark/rapids/RapidsBuffer.scala | 1 - .../spark/rapids/RapidsBufferCatalog.scala | 27 +- .../spark/rapids/RapidsBufferStore.scala | 2 +- .../com/nvidia/spark/rapids/RapidsConf.scala | 23 -- .../nvidia/spark/rapids/RapidsGdsStore.scala | 292 ------------------ .../rapids/shuffle/BufferSendState.scala | 2 +- .../DeviceMemoryEventHandlerSuite.scala | 5 - .../spark/rapids/RapidsGdsStoreSuite.scala | 171 ---------- 12 files changed, 11 insertions(+), 578 deletions(-) delete mode 100644 docs/additional-functionality/gds-spilling.md delete mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsGdsStore.scala delete mode 100644 tests/src/test/scala/com/nvidia/spark/rapids/RapidsGdsStoreSuite.scala diff --git a/docs/additional-functionality/advanced_configs.md b/docs/additional-functionality/advanced_configs.md index bce7569f7bd..1b2216c59e7 100644 --- a/docs/additional-functionality/advanced_configs.md +++ b/docs/additional-functionality/advanced_configs.md @@ -36,8 +36,6 @@ Name | Description | Default Value | Applicable at spark.rapids.gpu.resourceName|The name of the Spark resource that represents a GPU that you want the plugin to use if using custom resources with Spark.|gpu|Startup spark.rapids.memory.gpu.allocFraction|The fraction of available (free) GPU memory that should be allocated for pooled memory. This must be less than or equal to the maximum limit configured via spark.rapids.memory.gpu.maxAllocFraction, and greater than or equal to the minimum limit configured via spark.rapids.memory.gpu.minAllocFraction.|1.0|Startup spark.rapids.memory.gpu.debug|Provides a log of GPU memory allocations and frees. If set to STDOUT or STDERR the logging will go there. Setting it to NONE disables logging. All other values are reserved for possible future expansion and in the mean time will disable logging.|NONE|Startup -spark.rapids.memory.gpu.direct.storage.spill.batchWriteBuffer.size|The size of the GPU memory buffer used to batch small buffers when spilling to GDS. Note that this buffer is mapped to the PCI Base Address Register (BAR) space, which may be very limited on some GPUs (e.g. the NVIDIA T4 only has 256 MiB), and it is also used by UCX bounce buffers.|8388608|Startup -spark.rapids.memory.gpu.direct.storage.spill.enabled|Should GPUDirect Storage (GDS) be used to spill GPU memory buffers directly to disk. GDS must be enabled and the directory `spark.local.dir` must support GDS. This is an experimental feature. For more information on GDS, see https://docs.nvidia.com/gpudirect-storage/.|false|Startup spark.rapids.memory.gpu.oomDumpDir|The path to a local directory where a heap dump will be created if the GPU encounters an unrecoverable out-of-memory (OOM) error. The filename will be of the form: "gpu-oom--.hprof" where is the process ID, and the dumpId is a sequence number to disambiguate multiple heap dumps per process lifecycle|None|Startup spark.rapids.memory.gpu.pool|Select the RMM pooling allocator to use. Valid values are "DEFAULT", "ARENA", "ASYNC", and "NONE". With "DEFAULT", the RMM pool allocator is used; with "ARENA", the RMM arena allocator is used; with "ASYNC", the new CUDA stream-ordered memory allocator in CUDA 11.2+ is used. If set to "NONE", pooling is disabled and RMM just passes through to CUDA memory allocation directly.|ASYNC|Startup spark.rapids.memory.gpu.pooling.enabled|Should RMM act as a pooling allocator for GPU memory, or should it just pass through to CUDA memory allocation directly. DEPRECATED: please use spark.rapids.memory.gpu.pool instead.|true|Startup diff --git a/docs/additional-functionality/gds-spilling.md b/docs/additional-functionality/gds-spilling.md deleted file mode 100644 index 8daf891ba6a..00000000000 --- a/docs/additional-functionality/gds-spilling.md +++ /dev/null @@ -1,49 +0,0 @@ ---- -layout: page -title: GPUDirect Storage (GDS) Spilling -parent: Additional Functionality -nav_order: 6 ---- -# GPUDirect Storage (GDS) Spilling ---- -**NOTE** - -_GPUDirect Storage (GDS) Spilling_ is a beta feature! -It requires to install CUDA 11.8 to use this feature. - ---- -The [RAPIDS Shuffle Manager](rapids-shuffle.md) has a spillable cache that keeps GPU data in device -memory, but can spill to host memory and then to disk when the GPU is out of memory. Using -[GPUDirect Storage (GDS)](https://docs.nvidia.com/gpudirect-storage/), device buffers can be spilled -directly to storage. This direct path increases system bandwidth, decreases latency and -utilization load on the CPU. - -### System Setup -In order to enable GDS spilling, GDS must be installed on the host. GDS software can be -downloaded [here](https://developer.nvidia.com/gpudirect-storage). Follow the -[GDS Installation and Troubleshooting Guide]( -https://docs.nvidia.com/gpudirect-storage/troubleshooting-guide/index.html) -to install and configure GDS. - -### Spark App Configuration -After GDS is installed on the host, to enable GDS spilling: -* Make sure the [RAPIDS Shuffle Manager](rapids-shuffle.md) is enabled and configured correctly. -* Make sure the Spark "scratch" directory configured by `spark.local.dir` supports GDS. -* Set `spark.rapids.memory.gpu.direct.storage.spill.enabled=true` in the Spark app. - -To verify that GDS spilling is working correctly, add the following line to -`${SPARK_HOME}/conf/log4j.properties`: -```properties -log4j.logger.com.nvidia.spark.rapids.RapidsGdsStore=DEBUG -``` -When spilling happens, the log file should show information for writing to and reading from GDS. - -### Fine-Tuning -Writing many small device buffers through GDS incurs overhead that may affect spilling performance. -To combat this issue, small device buffers are concatenated together before written to disk in a -batch. The batch write buffer used for this purpose takes up PCI Base Address Register (BAR) space, -which can be very limited on some GPUs. For example, the NVIDIA T4 only has 256 MiB. On GPUs with a -larger BAR space (e.g. the NVIDIA V100 or the NVIDIA A100), you can increase the size of the -batch write buffer, which may further improve spilling performance. To change the batch write buffer -size from the default 8 MiB to, say, 64 MiB, set -`spark.rapids.memory.gpu.direct.storage.spill.batchWriteBuffer.size=64m` in the Spark app. diff --git a/pom.xml b/pom.xml index 43fca2e44ad..c2ba5c4185e 100644 --- a/pom.xml +++ b/pom.xml @@ -618,13 +618,6 @@ ${spark.version.classifier}.com.nvidia.shaded.spark none package - - - GdsTest true UTF-8 diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandler.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandler.scala index 8a4b46626d4..2cfcc542315 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandler.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandler.scala @@ -31,7 +31,6 @@ import org.apache.spark.sql.rapids.execution.TrampolineUtil * RMM event handler to trigger spilling from the device memory store. * @param store device memory store that will be triggered to spill * @param oomDumpDir local directory to create heap dumps on GPU OOM - * @param isGdsSpillEnabled true if GDS is enabled for device->disk spill * @param maxFailedOOMRetries maximum number of retries for OOMs after * depleting the device store */ @@ -39,7 +38,6 @@ class DeviceMemoryEventHandler( catalog: RapidsBufferCatalog, store: RapidsDeviceMemoryStore, oomDumpDir: Option[String], - isGdsSpillEnabled: Boolean, maxFailedOOMRetries: Int) extends RmmEventHandler with Logging { // Flag that ensures we dump stack traces once and not for every allocation @@ -159,11 +157,7 @@ class DeviceMemoryEventHandler( catalog.synchronousSpill(store, targetSize, Cuda.DEFAULT_STREAM) maybeAmountSpilled.foreach { amountSpilled => logInfo(s"Spilled $amountSpilled bytes from the device store") - if (isGdsSpillEnabled) { - TrampolineUtil.incTaskMetricsDiskBytesSpilled(amountSpilled) - } else { - TrampolineUtil.incTaskMetricsMemoryBytesSpilled(amountSpilled) - } + TrampolineUtil.incTaskMetricsMemoryBytesSpilled(amountSpilled) } true } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala index 4bdf2493647..28adb9301f7 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala @@ -61,7 +61,6 @@ object StorageTier extends Enumeration { val DEVICE: StorageTier = Value(0, "device memory") val HOST: StorageTier = Value(1, "host memory") val DISK: StorageTier = Value(2, "local disk") - val GDS: StorageTier = Value(3, "GPUDirect Storage") } /** diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala index 5c58e73bf55..1b49374f9fb 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala @@ -702,7 +702,6 @@ object RapidsBufferCatalog extends Logging { private var hostStorage: RapidsHostMemoryStore = _ private var diskBlockManager: RapidsDiskBlockManager = _ private var diskStorage: RapidsDiskStore = _ - private var gdsStorage: RapidsGdsStore = _ private var memoryEventHandler: DeviceMemoryEventHandler = _ private var _shouldUnspill: Boolean = _ private var _singleton: RapidsBufferCatalog = null @@ -776,28 +775,22 @@ object RapidsBufferCatalog extends Logging { rapidsConf.chunkedPackBounceBufferSize, rapidsConf.spillToDiskBounceBufferSize) diskBlockManager = new RapidsDiskBlockManager(conf) - if (rapidsConf.isGdsSpillEnabled) { - gdsStorage = new RapidsGdsStore(diskBlockManager, rapidsConf.gdsSpillBatchWriteBufferSize) - deviceStorage.setSpillStore(gdsStorage) + val hostSpillStorageSize = if (rapidsConf.hostSpillStorageSize == -1) { + // + 1 GiB by default to match backwards compatibility + rapidsConf.pinnedPoolSize + (1024 * 1024 * 1024) } else { - val hostSpillStorageSize = if (rapidsConf.hostSpillStorageSize == -1) { - // + 1 GiB by default to match backwards compatibility - rapidsConf.pinnedPoolSize + (1024 * 1024 * 1024) - } else { - rapidsConf.hostSpillStorageSize - } - hostStorage = new RapidsHostMemoryStore(hostSpillStorageSize) - diskStorage = new RapidsDiskStore(diskBlockManager) - deviceStorage.setSpillStore(hostStorage) - hostStorage.setSpillStore(diskStorage) + rapidsConf.hostSpillStorageSize } + hostStorage = new RapidsHostMemoryStore(hostSpillStorageSize) + diskStorage = new RapidsDiskStore(diskBlockManager) + deviceStorage.setSpillStore(hostStorage) + hostStorage.setSpillStore(diskStorage) logInfo("Installing GPU memory handler for spill") memoryEventHandler = new DeviceMemoryEventHandler( singleton, deviceStorage, rapidsConf.gpuOomDumpDir, - rapidsConf.isGdsSpillEnabled, rapidsConf.gpuOomMaxRetries) if (rapidsConf.sparkRmmStateEnable) { @@ -853,10 +846,6 @@ object RapidsBufferCatalog extends Logging { diskStorage.close() diskStorage = null } - if (gdsStorage != null) { - gdsStorage.close() - gdsStorage = null - } } def getDeviceStorage: RapidsDeviceMemoryStore = deviceStorage diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala index 090d1fc5f65..a58a8415cbf 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferStore.scala @@ -597,7 +597,7 @@ abstract class RapidsBufferStore(val tier: StorageTier) /** * Buffers that inherit from this type do not support changing the spillable status - * of a `RapidsBuffer`. This is only used right now for disk and GDS. + * of a `RapidsBuffer`. This is only used right now for disk. * @param tier storage tier of this store */ abstract class RapidsBufferStoreWithoutSpill(override val tier: StorageTier) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index f5c60dedd5b..adddeecdc61 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -477,25 +477,6 @@ object RapidsConf { .booleanConf .createWithDefault(false) - val GDS_SPILL = conf("spark.rapids.memory.gpu.direct.storage.spill.enabled") - .doc("Should GPUDirect Storage (GDS) be used to spill GPU memory buffers directly to disk. " + - "GDS must be enabled and the directory `spark.local.dir` must support GDS. This is an " + - "experimental feature. For more information on GDS, see " + - "https://docs.nvidia.com/gpudirect-storage/.") - .startupOnly() - .booleanConf - .createWithDefault(false) - - val GDS_SPILL_BATCH_WRITE_BUFFER_SIZE = - conf("spark.rapids.memory.gpu.direct.storage.spill.batchWriteBuffer.size") - .doc("The size of the GPU memory buffer used to batch small buffers when spilling to GDS. " + - "Note that this buffer is mapped to the PCI Base Address Register (BAR) space, which may " + - "be very limited on some GPUs (e.g. the NVIDIA T4 only has 256 MiB), and it is also used " + - "by UCX bounce buffers.") - .startupOnly() - .bytesConf(ByteUnit.BYTE) - .createWithDefault(ByteUnit.MiB.toBytes(8)) - val POOLED_MEM = conf("spark.rapids.memory.gpu.pooling.enabled") .doc("Should RMM act as a pooling allocator for GPU memory, or should it just pass " + "through to CUDA memory allocation directly. DEPRECATED: please use " + @@ -2289,10 +2270,6 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isUnspillEnabled: Boolean = get(UNSPILL) - lazy val isGdsSpillEnabled: Boolean = get(GDS_SPILL) - - lazy val gdsSpillBatchWriteBufferSize: Long = get(GDS_SPILL_BATCH_WRITE_BUFFER_SIZE) - lazy val needDecimalGuarantees: Boolean = get(NEED_DECIMAL_OVERFLOW_GUARANTEES) lazy val gpuTargetBatchSizeBytes: Long = get(GPU_BATCH_SIZE_BYTES) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsGdsStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsGdsStore.scala deleted file mode 100644 index 529f3fece63..00000000000 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsGdsStore.scala +++ /dev/null @@ -1,292 +0,0 @@ -/* - * Copyright (c) 2020-2023, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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 com.nvidia.spark.rapids - -import java.io.File -import java.util.concurrent.ConcurrentHashMap -import java.util.function.BiFunction - -import scala.collection.mutable.ArrayBuffer - -import ai.rapids.cudf._ -import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} -import com.nvidia.spark.rapids.RapidsPluginImplicits._ -import com.nvidia.spark.rapids.StorageTier.StorageTier -import com.nvidia.spark.rapids.format.TableMeta - -import org.apache.spark.sql.rapids.{RapidsDiskBlockManager, TempSpillBufferId} - -/** A buffer store using GPUDirect Storage (GDS). */ -class RapidsGdsStore( - diskBlockManager: RapidsDiskBlockManager, - batchWriteBufferSize: Long) - extends RapidsBufferStoreWithoutSpill(StorageTier.GDS) { - private[this] val batchSpiller = new BatchSpiller() - - override protected def createBuffer( - other: RapidsBuffer, - catalog: RapidsBufferCatalog, - stream: Cuda.Stream): Option[RapidsBufferBase] = { - // assume that we get 1 buffer - val otherBuffer = withResource(other.getCopyIterator) { it => - it.next() - } - - val buff = withResource(otherBuffer) { _ => - val deviceBuffer = otherBuffer match { - case d: BaseDeviceMemoryBuffer => d - case _ => throw new IllegalStateException("copying from buffer without device memory") - } - if (deviceBuffer.getLength < batchWriteBufferSize) { - batchSpiller.spill(other, deviceBuffer) - } else { - singleShotSpill(other, deviceBuffer) - } - } - Some(buff) - } - - override def close(): Unit = { - super.close() - batchSpiller.close() - } - - abstract class RapidsGdsBuffer( - override val id: RapidsBufferId, - val size: Long, - override val meta: TableMeta, - spillPriority: Long) - extends RapidsBufferBase(id, meta, spillPriority) { - override val storageTier: StorageTier = StorageTier.GDS - - override val memoryUsedBytes: Long = size - - override def getMemoryBuffer: MemoryBuffer = getDeviceMemoryBuffer - } - - class RapidsGdsSingleShotBuffer( - id: RapidsBufferId, path: File, fileOffset: Long, size: Long, meta: TableMeta, - spillPriority: Long) - extends RapidsGdsBuffer(id, size, meta, spillPriority) { - - override def materializeMemoryBuffer: MemoryBuffer = { - closeOnExcept(DeviceMemoryBuffer.allocate(size)) { buffer => - CuFile.readFileToDeviceBuffer(buffer, path, fileOffset) - logDebug(s"Created device buffer for $path $fileOffset:$size via GDS") - buffer - } - } - - override def copyToMemoryBuffer(srcOffset: Long, dst: MemoryBuffer, dstOffset: Long, - length: Long, stream: Cuda.Stream): Unit = { - dst match { - case dmOriginal: BaseDeviceMemoryBuffer => - val sliced = dmOriginal.slice(dstOffset, length).asInstanceOf[BaseDeviceMemoryBuffer] - withResource(sliced) { dm => - // TODO: switch to async API when it's released, using the passed in CUDA stream. - stream.sync() - CuFile.readFileToDeviceBuffer(dm, path, fileOffset + srcOffset) - logDebug(s"Created device buffer for $path ${fileOffset + srcOffset}:$length via GDS") - } - case _ => throw new IllegalStateException( - s"GDS can only copy to device buffer, not ${dst.getClass}") - } - } - - override protected def releaseResources(): Unit = { - if (id.canShareDiskPaths) { - // Buffers that share paths must be cleaned up elsewhere - } else { - if (!path.delete() && path.exists()) { - logWarning(s"Unable to delete GDS spill path $path") - } - } - } - } - - private def singleShotSpill(other: RapidsBuffer, deviceBuffer: BaseDeviceMemoryBuffer) - : RapidsBufferBase = { - val id = other.id - val path = id.getDiskPath(diskBlockManager) - // When sharing files, append to the file; otherwise, write from the beginning. - val fileOffset = if (id.canShareDiskPaths) { - // only one writer at a time for now when using shared files - path.synchronized { - CuFile.appendDeviceBufferToFile(path, deviceBuffer) - } - } else { - CuFile.writeDeviceBufferToFile(path, 0, deviceBuffer) - 0 - } - logDebug(s"Spilled to $path $fileOffset:${deviceBuffer.getLength} via GDS") - new RapidsGdsSingleShotBuffer( - id, - path, - fileOffset, - deviceBuffer.getLength, - other.meta, - other.getSpillPriority) - } - - class BatchSpiller() extends AutoCloseable { - private val blockSize = 4096 - private[this] val spilledBuffers = new ConcurrentHashMap[File, Set[RapidsBufferId]] - private[this] val pendingBuffers = ArrayBuffer.empty[RapidsGdsBatchedBuffer] - private[this] val batchWriteBuffer = CuFileBuffer.allocate(batchWriteBufferSize, true) - private[this] var currentFile = TempSpillBufferId().getDiskPath(diskBlockManager) - private[this] var currentOffset = 0L - - override def close(): Unit = { - pendingBuffers.safeFree() - pendingBuffers.clear() - batchWriteBuffer.close() - } - - def spill(other: RapidsBuffer, deviceBuffer: BaseDeviceMemoryBuffer): RapidsBufferBase = - this.synchronized { - if (deviceBuffer.getLength > batchWriteBufferSize - currentOffset) { - val path = currentFile.getAbsolutePath - withResource(new CuFileWriteHandle(path)) { handle => - handle.write(batchWriteBuffer, batchWriteBufferSize, 0) - logDebug(s"Spilled to $path 0:$currentOffset via GDS") - } - pendingBuffers.foreach(_.unsetPending()) - pendingBuffers.clear - currentFile = TempSpillBufferId().getDiskPath(diskBlockManager) - currentOffset = 0 - } - - batchWriteBuffer.copyFromMemoryBuffer( - currentOffset, deviceBuffer, 0, deviceBuffer.getLength, Cuda.DEFAULT_STREAM) - - val id = other.id - addBuffer(currentFile, id) - val gdsBuffer = new RapidsGdsBatchedBuffer( - id, - currentFile, - currentOffset, - deviceBuffer.getLength, - other.meta, - other.getSpillPriority) - currentOffset += alignUp(deviceBuffer.getLength) - pendingBuffers += gdsBuffer - gdsBuffer - } - - private def alignUp(length: Long): Long = { - (length + blockSize - 1) & ~(blockSize - 1) - } - - private def copyToBuffer( - buffer: MemoryBuffer, offset: Long, size: Long, stream: Cuda.Stream): Unit = { - buffer.copyFromMemoryBuffer(0, batchWriteBuffer, offset, size, stream) - } - - private def addBuffer(path: File, id: RapidsBufferId): Set[RapidsBufferId] = { - val updater = new BiFunction[File, Set[RapidsBufferId], Set[RapidsBufferId]] { - override def apply(key: File, value: Set[RapidsBufferId]): Set[RapidsBufferId] = { - if (value == null) { - Set(id) - } else { - value + id - } - } - } - spilledBuffers.compute(path, updater) - } - - private def removeBuffer(path: File, id: RapidsBufferId): Set[RapidsBufferId] = { - val updater = new BiFunction[File, Set[RapidsBufferId], Set[RapidsBufferId]] { - override def apply(key: File, value: Set[RapidsBufferId]): Set[RapidsBufferId] = { - val newValue = value - id - if (newValue.isEmpty) { - null - } else { - newValue - } - } - } - spilledBuffers.computeIfPresent(path, updater) - } - - class RapidsGdsBatchedBuffer( - id: RapidsBufferId, - path: File, - fileOffset: Long, - size: Long, - meta: TableMeta, - spillPriority: Long, - var isPending: Boolean = true) - extends RapidsGdsBuffer(id, size, meta, spillPriority) { - - override val memoryUsedBytes: Long = size - - override def materializeMemoryBuffer: MemoryBuffer = this.synchronized { - closeOnExcept(DeviceMemoryBuffer.allocate(size)) { buffer => - if (isPending) { - copyToBuffer(buffer, fileOffset, size, Cuda.DEFAULT_STREAM) - Cuda.DEFAULT_STREAM.sync() - logDebug(s"Created device buffer $size from batch write buffer") - } else { - CuFile.readFileToDeviceBuffer(buffer, path, fileOffset) - logDebug(s"Created device buffer for $path $fileOffset:$size via GDS") - } - buffer - } - } - - override def copyToMemoryBuffer(srcOffset: Long, dst: MemoryBuffer, dstOffset: Long, - length: Long, stream: Cuda.Stream): Unit = this.synchronized { - dst match { - case dmOriginal: BaseDeviceMemoryBuffer => - val sliced = dmOriginal.slice(dstOffset, length).asInstanceOf[BaseDeviceMemoryBuffer] - withResource(sliced) { dm => - if (isPending) { - copyToBuffer(dm, fileOffset + srcOffset, length, stream) - stream.sync() - logDebug(s"Created device buffer $length from batch write buffer") - } else { - // TODO: switch to async API when it's released, using the passed in CUDA stream. - stream.sync() - CuFile.readFileToDeviceBuffer(dm, path, fileOffset + srcOffset) - logDebug(s"Created device buffer for $path ${fileOffset + srcOffset}:$length " + - s"via GDS") - } - } - case _ => throw new IllegalStateException( - s"GDS can only copy to device buffer, not ${dst.getClass}") - } - } - - /** - * Mark this buffer as disk based, no longer in device memory. - */ - def unsetPending(): Unit = this.synchronized { - isPending = false - } - - override protected def releaseResources(): Unit = { - val ids = removeBuffer(path, id) - if (ids == null) { - if (!path.delete() && path.exists()) { - logWarning(s"Unable to delete GDS spill path $path") - } - } - } - } - } -} \ No newline at end of file diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/BufferSendState.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/BufferSendState.scala index c1d65f16980..08a1ae22f5e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/BufferSendState.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/BufferSendState.scala @@ -179,7 +179,7 @@ class BufferSendState( requestHandler.acquireShuffleBuffer(bufferId)) { rapidsBuffer => //these are closed later, after we synchronize streams rapidsBuffer.storageTier match { - case StorageTier.DEVICE | StorageTier.GDS => + case StorageTier.DEVICE => deviceBuffs += blockRange.rangeSize() case _ => // host/disk hostBuffs += blockRange.rangeSize() diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandlerSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandlerSuite.scala index 35d7f990798..0b531adabb7 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandlerSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/DeviceMemoryEventHandlerSuite.scala @@ -31,7 +31,6 @@ class DeviceMemoryEventHandlerSuite extends RmmSparkRetrySuiteBase with MockitoS mockCatalog, mockStore, None, - false, 2) assertResult(true)(handler.onAllocFailure(1024, 0)) } @@ -45,7 +44,6 @@ class DeviceMemoryEventHandlerSuite extends RmmSparkRetrySuiteBase with MockitoS mockCatalog, mockStore, None, - false, 2) assertResult(true)(handler.onAllocFailure(1024, 0)) // sync assertResult(true)(handler.onAllocFailure(1024, 1)) // sync 2 @@ -61,7 +59,6 @@ class DeviceMemoryEventHandlerSuite extends RmmSparkRetrySuiteBase with MockitoS mockCatalog, mockStore, None, - false, 2) // with this call we sync, and we mark our attempts at 1, we store 0 as the last count assertResult(true)(handler.onAllocFailure(1024, 0)) @@ -80,7 +77,6 @@ class DeviceMemoryEventHandlerSuite extends RmmSparkRetrySuiteBase with MockitoS mockCatalog, mockStore, None, - false, 2) assertThrows[IllegalArgumentException](handler.onAllocFailure(-1, 0)) } @@ -94,7 +90,6 @@ class DeviceMemoryEventHandlerSuite extends RmmSparkRetrySuiteBase with MockitoS mockCatalog, mockStore, None, - false, 2) assertThrows[IllegalArgumentException](handler.onAllocFailure(1024, -1)) } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsGdsStoreSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsGdsStoreSuite.scala deleted file mode 100644 index 94fa440fd72..00000000000 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsGdsStoreSuite.scala +++ /dev/null @@ -1,171 +0,0 @@ -/* - * Copyright (c) 2021-2023, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT 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 com.nvidia.spark.rapids - -import java.io.File - -import ai.rapids.cudf.{ContiguousTable, CuFile, Table} -import com.nvidia.spark.rapids.Arm.withResource -import org.mockito.ArgumentMatchers -import org.mockito.ArgumentMatchers.any -import org.mockito.Mockito.{spy, times, verify, when} -import org.scalatest.Tag -import org.scalatest.compatible.Assertion -import org.scalatestplus.mockito.MockitoSugar - -import org.apache.spark.sql.rapids.RapidsDiskBlockManager -import org.apache.spark.storage.BlockId - -object GdsTest extends Tag("GdsTest") - -class RapidsGdsStoreSuite extends FunSuiteWithTempDir with MockitoSugar { - - test("single shot spill with shared path", GdsTest) { - println("Trying to load CuFile") - assume(CuFile.libraryLoaded()) - println("DID LOAD") - verifySingleShotSpill(canShareDiskPaths = true) - } - - test("single shot spill with exclusive path", GdsTest) { - assume(CuFile.libraryLoaded()) - verifySingleShotSpill(canShareDiskPaths = false) - } - - test("batch spill", GdsTest) { - assume(CuFile.libraryLoaded()) - - val bufferIds = Array(MockRapidsBufferId(7), MockRapidsBufferId(8), MockRapidsBufferId(9)) - val diskBlockManager = mock[RapidsDiskBlockManager] - val paths = Array( - new File(TEST_FILES_ROOT, s"gdsbuffer-0"), new File(TEST_FILES_ROOT, s"gdsbuffer-1")) - when(diskBlockManager.getFile(any[BlockId]())) - .thenReturn(paths(0)) - .thenReturn(paths(1)) - paths.foreach(f => assert(!f.exists)) - val spillPriority = -7 - val batchWriteBufferSize = 16384 // Holds 2 buffers. - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = spy(new RapidsBufferCatalog(devStore)) - withResource(new RapidsGdsStore( - diskBlockManager, batchWriteBufferSize)) { gdsStore => - - devStore.setSpillStore(gdsStore) - assertResult(0)(gdsStore.currentSize) - - val bufferSizes = new Array[Long](bufferIds.length) - val bufferHandles = new Array[RapidsBufferHandle](bufferIds.length) - - bufferIds.zipWithIndex.foreach { case(id, ix) => - val (size, handle) = addTableToCatalog(catalog, id, spillPriority) - catalog.synchronousSpill(devStore, 0) - bufferSizes(ix) = size - bufferHandles(ix) = handle - } - - val totalSize = bufferSizes.sum - assertResult(totalSize)(gdsStore.currentSize) - - assert(paths(0).exists) - assert(!paths(1).exists) - val alignedSize = Math.ceil((bufferSizes(0) + bufferSizes(1)) / 4096d).toLong * 4096 - assertResult(alignedSize)(paths(0).length) - - verify(catalog, times(6)).registerNewBuffer(ArgumentMatchers.any[RapidsBuffer]) - (bufferIds, bufferSizes, bufferHandles).zipped.foreach { (id, size, handle) => - verify(catalog).removeBufferTier( - ArgumentMatchers.eq(id), ArgumentMatchers.eq(StorageTier.DEVICE)) - withResource(catalog.acquireBuffer(handle)) { buffer => - assertResult(StorageTier.GDS)(buffer.storageTier) - assertResult(id)(buffer.id) - assertResult(size)(buffer.memoryUsedBytes) - assertResult(spillPriority)(buffer.getSpillPriority) - } - } - - bufferHandles(0).close() - assert(paths(0).exists) - bufferHandles(1).close() - assert(!paths(0).exists) - } - } - } - - private def verifySingleShotSpill(canShareDiskPaths: Boolean): Assertion = { - val bufferId = MockRapidsBufferId(7, canShareDiskPaths) - val path = bufferId.getDiskPath(null) - assert(!path.exists) - val spillPriority = -7 - withResource(new RapidsDeviceMemoryStore) { devStore => - val catalog = spy(new RapidsBufferCatalog(devStore)) - withResource(new RapidsGdsStore(mock[RapidsDiskBlockManager], 4096)) { - gdsStore => - devStore.setSpillStore(gdsStore) - assertResult(0)(gdsStore.currentSize) - val (bufferSize, handle) = addTableToCatalog(catalog, bufferId, spillPriority) - catalog.synchronousSpill(devStore, 0) - assertResult(bufferSize)(gdsStore.currentSize) - assert(path.exists) - assertResult(bufferSize)(path.length) - verify(catalog, times(2)).registerNewBuffer(ArgumentMatchers.any[RapidsBuffer]) - verify(catalog).removeBufferTier( - ArgumentMatchers.eq(bufferId), ArgumentMatchers.eq(StorageTier.DEVICE)) - withResource(catalog.acquireBuffer(handle)) { buffer => - assertResult(StorageTier.GDS)(buffer.storageTier) - assertResult(bufferSize)(buffer.memoryUsedBytes) - assertResult(bufferId)(buffer.id) - assertResult(spillPriority)(buffer.getSpillPriority) - } - - handle.close() - if (canShareDiskPaths) { - assert(path.exists()) - } else { - assert(!path.exists) - } - } - } - } - - private def addTableToCatalog( - catalog: RapidsBufferCatalog, - bufferId: RapidsBufferId, - spillPriority: Long): (Long, RapidsBufferHandle) = { - withResource(buildContiguousTable()) { ct => - val bufferSize = ct.getBuffer.getLength - // store takes ownership of the table - val handle = catalog.addContiguousTable(bufferId, ct, spillPriority, false) - (bufferSize, handle) - } - } - - /** Build a table of size 7808 bytes. */ - private def buildContiguousTable(): ContiguousTable = { - withResource(new Table.TestBuilder() - .column(Array.fill[String](512)("Lorem Ipsum")) - .build()) { table => - table.contiguousSplit()(0) - } - } - - case class MockRapidsBufferId( - tableId: Int, - override val canShareDiskPaths: Boolean = false) extends RapidsBufferId { - override def getDiskPath(diskBlockManager: RapidsDiskBlockManager): File = - new File(TEST_FILES_ROOT, s"gdsbuffer-$tableId") - } -} From 01fce6c909cae55bd1b8146cc63e9ec055316ad7 Mon Sep 17 00:00:00 2001 From: Peixin Date: Thu, 21 Sep 2023 09:11:10 +0800 Subject: [PATCH 18/46] Add arm64 profile to build arm artifacts (#9273) Signed-off-by: Peixin Li --- api_validation/pom.xml | 4 ++-- dist/pom.xml | 12 ++++++------ integration_tests/pom.xml | 2 +- pom.xml | 9 ++++++++- shuffle-plugin/pom.xml | 2 +- sql-plugin-api/pom.xml | 2 +- sql-plugin/pom.xml | 2 +- tests/pom.xml | 2 +- udf-compiler/pom.xml | 2 +- 9 files changed, 22 insertions(+), 15 deletions(-) diff --git a/api_validation/pom.xml b/api_validation/pom.xml index 892bd83ef24..cbd72317451 100644 --- a/api_validation/pom.xml +++ b/api_validation/pom.xml @@ -100,7 +100,7 @@ com.nvidia spark-rapids-jni - ${cuda.version} + ${jni.classifier} provided @@ -115,7 +115,7 @@ com.nvidia rapids-4-spark_${scala.binary.version} ${project.version} - ${cuda.version} + ${jni.classifier} provided diff --git a/dist/pom.xml b/dist/pom.xml index e40de9e7d33..171592ea691 100644 --- a/dist/pom.xml +++ b/dist/pom.xml @@ -40,7 +40,7 @@ - ${project.build.directory}/${project.build.finalName}-${cuda.version}.jar + ${project.build.directory}/${project.build.finalName}-${jni.classifier}.jar jar:file:${dist.jar.name}!/META-INF/maven/${project.groupId}/${project.artifactId}/pom.xml none @@ -223,7 +223,7 @@ org.apache.maven.plugins maven-jar-plugin - ${cuda.version} + ${jni.classifier} @@ -235,7 +235,7 @@ ${dist.jar.compress} ${project.build.directory}/parallel-world - ${cuda.version} + ${jni.classifier} @@ -412,7 +412,7 @@ self.log("... OK") com.nvidia spark-rapids-jni - ${cuda.version} + ${jni.classifier} META-INF/** ${project.build.directory}/parallel-world true @@ -447,7 +447,7 @@ self.log("... OK") ${dist.jar.name} ${project.artifactId} - ${cuda.version} + ${jni.classifier} ${project.groupId} ${project.version} jar @@ -477,7 +477,7 @@ self.log("... OK") ${dist.jar.name} file://${java.io.tmpdir}/m2-repo ${project.artifactId} - ${cuda.version} + ${jni.classifier} ${project.groupId} jar diff --git a/udf-compiler/pom.xml b/udf-compiler/pom.xml index 1ff68981472..64021094c7d 100644 --- a/udf-compiler/pom.xml +++ b/udf-compiler/pom.xml @@ -39,7 +39,7 @@ com.nvidia spark-rapids-jni - ${cuda.version} + ${jni.classifier} org.scala-lang From d6c7db293a0a8e473cbf7701e9187d1e17186117 Mon Sep 17 00:00:00 2001 From: Navin Kumar <97137715+NVnavkumar@users.noreply.github.com> Date: Thu, 21 Sep 2023 07:59:14 -0700 Subject: [PATCH 19/46] Enforce Scala method syntax over deprecated procedure syntax (#9253) * Add scalastyle checks for procedure syntax. * Update tests to remove procedure syntax * Update source files to remove procedure syntax Signed-off-by: Navin Kumar * Update to a version of scalastyle that supports Scala 2.13 Signed-off-by: Navin Kumar * 2023 Copyrights Signed-off-by: Navin Kumar --------- Signed-off-by: Navin Kumar --- .../delta20x/GpuOptimisticTransaction.scala | 2 +- .../delta21x/GpuOptimisticTransaction.scala | 2 +- .../delta22x/GpuOptimisticTransaction.scala | 2 +- .../delta24x/GpuOptimisticTransaction.scala | 2 +- .../rapids/GpuOptimisticTransaction.scala | 2 +- .../rapids/GpuOptimisticTransaction.scala | 2 +- .../rapids/GpuOptimisticTransaction.scala | 2 +- pom.xml | 4 +-- scalastyle-config.xml | 17 ++++++++++ .../nvidia/spark/rapids/GpuOverrides.scala | 2 +- .../spark/rapids/GpuRunnableCommandExec.scala | 2 +- .../nvidia/spark/rapids/RapidsBuffer.scala | 2 +- .../com/nvidia/spark/rapids/RegexParser.scala | 34 ++++++++++--------- .../nvidia/spark/rapids/SamplingUtils.scala | 2 +- .../com/nvidia/spark/rapids/TypeChecks.scala | 2 +- .../shuffle/RapidsShuffleTransport.scala | 4 +-- .../sql/rapids/RapidsCachingReader.scala | 2 +- .../shims/PythonMapInArrowExecShims.scala | 2 +- .../spark/rapids/AdaptiveQueryExecSuite.scala | 14 ++++---- .../rapids/ApproximatePercentileSuite.scala | 6 ++-- .../spark/rapids/GpuBatchUtilsSuite.scala | 2 +- .../rapids/GpuCoalesceBatchesSuite.scala | 2 +- .../spark/rapids/HashAggregatesSuite.scala | 4 +-- .../spark/rapids/ParseDateTimeSuite.scala | 4 +-- .../RegularExpressionTranspilerSuite.scala | 18 +++++----- .../spark/rapids/DynamicPruningSuite.scala | 6 ++-- 26 files changed, 81 insertions(+), 62 deletions(-) diff --git a/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala b/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala index ec7380d5344..e8f2a9df685 100644 --- a/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala @@ -71,7 +71,7 @@ class GpuOptimisticTransaction * @param deltaLog The Delta Log for the table this transaction is modifying. * @param rapidsConf RAPIDS Accelerator config settings */ - def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) { + def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) = { this(deltaLog, deltaLog.update(), rapidsConf) } diff --git a/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala b/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala index b2be6747e44..b2d9854789e 100644 --- a/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala @@ -71,7 +71,7 @@ class GpuOptimisticTransaction * @param deltaLog The Delta Log for the table this transaction is modifying. * @param rapidsConf RAPIDS Accelerator config settings */ - def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) { + def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) = { this(deltaLog, deltaLog.update(), rapidsConf) } diff --git a/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala b/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala index 3efdb384be5..178f35969cf 100644 --- a/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala @@ -71,7 +71,7 @@ class GpuOptimisticTransaction * @param deltaLog The Delta Log for the table this transaction is modifying. * @param rapidsConf RAPIDS Accelerator config settings */ - def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) { + def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) = { this(deltaLog, deltaLog.update(), rapidsConf) } diff --git a/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala b/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala index a089e72f462..b781235aba6 100644 --- a/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala @@ -71,7 +71,7 @@ class GpuOptimisticTransaction * @param deltaLog The Delta Log for the table this transaction is modifying. * @param rapidsConf RAPIDS Accelerator config settings */ - def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) { + def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) = { this(deltaLog, deltaLog.update(), rapidsConf) } diff --git a/delta-lake/delta-spark321db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala b/delta-lake/delta-spark321db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala index 07f2298af66..f585d3a3214 100644 --- a/delta-lake/delta-spark321db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-spark321db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala @@ -62,7 +62,7 @@ class GpuOptimisticTransaction( * @param deltaLog The Delta Log for the table this transaction is modifying. * @param rapidsConf RAPIDS Accelerator config settings */ - def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) { + def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) = { this(deltaLog, deltaLog.update(), rapidsConf) } diff --git a/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala b/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala index 9bfd3347f00..eb2a58251be 100644 --- a/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala @@ -72,7 +72,7 @@ class GpuOptimisticTransaction( * @param deltaLog The Delta Log for the table this transaction is modifying. * @param rapidsConf RAPIDS Accelerator config settings */ - def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) { + def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) = { this(deltaLog, deltaLog.update(), rapidsConf) } diff --git a/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala index b4f454ba920..e1693e11d09 100644 --- a/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala @@ -72,7 +72,7 @@ class GpuOptimisticTransaction( * @param deltaLog The Delta Log for the table this transaction is modifying. * @param rapidsConf RAPIDS Accelerator config settings */ - def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) { + def this(deltaLog: DeltaLog, rapidsConf: RapidsConf)(implicit clock: Clock) = { this(deltaLog, deltaLog.update(), rapidsConf) } diff --git a/pom.xml b/pom.xml index 1940ce948d9..541b06524cb 100644 --- a/pom.xml +++ b/pom.xml @@ -1322,9 +1322,9 @@ - org.scalastyle + com.beautiful-scala scalastyle_${scala.binary.version} - 1.0.0 + 1.5.1 diff --git a/scalastyle-config.xml b/scalastyle-config.xml index af30a096710..acea68d8996 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -104,6 +104,23 @@ You can also disable only one rule, by specifying its rule id, as specified in: Use Javadoc style indentation for multiline comments + + + + + + + + def this\((.*)\) \{ + false + + procedure syntax is deprecated for constructors in Scala 2.13: add `=`, as in method definition + + + + procedure syntax is deprecated in Scala 2.13: add return type `: Unit` and `=` + + diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 7c60b561393..8814b14c22b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -393,7 +393,7 @@ trait GpuOverridesListener { def optimizedPlan( plan: SparkPlanMeta[SparkPlan], sparkPlan: SparkPlan, - costOptimizations: Seq[Optimization]) + costOptimizations: Seq[Optimization]): Unit } sealed trait FileFormatType diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala index d41bc12e07c..e3869960fc4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuRunnableCommandExec.scala @@ -75,7 +75,7 @@ object GpuRunnableCommand { } } - def assertEmptyRootPath(tablePath: URI, saveMode: SaveMode, hadoopConf: Configuration) { + def assertEmptyRootPath(tablePath: URI, saveMode: SaveMode, hadoopConf: Configuration): Unit = { if (saveMode == SaveMode.ErrorIfExists && !getAllowNonEmptyLocationInCTAS) { val filePath = new org.apache.hadoop.fs.Path(tablePath) val fs = filePath.getFileSystem(hadoopConf) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala index 28adb9301f7..7e6b5c7ccb5 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala @@ -303,7 +303,7 @@ trait RapidsBuffer extends AutoCloseable { * @param stream CUDA stream to use */ def copyToMemoryBuffer( - srcOffset: Long, dst: MemoryBuffer, dstOffset: Long, length: Long, stream: Cuda.Stream) + srcOffset: Long, dst: MemoryBuffer, dstOffset: Long, length: Long, stream: Cuda.Stream): Unit /** * Get the device memory buffer from the underlying storage. If the buffer currently resides diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala index acaa90b0f12..5867c3648c1 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala @@ -1739,14 +1739,14 @@ sealed case class RegexSequence(parts: ListBuffer[RegexAST]) extends RegexAST { sealed case class RegexGroup(capture: Boolean, term: RegexAST, val lookahead: Option[RegexLookahead]) extends RegexAST { - def this(capture: Boolean, term: RegexAST) { + def this(capture: Boolean, term: RegexAST) = { this(capture, term, None) } - def this(capture: Boolean, term: RegexAST, position: Int) { + def this(capture: Boolean, term: RegexAST, position: Int) = { this(capture, term, None) this.position = Some(position) } - def this(capture: Boolean, term: RegexAST, position: Int, lookahead: Option[RegexLookahead]) { + def this(capture: Boolean, term: RegexAST, position: Int, lookahead: Option[RegexLookahead]) = { this(capture, term, lookahead) this.position = Some(position) } @@ -1765,7 +1765,7 @@ sealed case class RegexGroup(capture: Boolean, term: RegexAST, } sealed case class RegexChoice(a: RegexAST, b: RegexAST) extends RegexAST { - def this(a: RegexAST, b: RegexAST, position: Int) { + def this(a: RegexAST, b: RegexAST, position: Int) = { this(a, b) this.position = Some(position) } @@ -1774,7 +1774,7 @@ sealed case class RegexChoice(a: RegexAST, b: RegexAST) extends RegexAST { } sealed case class RegexRepetition(a: RegexAST, quantifier: RegexQuantifier) extends RegexAST { - def this(a: RegexAST, quantifier: RegexQuantifier, position: Int) { + def this(a: RegexAST, quantifier: RegexQuantifier, position: Int) = { this(a, quantifier) this.position = Some(position) } @@ -1785,7 +1785,7 @@ sealed case class RegexRepetition(a: RegexAST, quantifier: RegexQuantifier) exte sealed trait RegexQuantifier extends RegexAST sealed case class SimpleQuantifier(ch: Char) extends RegexQuantifier { - def this(ch: Char, position: Int) { + def this(ch: Char, position: Int) = { this(ch) this.position = Some(position) } @@ -1795,7 +1795,7 @@ sealed case class SimpleQuantifier(ch: Char) extends RegexQuantifier { sealed case class QuantifierFixedLength(length: Int) extends RegexQuantifier { - def this(length: Int, position: Int) { + def this(length: Int, position: Int) = { this(length) this.position = Some(position) } @@ -1807,7 +1807,7 @@ sealed case class QuantifierFixedLength(length: Int) sealed case class QuantifierVariableLength(minLength: Int, maxLength: Option[Int]) extends RegexQuantifier{ - def this(minLength: Int, maxLength: Option[Int], position: Int) { + def this(minLength: Int, maxLength: Option[Int], position: Int) = { this(minLength, maxLength) this.position = Some(position) } @@ -1825,7 +1825,7 @@ sealed case class QuantifierVariableLength(minLength: Int, maxLength: Option[Int sealed trait RegexCharacterClassComponent extends RegexAST sealed case class RegexHexDigit(a: String) extends RegexCharacterClassComponent { - def this(a: String, position: Int) { + def this(a: String, position: Int) = { this(a) this.position = Some(position) } @@ -1842,7 +1842,7 @@ sealed case class RegexHexDigit(a: String) extends RegexCharacterClassComponent } sealed case class RegexOctalChar(a: String) extends RegexCharacterClassComponent { - def this(a: String, position: Int) { + def this(a: String, position: Int) = { this(a) this.position = Some(position) } @@ -1853,7 +1853,7 @@ sealed case class RegexOctalChar(a: String) extends RegexCharacterClassComponent } sealed case class RegexChar(ch: Char) extends RegexCharacterClassComponent { - def this(ch: Char, position: Int) { + def this(ch: Char, position: Int) = { this(ch) this.position = Some(position) } @@ -1862,7 +1862,7 @@ sealed case class RegexChar(ch: Char) extends RegexCharacterClassComponent { } sealed case class RegexEscaped(a: Char) extends RegexCharacterClassComponent { - def this(a: Char, position: Int) { + def this(a: Char, position: Int) = { this(a) this.position = Some(position) } @@ -1873,7 +1873,9 @@ sealed case class RegexEscaped(a: Char) extends RegexCharacterClassComponent { sealed case class RegexCharacterRange(start: RegexCharacterClassComponent, end: RegexCharacterClassComponent) extends RegexCharacterClassComponent{ - def this(start: RegexCharacterClassComponent, end: RegexCharacterClassComponent, position: Int) { + def this(start: RegexCharacterClassComponent, + end: RegexCharacterClassComponent, + position: Int) = { this(start, end) this.position = Some(position) } @@ -1889,7 +1891,7 @@ sealed case class RegexCharacterClass( def this ( negated: Boolean, characters: ListBuffer[RegexCharacterClassComponent], - position: Int) { + position: Int) = { this(negated, characters) this.position = Some(position) } @@ -1946,7 +1948,7 @@ sealed case class RegexCharacterClass( } sealed case class RegexBackref(num: Int, isNew: Boolean = false) extends RegexAST { - def this(num: Int, isNew: Boolean, position: Int) { + def this(num: Int, isNew: Boolean, position: Int) = { this(num, isNew) this.position = Some(position) } @@ -1956,7 +1958,7 @@ sealed case class RegexBackref(num: Int, isNew: Boolean = false) extends RegexAS sealed case class RegexReplacement(parts: ListBuffer[RegexAST], var numCaptureGroups: Int = 0) extends RegexAST { - def this(parts: ListBuffer[RegexAST], numCaptureGroups: Int, position: Int) { + def this(parts: ListBuffer[RegexAST], numCaptureGroups: Int, position: Int) = { this(parts, numCaptureGroups) this.position = Some(position) } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SamplingUtils.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SamplingUtils.scala index ce3a5783414..4b4e6128ca0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SamplingUtils.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/SamplingUtils.scala @@ -271,7 +271,7 @@ private[spark] class XORShiftRandom(init: Long) extends JavaRandom(init) { (nextSeed & ((1L << bits) -1)).asInstanceOf[Int] } - override def setSeed(s: Long) { + override def setSeed(s: Long): Unit = { seed = XORShiftRandom.hashSeed(s) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index a50853eb509..049c0c5d230 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -2235,7 +2235,7 @@ object SupportedOpsForTools { } } - private def outputSupportIO() { + private def outputSupportIO(): Unit = { // Look at what we have for defaults for some configs because if the configs are off // it likely means something isn't completely compatible. val conf = new RapidsConf(Map.empty[String, String]) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTransport.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTransport.scala index 093903fec12..998f5675efe 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTransport.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/shuffle/RapidsShuffleTransport.scala @@ -32,7 +32,7 @@ trait TransactionCallback { } trait MemoryRegistrationCallback { - def apply(error: Option[Throwable] = None) + def apply(error: Option[Throwable] = None): Unit } /** @@ -344,7 +344,7 @@ trait RapidsShuffleTransport extends AutoCloseable { * (throttle) Adds a set of requests to be throttled as limits allowed. * @param reqs requests to add to the throttle queue */ - def queuePending(reqs: Seq[PendingTransferRequest]) + def queuePending(reqs: Seq[PendingTransferRequest]): Unit /** * Cancel requests that are waiting in the queue (not in-flight) for a specific diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala index 021aecf0067..67f56cfabc3 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/RapidsCachingReader.scala @@ -44,7 +44,7 @@ trait ShuffleMetricsUpdater { fetchWaitTimeInMs: Long, remoteBlocksFetched: Long, remoteBytesRead: Long, - rowsFetched: Long) + rowsFetched: Long): Unit } class RapidsCachingReader[K, C]( diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala index 5390f14122f..1151f8fd869 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/PythonMapInArrowExecShims.scala @@ -38,7 +38,7 @@ object PythonMapInArrowExecShims { ExecChecks((TypeSig.commonCudfTypes + TypeSig.ARRAY + TypeSig.STRUCT).nested(), TypeSig.all), (mapPy, conf, p, r) => new GpuPythonMapInArrowExecMeta(mapPy, conf, p, r) { - override def tagPlanForGpu() { + override def tagPlanForGpu(): Unit = { super.tagPlanForGpu() if (SQLConf.get.getConf(SQLConf.ARROW_EXECUTION_USE_LARGE_VAR_TYPES)) { diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala index 04442a4698c..85c5806b47e 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/AdaptiveQueryExecSuite.scala @@ -632,7 +632,7 @@ class AdaptiveQueryExecSuite numLocalReaders.length } - def skewJoinTest(fun: SparkSession => Unit) { + def skewJoinTest(fun: SparkSession => Unit): Unit = { val conf = new SparkConf() .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") @@ -699,7 +699,7 @@ class AdaptiveQueryExecSuite } /** Ported from org.apache.spark.sql.test.SQLTestData */ - private def testData(spark: SparkSession) { + private def testData(spark: SparkSession): Unit = { import spark.implicits._ val data: Seq[(Int, String)] = (1 to 100).map(i => (i, i.toString)) val df = data.toDF("key", "value") @@ -707,7 +707,7 @@ class AdaptiveQueryExecSuite registerAsParquetTable(spark, df, "testData") } /** Ported from org.apache.spark.sql.test.SQLTestData */ - private def testData2(spark: SparkSession) { + private def testData2(spark: SparkSession): Unit = { import spark.implicits._ val df = Seq[(Int, Int)]((1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2)) .toDF("a", "b") @@ -716,7 +716,7 @@ class AdaptiveQueryExecSuite } /** Ported from org.apache.spark.sql.test.SQLTestData */ - private def testData3(spark: SparkSession) { + private def testData3(spark: SparkSession): Unit = { import spark.implicits._ val df = Seq[(Int, Option[Int])]((1, None), (2, Some(2))) .toDF("a", "b") @@ -725,7 +725,7 @@ class AdaptiveQueryExecSuite } /** Ported from org.apache.spark.sql.test.SQLTestData */ - private def uncommonTypeTestData(spark: SparkSession) { + private def uncommonTypeTestData(spark: SparkSession): Unit = { import scala.collection.JavaConverters._ val df = spark.createDataFrame( List.tabulate(20)(i => Row(i % 3, BigDecimal(i), Array(i, i), Row(i))).asJava, @@ -740,7 +740,7 @@ class AdaptiveQueryExecSuite } /** Ported from org.apache.spark.sql.test.SQLTestData */ - private def lowerCaseData(spark: SparkSession) { + private def lowerCaseData(spark: SparkSession): Unit = { import spark.implicits._ // note that this differs from the original Spark test by generating a larger data set so that // we can trigger larger stats in the logical mode, preventing BHJ, and then our queries filter @@ -753,7 +753,7 @@ class AdaptiveQueryExecSuite registerAsParquetTable(spark, df, "lowercaseData") } - private def registerAsParquetTable(spark: SparkSession, df: Dataset[Row], name: String) { + private def registerAsParquetTable(spark: SparkSession, df: Dataset[Row], name: String): Unit = { val path = new File(TEST_FILES_ROOT, s"$name.parquet").getAbsolutePath df.write .mode(SaveMode.Overwrite) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/ApproximatePercentileSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/ApproximatePercentileSuite.scala index 8f3dc2305ce..7e0a4c5a3f2 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/ApproximatePercentileSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/ApproximatePercentileSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2021-2022, NVIDIA CORPORATION. + * Copyright (c) 2021-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -102,7 +102,7 @@ class ApproximatePercentileSuite extends SparkQueryCompareTestSuite { df.sparkSession.sql("SELECT approx_percentile(salary, array(0.5)) FROM salaries") } - def sqlFallbackTest(sql: String) { + def sqlFallbackTest(sql: String): Unit = { val conf = new SparkConf() .set("spark.rapids.sql.incompatibleOps.enabled", "true") @@ -126,7 +126,7 @@ class ApproximatePercentileSuite extends SparkQueryCompareTestSuite { private def doTest( func: SparkSession => DataFrame, percentileArg: Either[Double, Array[Double]] = Right(DEFAULT_PERCENTILES), - delta: Option[Int]) { + delta: Option[Int]): Unit = { val percentiles = withCpuSparkSession { spark => calcPercentiles(spark, func, percentileArg, delta, diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala index 99f0f406d91..5190f4434a4 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala @@ -158,7 +158,7 @@ class GpuBatchUtilsSuite extends AnyFunSuite { assert(GpuBatchUtils.estimateRowCount(200, 0, 1) == 1) } - private def compareEstimateWithActual(schema: StructType, rowCount: Int) { + private def compareEstimateWithActual(schema: StructType, rowCount: Int): Unit = { val rows = GpuBatchUtilsSuite.createRows(schema, rowCount) val estimate = GpuBatchUtils.estimateGpuMemory(schema, rows.length) val actual = calculateGpuMemory(schema, rows) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesSuite.scala index 16236810181..20919d171d4 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuCoalesceBatchesSuite.scala @@ -487,7 +487,7 @@ class GpuCoalesceBatchesSuite extends SparkQueryCompareTestSuite { }, conf) } - def testCompressedBatches(maxCompressedBatchMemoryLimit: Long) { + def testCompressedBatches(maxCompressedBatchMemoryLimit: Long): Unit = { val coalesceTargetBytes = 8000 val stop = 10000 var start = 0 diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala index b6cf36ac51c..0a2c6a654e9 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/HashAggregatesSuite.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2022, NVIDIA CORPORATION. + * Copyright (c) 2019-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -61,7 +61,7 @@ class HashAggregatesSuite extends SparkQueryCompareTestSuite { batchSize: Int = 0, repart: Int = 1, maxFloatDiff: Double = 0.0) - (fn: DataFrame => DataFrame) { + (fn: DataFrame => DataFrame): Unit = { if (batchSize > 0) { makeBatchedBytes(batchSize, conf) } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala index a1e7f16c17d..702928225a7 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/ParseDateTimeSuite.scala @@ -39,11 +39,11 @@ class ParseDateTimeSuite extends SparkQueryCompareTestSuite with BeforeAndAfterE .set(SQLConf.LEGACY_TIME_PARSER_POLICY.key, "LEGACY") .set(RapidsConf.INCOMPATIBLE_DATE_FORMATS.key, "true") - override def beforeEach() { + override def beforeEach(): Unit = { GpuOverrides.removeAllListeners() } - override def afterEach() { + override def afterEach(): Unit = { GpuOverrides.removeAllListeners() } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala index b98c1ef4482..79e2ab782a2 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala @@ -672,7 +672,7 @@ class RegularExpressionTranspilerSuite extends AnyFunSuite { doFuzzTest(None, RegexFindMode) } - private def doFuzzTest(validChars: Option[String], mode: RegexMode) { + private def doFuzzTest(validChars: Option[String], mode: RegexMode): Unit = { val r = new EnhancedRandom(new Random(seed = 0L), options = FuzzerOptions(validChars, maxStringLen = 12)) @@ -826,7 +826,7 @@ class RegularExpressionTranspilerSuite extends AnyFunSuite { doStringSplitTest(patterns, data, -1) } - def assertTranspileToSplittableString(patterns: Set[String]) { + def assertTranspileToSplittableString(patterns: Set[String]): Unit = { for (pattern <- patterns) { val transpiler = new CudfRegexTranspiler(RegexSplitMode) transpiler.transpileToSplittableString(pattern) match { @@ -839,7 +839,7 @@ class RegularExpressionTranspilerSuite extends AnyFunSuite { } } - def assertNoTranspileToSplittableString(patterns: Set[String]) { + def assertNoTranspileToSplittableString(patterns: Set[String]): Unit = { for (pattern <- patterns) { val transpiler = new CudfRegexTranspiler(RegexSplitMode) transpiler.transpileToSplittableString(pattern) match { @@ -853,7 +853,7 @@ class RegularExpressionTranspilerSuite extends AnyFunSuite { } } - def doStringSplitTest(patterns: Set[String], data: Seq[String], limit: Int) { + def doStringSplitTest(patterns: Set[String], data: Seq[String], limit: Int): Unit = { for (pattern <- patterns) { val cpu = cpuSplit(pattern, data, limit) val transpiler = new CudfRegexTranspiler(RegexSplitMode) @@ -883,7 +883,7 @@ class RegularExpressionTranspilerSuite extends AnyFunSuite { } private def doAstFuzzTest(validDataChars: Option[String], validPatternChars: String, - mode: RegexMode) { + mode: RegexMode): Unit = { val (data, patterns) = generateDataAndPatterns(validDataChars, validPatternChars, mode) if (mode == RegexReplaceMode) { assertCpuGpuMatchesRegexpReplace(patterns.toSeq, data) @@ -922,7 +922,7 @@ class RegularExpressionTranspilerSuite extends AnyFunSuite { (data, patterns.toSet) } - private def assertCpuGpuMatchesRegexpFind(javaPatterns: Seq[String], input: Seq[String]) = { + private def assertCpuGpuMatchesRegexpFind(javaPatterns: Seq[String], input: Seq[String]): Unit = { for ((javaPattern, patternIndex) <- javaPatterns.zipWithIndex) { val cpu = cpuContains(javaPattern, input) val (cudfPattern, _) = @@ -946,7 +946,7 @@ class RegularExpressionTranspilerSuite extends AnyFunSuite { private def assertCpuGpuMatchesRegexpReplace( javaPatterns: Seq[String], - input: Seq[String]) = { + input: Seq[String]): Unit = { for ((javaPattern, patternIndex) <- javaPatterns.zipWithIndex) { val cpu = cpuReplace(javaPattern, input) val (cudfPattern, replaceString) = @@ -1048,12 +1048,12 @@ class RegularExpressionTranspilerSuite extends AnyFunSuite { } } - private def doTranspileTest(pattern: String, expected: String) { + private def doTranspileTest(pattern: String, expected: String): Unit = { val transpiled: String = transpile(pattern, RegexFindMode) assert(toReadableString(transpiled) === toReadableString(expected)) } - private def doTranspileTest(pattern: String, expected: String, groupIdx: Int) { + private def doTranspileTest(pattern: String, expected: String, groupIdx: Int): Unit = { val transpiled: String = transpile(pattern, groupIdx) assert(toReadableString(transpiled) === toReadableString(expected)) } diff --git a/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala b/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala index f33c145c622..9a53648f80c 100644 --- a/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala +++ b/tests/src/test/spark321/scala/com/nvidia/spark/rapids/DynamicPruningSuite.scala @@ -190,7 +190,7 @@ class DynamicPruningSuite factData(spark) } - private def dimData(spark: SparkSession) { + private def dimData(spark: SparkSession): Unit = { val schema = StructType(Seq( StructField("key", DataTypes.IntegerType, false), StructField("skey", DataTypes.IntegerType, false), @@ -204,7 +204,7 @@ class DynamicPruningSuite )) registerAsParquetTable(spark, df, "dim", None) } - private def factData(spark: SparkSession) { + private def factData(spark: SparkSession): Unit = { val schema = StructType(Seq( StructField("key", DataTypes.IntegerType, false), StructField("skey", DataTypes.IntegerType, false), @@ -218,7 +218,7 @@ class DynamicPruningSuite registerAsParquetTable(spark, df, "fact", Some(List("key", "skey"))) } private def registerAsParquetTable(spark: SparkSession, df: Dataset[Row], name: String, - partitionBy: Option[Seq[String]]) { + partitionBy: Option[Seq[String]]): Unit = { val path = new File(TEST_FILES_ROOT, s"$name.parquet").getAbsolutePath partitionBy match { From 2a8518ee750f7343423e0059409f65e285993a59 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 21 Sep 2023 11:49:08 -0500 Subject: [PATCH 20/46] Have host spill use the new HostAlloc API (#9257) Signed-off-by: Robert (Bobby) Evans --- .../com/nvidia/spark/rapids/HostAlloc.scala | 2 +- .../nvidia/spark/rapids/RapidsBuffer.scala | 3 - .../spark/rapids/RapidsBufferCatalog.scala | 9 +- .../nvidia/spark/rapids/RapidsDiskStore.scala | 4 +- .../spark/rapids/RapidsHostMemoryStore.scala | 118 ++++++++---------- .../rapids/RapidsBufferCatalogSuite.scala | 2 +- .../spark/rapids/RapidsDiskStoreSuite.scala | 12 +- .../rapids/RapidsHostMemoryStoreSuite.scala | 101 +++++++-------- .../spark/rapids/RmmSparkRetrySuiteBase.scala | 2 +- 9 files changed, 114 insertions(+), 139 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostAlloc.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostAlloc.scala index 4e26e2c2f5e..587bffe7ebc 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostAlloc.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/HostAlloc.scala @@ -284,7 +284,7 @@ private class HostAlloc(nonPinnedLimit: Long) { private def canNeverSucceed(amount: Long, preferPinned: Boolean): Boolean = { val pinnedFailed = (isPinnedOnly || preferPinned) && (amount > pinnedLimit) val nonPinnedFailed = isPinnedOnly || (amount > nonPinnedLimit) - pinnedFailed && nonPinnedFailed + !isUnlimited && pinnedFailed && nonPinnedFailed } private def checkSize(amount: Long, preferPinned: Boolean): Unit = { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala index 7e6b5c7ccb5..9e7a0eb7a47 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBuffer.scala @@ -206,14 +206,11 @@ class RapidsBufferCopyIterator(buffer: RapidsBuffer) } override def close(): Unit = { - val hasNextBeforeClose = hasNext val toClose = new ArrayBuffer[AutoCloseable]() toClose.appendAll(chunkedPacker) toClose.appendAll(Option(singleShotBuffer)) toClose.safeClose() - require(!hasNextBeforeClose, - "RapidsBufferCopyIterator was closed before exhausting") } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala index 1b49374f9fb..61a636c1708 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsBufferCatalog.scala @@ -775,11 +775,14 @@ object RapidsBufferCatalog extends Logging { rapidsConf.chunkedPackBounceBufferSize, rapidsConf.spillToDiskBounceBufferSize) diskBlockManager = new RapidsDiskBlockManager(conf) - val hostSpillStorageSize = if (rapidsConf.hostSpillStorageSize == -1) { + val hostSpillStorageSize = if (rapidsConf.offHeapLimitEnabled) { + // Disable the limit because it is handled by the RapidsHostMemoryStore + None + } else if (rapidsConf.hostSpillStorageSize == -1) { // + 1 GiB by default to match backwards compatibility - rapidsConf.pinnedPoolSize + (1024 * 1024 * 1024) + Some(rapidsConf.pinnedPoolSize + (1024 * 1024 * 1024)) } else { - rapidsConf.hostSpillStorageSize + Some(rapidsConf.hostSpillStorageSize) } hostStorage = new RapidsHostMemoryStore(hostSpillStorageSize) diskStorage = new RapidsDiskStore(diskBlockManager) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDiskStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDiskStore.scala index d61f6061116..63f1b723ff1 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDiskStore.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsDiskStore.scala @@ -21,7 +21,7 @@ import java.nio.channels.FileChannel.MapMode import java.util.concurrent.ConcurrentHashMap import ai.rapids.cudf.{Cuda, HostMemoryBuffer, MemoryBuffer} -import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} +import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.StorageTier.StorageTier import com.nvidia.spark.rapids.format.TableMeta @@ -192,7 +192,7 @@ class RapidsDiskStore(diskBlockManager: RapidsDiskBlockManager) val path = id.getDiskPath(diskBlockManager) withResource(new FileInputStream(path)) { fis => val (header, hostBuffer) = SerializedHostTableUtils.readTableHeaderAndBuffer(fis) - val hostCols = closeOnExcept(hostBuffer) { _ => + val hostCols = withResource(hostBuffer) { _ => SerializedHostTableUtils.buildHostColumns(header, hostBuffer, sparkTypes) } new ColumnarBatch(hostCols.toArray, header.getNumRows) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsHostMemoryStore.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsHostMemoryStore.scala index dbdbb38f13c..05958a7e4b1 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsHostMemoryStore.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsHostMemoryStore.scala @@ -22,7 +22,7 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable -import ai.rapids.cudf.{Cuda, DeviceMemoryBuffer, HostColumnVector, HostMemoryBuffer, JCudfSerialization, MemoryBuffer, NvtxColor, NvtxRange, PinnedMemoryPool} +import ai.rapids.cudf.{Cuda, DeviceMemoryBuffer, HostColumnVector, HostMemoryBuffer, JCudfSerialization, MemoryBuffer, NvtxColor, NvtxRange} import com.nvidia.spark.rapids.Arm.{closeOnExcept, freeOnExcept, withResource} import com.nvidia.spark.rapids.SpillPriorities.{applyPriorityOffset, HOST_MEMORY_BUFFER_SPILL_OFFSET} import com.nvidia.spark.rapids.StorageTier.StorageTier @@ -36,29 +36,14 @@ import org.apache.spark.sql.vectorized.ColumnarBatch /** * A buffer store using host memory. * @param maxSize maximum size in bytes for all buffers in this store - * @param pageableMemoryPoolSize maximum size in bytes for the internal pageable memory pool */ class RapidsHostMemoryStore( - maxSize: Long) + maxSize: Option[Long]) extends RapidsBufferStore(StorageTier.HOST) { override protected def spillableOnAdd: Boolean = false - override def getMaxSize: Option[Long] = Some(maxSize) - - private def allocateHostBuffer( - size: Long, - preferPinned: Boolean = true): HostMemoryBuffer = { - var buffer: HostMemoryBuffer = null - if (preferPinned) { - buffer = PinnedMemoryPool.tryAllocate(size) - if (buffer != null) { - return buffer - } - } - - HostMemoryBuffer.allocate(size, false) - } + override def getMaxSize: Option[Long] = maxSize def addBuffer( id: RapidsBufferId, @@ -102,21 +87,23 @@ class RapidsHostMemoryStore( buffer: RapidsBuffer, catalog: RapidsBufferCatalog, stream: Cuda.Stream): Boolean = { - // this spillStore has a maximum size requirement (host only). We need to spill from it - // in order to make room for `buffer`. - val targetTotalSize = maxSize - buffer.memoryUsedBytes - if (targetTotalSize <= 0) { - // lets not spill to host when the buffer we are about - // to spill is larger than our limit - false - } else { - val amountSpilled = synchronousSpill(targetTotalSize, catalog, stream) - if (amountSpilled != 0) { - logDebug(s"Spilled $amountSpilled bytes from ${name} to make room for ${buffer.id}") - TrampolineUtil.incTaskMetricsDiskBytesSpilled(amountSpilled) + maxSize.forall { ms => + // this spillStore has a maximum size requirement (host only). We need to spill from it + // in order to make room for `buffer`. + val targetTotalSize = ms - buffer.memoryUsedBytes + if (targetTotalSize < 0) { + // lets not spill to host when the buffer we are about + // to spill is larger than our limit + false + } else { + val amountSpilled = synchronousSpill(targetTotalSize, catalog, stream) + if (amountSpilled != 0) { + logDebug(s"Spilled $amountSpilled bytes from ${name} to make room for ${buffer.id}") + TrampolineUtil.incTaskMetricsDiskBytesSpilled(amountSpilled) + } + // if after spill we can fit the new buffer, return true + buffer.memoryUsedBytes <= (ms - currentSize) } - // if after spill we can fit the new buffer, return true - buffer.memoryUsedBytes <= (maxSize - currentSize) } } @@ -125,53 +112,58 @@ class RapidsHostMemoryStore( catalog: RapidsBufferCatalog, stream: Cuda.Stream): Option[RapidsBufferBase] = { val wouldFit = trySpillToMaximumSize(other, catalog, stream) - // TODO: this is disabled for now since subsequent work will tie this into - // our host allocator apis. - if (false && !wouldFit) { + if (!wouldFit) { // skip host - logWarning(s"Buffer ${other} with size ${other.memoryUsedBytes} does not fit " + + logWarning(s"Buffer $other with size ${other.memoryUsedBytes} does not fit " + s"in the host store, skipping tier.") None } else { withResource(other.getCopyIterator) { otherBufferIterator => val isChunked = otherBufferIterator.isChunked val totalCopySize = otherBufferIterator.getTotalCopySize - closeOnExcept(allocateHostBuffer(totalCopySize)) { hostBuffer => - withResource(new NvtxRange("spill to host", NvtxColor.BLUE)) { _ => - var hostOffset = 0L - val start = System.nanoTime() - while (otherBufferIterator.hasNext) { - val otherBuffer = otherBufferIterator.next() - withResource(otherBuffer) { _ => - otherBuffer match { - case devBuffer: DeviceMemoryBuffer => - hostBuffer.copyFromMemoryBufferAsync( - hostOffset, devBuffer, 0, otherBuffer.getLength, stream) - hostOffset += otherBuffer.getLength - case _ => - throw new IllegalStateException("copying from buffer without device memory") + closeOnExcept(HostAlloc.allocHighPriority(totalCopySize)) { hb => + hb.map { hostBuffer => + withResource(new NvtxRange("spill to host", NvtxColor.BLUE)) { _ => + var hostOffset = 0L + val start = System.nanoTime() + while (otherBufferIterator.hasNext) { + val otherBuffer = otherBufferIterator.next() + withResource(otherBuffer) { _ => + otherBuffer match { + case devBuffer: DeviceMemoryBuffer => + hostBuffer.copyFromMemoryBufferAsync( + hostOffset, devBuffer, 0, otherBuffer.getLength, stream) + hostOffset += otherBuffer.getLength + case _ => + throw new IllegalStateException("copying from buffer without device memory") + } } } + stream.sync() + val end = System.nanoTime() + val szMB = (totalCopySize.toDouble / 1024.0 / 1024.0).toLong + val bw = (szMB.toDouble / ((end - start).toDouble / 1000000000.0)).toLong + logDebug(s"Spill to host (chunked=$isChunked) " + + s"size=$szMB MiB bandwidth=$bw MiB/sec") } - stream.sync() - val end = System.nanoTime() - val szMB = (totalCopySize.toDouble / 1024.0 / 1024.0).toLong - val bw = (szMB.toDouble / ((end - start).toDouble / 1000000000.0)).toLong - logDebug(s"Spill to host (chunked=$isChunked) " + - s"size=$szMB MiB bandwidth=$bw MiB/sec") + new RapidsHostMemoryBuffer( + other.id, + totalCopySize, + other.meta, + applyPriorityOffset(other.getSpillPriority, HOST_MEMORY_BUFFER_SPILL_OFFSET), + hostBuffer) + }.orElse { + // skip host + logWarning(s"Buffer $other with size ${other.memoryUsedBytes} does not fit " + + s"in the host store, skipping tier.") + None } - Some(new RapidsHostMemoryBuffer( - other.id, - totalCopySize, - other.meta, - applyPriorityOffset(other.getSpillPriority, HOST_MEMORY_BUFFER_SPILL_OFFSET), - hostBuffer)) } } } } - def numBytesFree: Long = maxSize - currentSize + def numBytesFree: Option[Long] = maxSize.map(_ - currentSize) class RapidsHostMemoryBuffer( id: RapidsBufferId, diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsBufferCatalogSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsBufferCatalogSuite.scala index 3666b85458e..61940ffd463 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsBufferCatalogSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsBufferCatalogSuite.scala @@ -215,7 +215,7 @@ class RapidsBufferCatalogSuite extends AnyFunSuite with MockitoSugar { withResource(spy(new RapidsDeviceMemoryStore)) { deviceStore => val mockStore = mock[RapidsBufferStore] withResource( - new RapidsHostMemoryStore(10000)) { hostStore => + new RapidsHostMemoryStore(Some(10000))) { hostStore => deviceStore.setSpillStore(hostStore) hostStore.setSpillStore(mockStore) val catalog = new RapidsBufferCatalog(deviceStore) diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDiskStoreSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDiskStoreSuite.scala index 6adcbcc1909..b911bccb286 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDiskStoreSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsDiskStoreSuite.scala @@ -62,7 +62,7 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = spy(new RapidsBufferCatalog(devStore)) - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => devStore.setSpillStore(hostStore) withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { diskStore => @@ -102,7 +102,7 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore) - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => devStore.setSpillStore(hostStore) withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { @@ -144,7 +144,7 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore) - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => devStore.setSpillStore(hostStore) withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { diskStore => @@ -288,7 +288,7 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore) - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => devStore.setSpillStore(hostStore) withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { diskStore => hostStore.setSpillStore(diskStore) @@ -340,7 +340,7 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { testBufferFileDeletion(canShareDiskPaths = true) } - class AlwaysFailingRapidsHostMemoryStore extends RapidsHostMemoryStore(0L){ + class AlwaysFailingRapidsHostMemoryStore extends RapidsHostMemoryStore(Some(0L)){ override def createBuffer( other: RapidsBuffer, catalog: RapidsBufferCatalog, @@ -357,7 +357,7 @@ class RapidsDiskStoreSuite extends FunSuiteWithTempDir with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore) - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => devStore.setSpillStore(hostStore) withResource(new RapidsDiskStore(mock[RapidsDiskBlockManager])) { diskStore => diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsHostMemoryStoreSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsHostMemoryStoreSuite.scala index 2d028f0cf7b..153b8da6556 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RapidsHostMemoryStoreSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RapidsHostMemoryStoreSuite.scala @@ -22,8 +22,7 @@ import java.math.RoundingMode import ai.rapids.cudf.{ContiguousTable, Cuda, HostColumnVector, HostMemoryBuffer, Table} import com.nvidia.spark.rapids.Arm._ import org.mockito.{ArgumentCaptor, ArgumentMatchers} -import org.mockito.ArgumentMatchers.any -import org.mockito.Mockito.{never, spy, times, verify, when} +import org.mockito.Mockito.{spy, times, verify} import org.scalatest.funsuite.AnyFunSuite import org.scalatestplus.mockito.MockitoSugar @@ -92,10 +91,10 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val mockStore = mock[RapidsHostMemoryStore] withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = spy(new RapidsBufferCatalog(devStore)) - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => assertResult(0)(hostStore.currentSize) - assertResult(hostStoreMaxSize)(hostStore.numBytesFree) + assertResult(hostStoreMaxSize)(hostStore.numBytesFree.get) devStore.setSpillStore(hostStore) hostStore.setSpillStore(mockStore) @@ -110,7 +109,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { catalog.synchronousSpill(devStore, 0) assertResult(bufferSize)(hostStore.currentSize) - assertResult(hostStoreMaxSize - bufferSize)(hostStore.numBytesFree) + assertResult(hostStoreMaxSize - bufferSize)(hostStore.numBytesFree.get) verify(catalog, times(2)).registerNewBuffer(ArgumentMatchers.any[RapidsBuffer]) verify(catalog).removeBufferTier( ArgumentMatchers.eq(handle.id), ArgumentMatchers.eq(StorageTier.DEVICE)) @@ -130,7 +129,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val mockStore = mock[RapidsHostMemoryStore] withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore) - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => devStore.setSpillStore(hostStore) hostStore.setSpillStore(mockStore) @@ -165,7 +164,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val mockStore = mock[RapidsHostMemoryStore] withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore) - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => devStore.setSpillStore(hostStore) hostStore.setSpillStore(mockStore) @@ -204,7 +203,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val (catalog, devStore, hostStore, diskStore) = closeOnExcept(new RapidsDiskStore(bm)) { diskStore => closeOnExcept(new RapidsDeviceMemoryStore()) { devStore => - closeOnExcept(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + closeOnExcept(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => devStore.setSpillStore(hostStore) hostStore.setSpillStore(diskStore) val catalog = closeOnExcept( @@ -257,7 +256,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val spillPriority = -10 val hostStoreMaxSize = 1L * 1024 * 1024 val mockStore = mock[RapidsDiskStore] - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore, hostStore) devStore.setSpillStore(hostStore) @@ -283,7 +282,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 val bm = new RapidsDiskBlockManager(new SparkConf()) withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore, hostStore) devStore.setSpillStore(hostStore) @@ -314,7 +313,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 val bm = new RapidsDiskBlockManager(new SparkConf()) withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore, hostStore) devStore.setSpillStore(hostStore) @@ -347,7 +346,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 val bm = new RapidsDiskBlockManager(new SparkConf()) withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore, hostStore) devStore.setSpillStore(hostStore) @@ -395,7 +394,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 val bm = new RapidsDiskBlockManager(new SparkConf()) withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore, hostStore) devStore.setSpillStore(hostStore) @@ -437,7 +436,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 val bm = new RapidsDiskBlockManager(new SparkConf()) withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore, hostStore) devStore.setSpillStore(hostStore) @@ -480,7 +479,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 val bm = new RapidsDiskBlockManager(new SparkConf()) withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore, hostStore) val hostBatch = buildHostBatch() @@ -514,7 +513,7 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val hostStoreMaxSize = 1L * 1024 * 1024 val bm = new RapidsDiskBlockManager(new SparkConf()) withResource(new RapidsDiskStore(bm)) { diskStore => - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore, hostStore) val hostBatch = buildHostBatchWithDuplicate() @@ -549,62 +548,46 @@ class RapidsHostMemoryStoreSuite extends AnyFunSuite with MockitoSugar { val hostStoreMaxSize = 256 withResource(new RapidsDeviceMemoryStore) { devStore => val catalog = new RapidsBufferCatalog(devStore) - val mockStore = mock[RapidsBufferStore] - val mockBuff = mock[mockStore.RapidsBufferBase] - when(mockBuff.id).thenReturn(new RapidsBufferId { - override val tableId: Int = 0 - override def getDiskPath(diskBlockManager: RapidsDiskBlockManager): File = null - }) - when(mockStore.getMaxSize).thenAnswer(_ => None) - when(mockStore.copyBuffer(any(), any(), any())).thenReturn(Some(mockBuff)) - when(mockStore.tier) thenReturn (StorageTier.DISK) - withResource(new RapidsHostMemoryStore(hostStoreMaxSize)) { hostStore => + val spyStore = spy(new RapidsDiskStore(new RapidsDiskBlockManager(new SparkConf()))) + withResource(new RapidsHostMemoryStore(Some(hostStoreMaxSize))) { hostStore => devStore.setSpillStore(hostStore) - hostStore.setSpillStore(mockStore) + hostStore.setSpillStore(spyStore) var bigHandle: RapidsBufferHandle = null var bigTable = buildContiguousTable(1024 * 1024) - var smallTable = buildContiguousTable(1) closeOnExcept(bigTable) { _ => - closeOnExcept(smallTable) { _ => - // make a copy of the table so we can compare it later to the - // one reconstituted after the spill - val expectedBatch = - withResource(bigTable.getTable.contiguousSplit()) { expectedTable => - GpuColumnVector.from(expectedTable(0).getTable, sparkTypes) - } - withResource(expectedBatch) { _ => - bigHandle = withResource(bigTable) { _ => - catalog.addContiguousTable( - bigTable, - spillPriority) - } // close the bigTable so it can be spilled - bigTable = null - catalog.synchronousSpill(devStore, 0) - verify(mockStore, never()).copyBuffer( - ArgumentMatchers.any[RapidsBuffer], - ArgumentMatchers.any[RapidsBufferCatalog], - ArgumentMatchers.any[Cuda.Stream]) - withResource(catalog.acquireBuffer(bigHandle)) { buffer => - assertResult(StorageTier.HOST)(buffer.storageTier) - withResource(buffer.getColumnarBatch(sparkTypes)) { actualBatch => - TestUtils.compareBatches(expectedBatch, actualBatch) - } - } + // make a copy of the table so we can compare it later to the + // one reconstituted after the spill + val expectedBatch = + withResource(bigTable.getTable.contiguousSplit()) { expectedTable => + GpuColumnVector.from(expectedTable(0).getTable, sparkTypes) } - withResource(smallTable) { _ => + withResource(expectedBatch) { _ => + bigHandle = withResource(bigTable) { _ => catalog.addContiguousTable( - smallTable, spillPriority, - false) - } // close the smallTable so it can be spilled - smallTable = null + bigTable, + spillPriority) + } // close the bigTable so it can be spilled + bigTable = null + withResource(catalog.acquireBuffer(bigHandle)) { buffer => + assertResult(StorageTier.DEVICE)(buffer.storageTier) + withResource(buffer.getColumnarBatch(sparkTypes)) { actualBatch => + TestUtils.compareBatches(expectedBatch, actualBatch) + } + } catalog.synchronousSpill(devStore, 0) val rapidsBufferCaptor: ArgumentCaptor[RapidsBuffer] = ArgumentCaptor.forClass(classOf[RapidsBuffer]) - verify(mockStore).copyBuffer( + verify(spyStore).copyBuffer( rapidsBufferCaptor.capture(), ArgumentMatchers.any[RapidsBufferCatalog], ArgumentMatchers.any[Cuda.Stream]) assertResult(bigHandle.id)(rapidsBufferCaptor.getValue.id) + withResource(catalog.acquireBuffer(bigHandle)) { buffer => + assertResult(StorageTier.DISK)(buffer.storageTier) + withResource(buffer.getColumnarBatch(sparkTypes)) { actualBatch => + TestUtils.compareBatches(expectedBatch, actualBatch) + } + } } } } diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RmmSparkRetrySuiteBase.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RmmSparkRetrySuiteBase.scala index d0db74d2e9d..a8e0ad550ea 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RmmSparkRetrySuiteBase.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RmmSparkRetrySuiteBase.scala @@ -37,7 +37,7 @@ class RmmSparkRetrySuiteBase extends AnyFunSuite with BeforeAndAfterEach { Rmm.initialize(RmmAllocationMode.CUDA_DEFAULT, null, 512 * 1024 * 1024) } deviceStorage = spy(new RapidsDeviceMemoryStore()) - val hostStore = new RapidsHostMemoryStore(1L * 1024 * 1024) + val hostStore = new RapidsHostMemoryStore(Some(1L * 1024 * 1024)) deviceStorage.setSpillStore(hostStore) val catalog = new RapidsBufferCatalog(deviceStorage, hostStore) // set these against the singleton so we close them later From c3df43ecd9f0ccc1aadadef032e52ca019f0e372 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 21 Sep 2023 12:33:01 -0500 Subject: [PATCH 21/46] Fix some issues found with different seeds in integration tests (#9280) Signed-off-by: Robert (Bobby) Evans --- integration_tests/src/main/python/cast_test.py | 9 +++++---- .../src/main/python/hash_aggregate_test.py | 8 ++++++-- integration_tests/src/main/python/map_test.py | 2 +- .../src/main/python/regexp_test.py | 2 +- .../com/nvidia/spark/rapids/RegexParser.scala | 5 +++-- .../RegularExpressionTranspilerSuite.scala | 18 +++++++++++++++--- 6 files changed, 31 insertions(+), 13 deletions(-) diff --git a/integration_tests/src/main/python/cast_test.py b/integration_tests/src/main/python/cast_test.py index bafc04e9d51..496002d931d 100644 --- a/integration_tests/src/main/python/cast_test.py +++ b/integration_tests/src/main/python/cast_test.py @@ -21,7 +21,7 @@ from marks import allow_non_gpu, approximate_float from pyspark.sql.types import * from spark_init_internal import spark_version -from datetime import datetime +from datetime import date, datetime import math _decimal_gen_36_5 = DecimalGen(precision=36, scale=5) @@ -667,14 +667,15 @@ def test_cast_int_to_string_not_UTC(): {"spark.sql.session.timeZone": "+08"}) not_utc_fallback_test_params = [(timestamp_gen, 'STRING'), (timestamp_gen, 'DATE'), - (date_gen, 'TIMESTAMP'), + # python does not like year 0, and with time zones the default start date can become year 0 :( + (DateGen(start=date(1, 1, 3)), 'TIMESTAMP'), (SetValuesGen(StringType(), ['2023-03-20 10:38:50', '2023-03-20 10:39:02']), 'TIMESTAMP')] @allow_non_gpu('ProjectExec') -@pytest.mark.parametrize('from_gen, to_type', not_utc_fallback_test_params) +@pytest.mark.parametrize('from_gen, to_type', not_utc_fallback_test_params, ids=idfn) def test_cast_fallback_not_UTC(from_gen, to_type): assert_gpu_fallback_collect( - lambda spark: unary_op_df(spark, from_gen, 100).selectExpr("CAST(a AS {}) as casted".format(to_type)), + lambda spark: unary_op_df(spark, from_gen).selectExpr("CAST(a AS {}) as casted".format(to_type)), "Cast", {"spark.sql.session.timeZone": "+08", "spark.rapids.sql.castStringToTimestamp.enabled": "true"}) diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index 6f00bb8d3d6..9ccdf3ef8fa 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -672,10 +672,12 @@ def test_min_max_group_by(data_gen): @pytest.mark.parametrize('use_obj_hash_agg', [True, False], ids=idfn) def test_hash_groupby_collect_list(data_gen, use_obj_hash_agg): assert_gpu_and_cpu_are_equal_collect( - lambda spark: gen_df(spark, data_gen, length=100).coalesce(1) + lambda spark: gen_df(spark, data_gen, length=100, num_slices=1) .groupby('a') .agg(f.collect_list('b')), conf={'spark.sql.execution.useObjectHashAggregateExec': str(use_obj_hash_agg).lower(), + # Disable RADIX sort as the CPU sort is not stable if it is + 'spark.sql.sort.enableRadixSort': False, 'spark.sql.shuffle.partitions': '1'}) @ignore_order(local=True) @@ -1201,7 +1203,9 @@ def test_groupby_first_last(data_gen): assert_gpu_and_cpu_are_equal_collect( # First and last are not deterministic when they are run in a real distributed setup. # We set parallelism 1 to prevent nondeterministic results because of distributed setup. - lambda spark: agg_fn(gen_df(spark, gen_fn, num_slices=1))) + lambda spark: agg_fn(gen_df(spark, gen_fn, num_slices=1)), + # Disable RADIX sort as the CPU sort is not stable if it is + conf={'spark.sql.sort.enableRadixSort': False}) @ignore_order(local=True) @pytest.mark.parametrize('data_gen', all_gen + _struct_only_nested_gens, ids=idfn) diff --git a/integration_tests/src/main/python/map_test.py b/integration_tests/src/main/python/map_test.py index bf31dfebb43..90dc14f4afd 100644 --- a/integration_tests/src/main/python/map_test.py +++ b/integration_tests/src/main/python/map_test.py @@ -337,7 +337,7 @@ def test_str_to_map_expr_fixed_pattern_input(): # Test pattern "key1:val1,key2:val2". # In order to prevent duplicate keys, the first key starts with a number [0-9] and the second # key start with a letter [a-zA-Z]. - data_gen = [('a', StringGen(pattern='[0-9].{0,10}:.{0,10},[a-zA-Z].{0,10}:.{0,10}', + data_gen = [('a', StringGen(pattern='[0-9][^:,]{0,10}:[^:,]{0,10},[a-zA-Z][^:,]{0,10}:[^:,]{0,10}', nullable=True))] assert_gpu_and_cpu_are_equal_collect( lambda spark: gen_df(spark, data_gen).selectExpr( diff --git a/integration_tests/src/main/python/regexp_test.py b/integration_tests/src/main/python/regexp_test.py index 63a96d0fd37..fa563d69e88 100644 --- a/integration_tests/src/main/python/regexp_test.py +++ b/integration_tests/src/main/python/regexp_test.py @@ -695,7 +695,7 @@ def test_regexp_replace_word(): def test_predefined_character_classes(): gen = mk_str_gen('[a-zA-Z]{0,2}[\r\n!"#$%&\'()*+,-./:;<=>?@[\\]^_`{|}~]{0,2}[0-9]{0,2}') assert_gpu_and_cpu_are_equal_collect( - lambda spark: unary_op_df(spark, gen).selectExpr( + lambda spark: unary_op_df(spark, gen, length=4096).selectExpr( 'regexp_replace(a, "\\\\p{Lower}", "x")', 'regexp_replace(a, "\\\\p{Upper}", "x")', 'regexp_replace(a, "\\\\p{ASCII}", "x")', diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala index 5867c3648c1..1db0f868213 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RegexParser.scala @@ -41,7 +41,8 @@ import com.nvidia.spark.rapids.RegexParser.toReadableString * - https://matt.might.net/articles/parsing-regex-with-recursive-descent/ */ class RegexParser(pattern: String) { - private val regexPunct = "!\"#$%&'()*+,-./:;<=>?@\\^_`{|}~" + // Note that [, ] and \ should be part of Punct, but they are handled separately + private val regexPunct = """!"#$%&'()*+,-./:;<=>?@^_`{|}~""" private val escapeChars = Map('n' -> '\n', 'r' -> '\r', 't' -> '\t', 'f' -> '\f', 'a' -> '\u0007', 'b' -> '\b', 'e' -> '\u001b') @@ -475,7 +476,7 @@ class RegexParser(pattern: String) { case "Punct" => val res:ListBuffer[RegexCharacterClassComponent] = ListBuffer(regexPunct.map(RegexChar): _*) - res ++= ListBuffer(RegexEscaped('['), RegexEscaped(']')) + res ++= ListBuffer(RegexEscaped('['), RegexEscaped(']'), RegexEscaped('\\')) case "Graph" => ListBuffer(getCharacters("Alnum"), getCharacters("Punct")).flatten case "Print" => diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala index 79e2ab782a2..e63c1cce8de 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/RegularExpressionTranspilerSuite.scala @@ -484,12 +484,24 @@ class RegularExpressionTranspilerSuite extends AnyFunSuite { doTranspileTest("a\\Z{1,}", expected) } - test("transpile predefined character classes") { + test("transpile predefined character classes Lower") { doTranspileTest("\\p{Lower}", "[a-z]") + } + + test("transpile predefined character classes Alpha") { doTranspileTest("\\p{Alpha}", "[a-zA-Z]") + } + + test("transpile predefined character classes Alnum") { doTranspileTest("\\p{Alnum}", "[a-zA-Z0-9]") - doTranspileTest("\\p{Punct}", "[!\"#$%&'()*+,\\-./:;<=>?@\\^_`{|}~\\[\\]]") - doTranspileTest("\\p{Print}", "[a-zA-Z0-9!\"#$%&'()*+,\\-./:;<=>?@\\^_`{|}~\\[\\]\u0020]") + } + + test("transpile predefined character classes Punct") { + doTranspileTest("\\p{Punct}", "[!\"#$%&'()*+,\\-./:;<=>?@^_`{|}~\\[\\]\\\\]") + } + + test("transpile predefined character classes Print") { + doTranspileTest("\\p{Print}", "[a-zA-Z0-9!\"#$%&'()*+,\\-./:;<=>?@^_`{|}~\\[\\]\\\\\u0020]") } test("transpile with group index to extract") { From 599f8cf96e213e2af6e94251ffb1f5f5cbd75289 Mon Sep 17 00:00:00 2001 From: "Robert (Bobby) Evans" Date: Thu, 21 Sep 2023 12:37:25 -0500 Subject: [PATCH 22/46] Fix leak in test and double free in corner case (#9264) Signed-off-by: Robert (Bobby) Evans --- .../spark/rapids/GpuCoalesceBatches.scala | 41 ++++-- .../spark/rapids/GpuColumnarToRowExec.scala | 4 +- ...ternalRowToCudfRowIteratorRetrySuite.scala | 134 +++++++++--------- 3 files changed, 103 insertions(+), 76 deletions(-) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala index 13928fceb89..e7acd575285 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCoalesceBatches.scala @@ -409,6 +409,26 @@ abstract class AbstractGpuCoalesceIterator( } } + /** + * A Simple wrapper around a ColumnarBatch to let us avoid closing it in some cases. + */ + private class BatchWrapper(var cb: ColumnarBatch) extends AutoCloseable { + def get: ColumnarBatch = cb + + def release: ColumnarBatch = { + val tmp = cb + cb = null + tmp + } + + override def close(): Unit = { + if (cb != null) { + cb.close() + cb = null + } + } + } + /** * Add input batches to the `batches` collection up to the limit specified * by the goal. Note: for a size goal, if any incoming batch is greater than this size @@ -447,13 +467,13 @@ abstract class AbstractGpuCoalesceIterator( } while(maybeFilteredIter.hasNext) { - var cb = maybeFilteredIter.next() + val cb = new BatchWrapper(maybeFilteredIter.next()) closeOnExcept(cb) { _ => - val nextRows = cb.numRows() + val nextRows = cb.get.numRows() // filter out empty batches if (nextRows > 0) { numInputRows += nextRows - val nextBytes = getBatchDataSize(cb) + val nextBytes = getBatchDataSize(cb.get) // calculate the new sizes based on this input batch being added to the current // output batch @@ -488,9 +508,10 @@ abstract class AbstractGpuCoalesceIterator( } } // 2) Filter the incoming batch. - val filteredCbIter = GpuFilter.filterAndClose(cb, filterTier, + // filterAndClose takes ownership of CB so we should not close it on a failure + // anymore... + val filteredCbIter = GpuFilter.filterAndClose(cb.release, filterTier, NoopMetric, NoopMetric, opTime) - cb = null // null out `cb` to prevent multiple close calls while (filteredCbIter.hasNext) { closeOnExcept(filteredCbIter.next()) { filteredCb => val filteredWouldBeRows = filteredNumRows + filteredCb.numRows() @@ -521,23 +542,23 @@ abstract class AbstractGpuCoalesceIterator( s"At least $wouldBeRows are in this partition, even after filtering " + s"nulls. Please try increasing your partition count.") } - case _ => saveOnDeck(cb) // not a single batch requirement + case _ => saveOnDeck(cb.get) // not a single batch requirement } } else if (batchRowLimit > 0 && wouldBeRows > batchRowLimit) { - saveOnDeck(cb) + saveOnDeck(cb.get) } else if (wouldBeBytes > goal.targetSizeBytes && numBytes > 0) { // There are no explicit checks for the concatenate result exceeding the cudf 2^31 // row count limit for any column. We are relying on cudf's concatenate to throw // an exception if this occurs and limiting performance-oriented goals to under // 2GB data total to avoid hitting that error. - saveOnDeck(cb) + saveOnDeck(cb.get) } else { - addBatch(cb) + addBatch(cb.get) numRows = wouldBeRows numBytes = wouldBeBytes } } else { - cleanupInputBatch(cb) + cleanupInputBatch(cb.get) } } // end of closeOnExcept(cb) } // end of while(maybeFilteredIter.hasNext) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarToRowExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarToRowExec.scala index 6ccf12af5ae..291171ccebb 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarToRowExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuColumnarToRowExec.scala @@ -223,7 +223,7 @@ class ColumnarToRowIterator(batches: Iterator[ColumnarBatch], opTime: GpuMetric, streamTime: GpuMetric, nullSafe: Boolean = false, - releaseSemaphore: Boolean = true) extends Iterator[InternalRow] { + releaseSemaphore: Boolean = true) extends Iterator[InternalRow] with AutoCloseable { // GPU batches read in must be closed by the receiver (us) @transient private var cb: ColumnarBatch = null private var it: java.util.Iterator[InternalRow] = null @@ -241,6 +241,8 @@ class ColumnarToRowIterator(batches: Iterator[ColumnarBatch], } } + override def close(): Unit = closeCurrentBatch() + private def closeCurrentBatch(): Unit = { if (cb != null) { cb.close() diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GeneratedInternalRowToCudfRowIteratorRetrySuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GeneratedInternalRowToCudfRowIteratorRetrySuite.scala index ea8ca9bc5cf..2080d79b3ca 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GeneratedInternalRowToCudfRowIteratorRetrySuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GeneratedInternalRowToCudfRowIteratorRetrySuite.scala @@ -48,23 +48,24 @@ class GeneratedInternalRowToCudfRowIteratorRetrySuite test("a retry when copying to device is handled") { val batch = buildBatch() - val ctriter = new ColumnarToRowIterator( - Seq(batch).iterator, - NoopMetric, NoopMetric, NoopMetric, NoopMetric) - val schema = Array(AttributeReference("longcol", LongType)().toAttribute) - val myIter = GeneratedInternalRowToCudfRowIterator( - ctriter, schema, TargetSize(Int.MaxValue), - NoopMetric, NoopMetric, NoopMetric, NoopMetric, NoopMetric) - // this forces a retry on the copy of the host column to a device column - RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId) - withResource(myIter.next()) { devBatch => - withResource(buildBatch()) { expected => - TestUtils.compareBatches(expected, devBatch) + val batchIter = Seq(batch).iterator + withResource(new ColumnarToRowIterator(batchIter, NoopMetric, NoopMetric, NoopMetric, + NoopMetric)) { ctriter => + val schema = Array(AttributeReference("longcol", LongType)().toAttribute) + val myIter = GeneratedInternalRowToCudfRowIterator( + ctriter, schema, TargetSize(Int.MaxValue), + NoopMetric, NoopMetric, NoopMetric, NoopMetric, NoopMetric) + // this forces a retry on the copy of the host column to a device column + RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId) + withResource(myIter.next()) { devBatch => + withResource(buildBatch()) { expected => + TestUtils.compareBatches(expected, devBatch) + } } + assert(!GpuColumnVector.extractBases(batch).exists(_.getRefCount > 0)) + assert(!myIter.hasNext) + assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) } - assert(!GpuColumnVector.extractBases(batch).exists(_.getRefCount > 0)) - assert(!myIter.hasNext) - assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) } test("a retry when converting to a table is handled") { @@ -83,28 +84,29 @@ class GeneratedInternalRowToCudfRowIteratorRetrySuite }).when(deviceStorage) .addTable(any(), any(), any(), any()) - val ctriter = new ColumnarToRowIterator(batchIter, - NoopMetric, NoopMetric, NoopMetric, NoopMetric) - val schema = Array(AttributeReference("longcol", LongType)().toAttribute) - val myIter = spy(GeneratedInternalRowToCudfRowIterator( - ctriter, schema, TargetSize(Int.MaxValue), - NoopMetric, NoopMetric, NoopMetric, NoopMetric, NoopMetric)) - RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId, 2) - assertResult(0)(getAndResetNumRetryThrowCurrentTask) - withResource(myIter.next()) { devBatch => - withResource(buildBatch()) { expected => - TestUtils.compareBatches(expected, devBatch) + withResource(new ColumnarToRowIterator(batchIter, NoopMetric, NoopMetric, NoopMetric, + NoopMetric)) { ctriter => + val schema = Array(AttributeReference("longcol", LongType)().toAttribute) + val myIter = spy(GeneratedInternalRowToCudfRowIterator( + ctriter, schema, TargetSize(Int.MaxValue), + NoopMetric, NoopMetric, NoopMetric, NoopMetric, NoopMetric)) + RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId, 2) + assertResult(0)(getAndResetNumRetryThrowCurrentTask) + withResource(myIter.next()) { devBatch => + withResource(buildBatch()) { expected => + TestUtils.compareBatches(expected, devBatch) + } } + assertResult(5)(getAndResetNumRetryThrowCurrentTask) + assert(!myIter.hasNext) + assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) + // This is my wrap around of checking that we did retry the last part + // where we are converting the device column of rows into an actual column. + // Because we asked for 3 retries, we would ask the spill framework 4 times to materialize + // a batch. + verify(rapidsBufferSpy, times(4)) + .getColumnarBatch(any()) } - assertResult(5)(getAndResetNumRetryThrowCurrentTask) - assert(!myIter.hasNext) - assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) - // This is my wrap around of checking that we did retry the last part - // where we are converting the device column of rows into an actual column. - // Because we asked for 3 retries, we would ask the spill framework 4 times to materialize - // a batch. - verify(rapidsBufferSpy, times(4)) - .getColumnarBatch(any()) } test("spilling the device column of rows works") { @@ -126,44 +128,46 @@ class GeneratedInternalRowToCudfRowIteratorRetrySuite }).when(deviceStorage) .addTable(any(), any(), any(), any()) - val ctriter = new ColumnarToRowIterator(batchIter, - NoopMetric, NoopMetric, NoopMetric, NoopMetric) - val schema = Array(AttributeReference("longcol", LongType)().toAttribute) - val myIter = spy(GeneratedInternalRowToCudfRowIterator( - ctriter, schema, TargetSize(Int.MaxValue), - NoopMetric, NoopMetric, NoopMetric, NoopMetric, NoopMetric)) - RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId, 2) - assertResult(0)(getAndResetNumRetryThrowCurrentTask) - withResource(myIter.next()) { devBatch => - withResource(buildBatch()) { expected => - TestUtils.compareBatches(expected, devBatch) + withResource(new ColumnarToRowIterator(batchIter, NoopMetric, NoopMetric, NoopMetric, + NoopMetric)) { ctriter => + val schema = Array(AttributeReference("longcol", LongType)().toAttribute) + val myIter = spy(GeneratedInternalRowToCudfRowIterator( + ctriter, schema, TargetSize(Int.MaxValue), + NoopMetric, NoopMetric, NoopMetric, NoopMetric, NoopMetric)) + RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId, 2) + assertResult(0)(getAndResetNumRetryThrowCurrentTask) + withResource(myIter.next()) { devBatch => + withResource(buildBatch()) { expected => + TestUtils.compareBatches(expected, devBatch) + } } + assertResult(5)(getAndResetNumRetryThrowCurrentTask) + assert(!myIter.hasNext) + assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) + // This is my wrap around of checking that we did retry the last part + // where we are converting the device column of rows into an actual column. + // Because we asked for 3 retries, we would ask the spill framework 4 times to materialize + // a batch. + verify(rapidsBufferSpy, times(4)) + .getColumnarBatch(any()) } - assertResult(5)(getAndResetNumRetryThrowCurrentTask) - assert(!myIter.hasNext) - assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) - // This is my wrap around of checking that we did retry the last part - // where we are converting the device column of rows into an actual column. - // Because we asked for 3 retries, we would ask the spill framework 4 times to materialize - // a batch. - verify(rapidsBufferSpy, times(4)) - .getColumnarBatch(any()) } test("a split and retry when copying to device is not handled, and we throw") { val batch = buildBatch() val batchIter = Seq(batch).iterator - val ctriter = new ColumnarToRowIterator(batchIter, NoopMetric, NoopMetric, NoopMetric, - NoopMetric) - val schema = Array(AttributeReference("longcol", LongType)().toAttribute) - val myIter = GeneratedInternalRowToCudfRowIterator( - ctriter, schema, TargetSize(1), - NoopMetric, NoopMetric, NoopMetric, NoopMetric, NoopMetric) - RmmSpark.forceSplitAndRetryOOM(RmmSpark.getCurrentThreadId) - assertThrows[SplitAndRetryOOM] { - myIter.next() + withResource(new ColumnarToRowIterator(batchIter, NoopMetric, NoopMetric, NoopMetric, + NoopMetric)) { ctriter => + val schema = Array(AttributeReference("longcol", LongType)().toAttribute) + val myIter = GeneratedInternalRowToCudfRowIterator( + ctriter, schema, TargetSize(1), + NoopMetric, NoopMetric, NoopMetric, NoopMetric, NoopMetric) + RmmSpark.forceSplitAndRetryOOM(RmmSpark.getCurrentThreadId) + assertThrows[SplitAndRetryOOM] { + myIter.next() + } + assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) } - assertResult(0)(RapidsBufferCatalog.getDeviceStorage.currentSize) } } \ No newline at end of file From 06afeecdac30c041f54ecc4a63d4dd67ab45d048 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Thu, 21 Sep 2023 16:57:05 -0500 Subject: [PATCH 23/46] Refactor ExternalSource to move some APIs to converted GPU format or scan (#9283) Signed-off-by: Jason Lowe --- .../spark/source/GpuSparkBatchQueryScan.java | 4 +- .../iceberg/spark/source/GpuSparkScan.java | 4 +- .../nvidia/spark/rapids/AvroProvider.scala | 24 +------- .../com/nvidia/spark/rapids/GpuCSVScan.scala | 7 +-- .../com/nvidia/spark/rapids/GpuOrcScan.scala | 4 +- .../nvidia/spark/rapids/GpuOverrides.scala | 4 +- .../nvidia/spark/rapids/GpuParquetScan.scala | 4 +- .../spark/rapids/GpuReadCSVFileFormat.scala | 14 ++++- .../rapids/GpuReadFileFormatWithMetrics.scala | 13 ++++- .../spark/rapids/GpuReadOrcFileFormat.scala | 23 +++++++- .../rapids/GpuReadParquetFileFormat.scala | 24 +++++++- .../com/nvidia/spark/rapids/GpuScan.scala | 27 +++++++++ .../spark/rapids/GpuTransitionOverrides.scala | 21 +++---- .../com/nvidia/spark/rapids/RapidsMeta.scala | 4 +- .../rapids/iceberg/IcebergProvider.scala | 6 +- .../rapids/iceberg/IcebergProviderImpl.scala | 13 +---- .../catalyst/json/rapids/GpuJsonScan.scala | 4 +- .../json/rapids/GpuReadJsonFileFormat.scala | 14 ++++- .../spark/sql/rapids/AvroProviderImpl.scala | 13 +---- .../spark/sql/rapids/ExternalSource.scala | 55 +----------------- .../apache/spark/sql/rapids/GpuAvroScan.scala | 4 +- .../sql/rapids/GpuFileSourceScanExec.scala | 57 ++++--------------- .../sql/rapids/GpuReadAvroFileFormat.scala | 23 +++++++- .../spark/rapids/shims/GpuBatchScanExec.scala | 10 +--- .../rapids/shims/RapidsOrcScanMeta.scala | 5 +- .../rapids/shims/RapidsParquetScanMeta.scala | 5 +- .../spark/rapids/shims/GpuBatchScanExec.scala | 10 +--- .../rapids/shims/RapidsCsvScanMeta.scala | 5 +- .../rapids/shims/RapidsOrcScanMeta.scala | 5 +- .../rapids/shims/RapidsParquetScanMeta.scala | 5 +- .../spark/rapids/shims/GpuBatchScanExec.scala | 10 +--- .../rapids/shims/RapidsOrcScanMeta.scala | 5 +- .../rapids/shims/RapidsParquetScanMeta.scala | 5 +- .../spark/rapids/shims/GpuBatchScanExec.scala | 10 +--- .../spark/rapids/shims/GpuBatchScanExec.scala | 10 +--- .../spark/rapids/shims/GpuBatchScanExec.scala | 9 +-- 36 files changed, 215 insertions(+), 245 deletions(-) create mode 100644 sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuScan.scala diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkBatchQueryScan.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkBatchQueryScan.java index b89ad62163e..c20300c1441 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkBatchQueryScan.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkBatchQueryScan.java @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022, NVIDIA CORPORATION. + * Copyright (c) 2022-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -368,7 +368,7 @@ public String toString() { } /** Return a copy of "this" but with "queryUsesInputFile = true" */ - public GpuSparkBatchQueryScan copyWithInputFileTrue() { + public GpuSparkBatchQueryScan withInputFile() { return new GpuSparkBatchQueryScan(SparkSession.active(), table(), this.scan, readConf(), expectedSchema(), filterExpressions(), rapidsConf(), true // queryUsesInputFile diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java index 62cbd3d14c2..1c6ae738e1a 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuSparkScan.java @@ -23,9 +23,9 @@ import java.util.stream.Collectors; import com.nvidia.spark.rapids.GpuMetric; +import com.nvidia.spark.rapids.GpuScanWrapper; import com.nvidia.spark.rapids.MultiFileReaderUtils; import com.nvidia.spark.rapids.RapidsConf; -import com.nvidia.spark.rapids.ScanWithMetricsWrapper; import com.nvidia.spark.rapids.iceberg.spark.Spark3Util; import com.nvidia.spark.rapids.iceberg.spark.SparkReadConf; import com.nvidia.spark.rapids.iceberg.spark.SparkSchemaUtil; @@ -67,7 +67,7 @@ * GPU-accelerated Iceberg Scan. * This is derived from Apache Iceberg's SparkScan class. */ -abstract class GpuSparkScan extends ScanWithMetricsWrapper +abstract class GpuSparkScan extends GpuScanWrapper implements Scan, SupportsReportStatistics { private static final Logger LOG = LoggerFactory.getLogger(GpuSparkScan.class); diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AvroProvider.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AvroProvider.scala index 03bf3362062..05ea12d086e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AvroProvider.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AvroProvider.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022, NVIDIA CORPORATION. + * Copyright (c) 2022-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,20 +16,14 @@ package com.nvidia.spark.rapids -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.sql.connector.read.{PartitionReaderFactory, Scan} +import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.FileFormat -import org.apache.spark.sql.rapids.GpuFileSourceScanExec -import org.apache.spark.sql.sources.Filter -import org.apache.spark.util.SerializableConfiguration trait AvroProvider { /** If the file format is supported as an external source */ def isSupportedFormat(format: FileFormat): Boolean - def isPerFileReadEnabledForFormat(format: FileFormat, conf: RapidsConf): Boolean - def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit /** @@ -38,19 +32,5 @@ trait AvroProvider { */ def getReadFileFormat(format: FileFormat): FileFormat - /** - * Create a multi-file reader factory for the input format. - * Better to check if the format is supported first by calling 'isSupportedFormat' - */ - def createMultiFileReaderFactory( - format: FileFormat, - broadcastedConf: Broadcast[SerializableConfiguration], - pushedFilters: Array[Filter], - fileScan: GpuFileSourceScanExec): PartitionReaderFactory - def getScans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] - - def isSupportedScan(scan: Scan): Boolean - - def copyScanWithInputFileTrue(scan: Scan): Scan } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala index db323c36eff..8b77ef6c71c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCSVScan.scala @@ -51,9 +51,6 @@ trait ScanWithMetrics { var metrics : Map[String, GpuMetric] = Map.empty } -// Allows use of ScanWithMetrics from Java code -class ScanWithMetricsWrapper extends ScanWithMetrics - object GpuCSVScan { def tagSupport(scanMeta: ScanMeta[CSVScan]) : Unit = { val scan = scanMeta.wrapped @@ -213,7 +210,7 @@ case class GpuCSVScan( dataFilters: Seq[Expression], maxReaderBatchSizeRows: Integer, maxReaderBatchSizeBytes: Long) - extends TextBasedFileScan(sparkSession, options) with ScanWithMetrics { + extends TextBasedFileScan(sparkSession, options) with GpuScan { private lazy val parsedOptions: CSVOptions = new CSVOptions( options.asScala.toMap, @@ -251,6 +248,8 @@ case class GpuCSVScan( partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) + override def withInputFile(): GpuScan = this + override def equals(obj: Any): Boolean = obj match { case c: GpuCSVScan => super.equals(c) && dataSchema == c.dataSchema && options == c.options && diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index 919bd539abe..6e48d7bb8d0 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -84,7 +84,7 @@ case class GpuOrcScan( dataFilters: Seq[Expression], rapidsConf: RapidsConf, queryUsesInputFile: Boolean = false) - extends ScanWithMetrics with FileScan with Logging { + extends FileScan with GpuScan with Logging { override def isSplitable(path: Path): Boolean = true @@ -124,6 +124,8 @@ case class GpuOrcScan( def withFilters( partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) + + override def withInputFile(): GpuScan = copy(queryUsesInputFile = true) } object GpuOrcScan { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index 8814b14c22b..b3db1b6168c 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -3619,7 +3619,7 @@ object GpuOverrides extends Logging { (a, conf, p, r) => new ScanMeta[CSVScan](a, conf, p, r) { override def tagSelfForGpu(): Unit = GpuCSVScan.tagSupport(this) - override def convertToGpu(): Scan = + override def convertToGpu(): GpuScan = GpuCSVScan(a.sparkSession, a.fileIndex, a.dataSchema, @@ -3636,7 +3636,7 @@ object GpuOverrides extends Logging { (a, conf, p, r) => new ScanMeta[JsonScan](a, conf, p, r) { override def tagSelfForGpu(): Unit = GpuJsonScan.tagSupport(this) - override def convertToGpu(): Scan = + override def convertToGpu(): GpuScan = GpuJsonScan(a.sparkSession, a.fileIndex, a.dataSchema, diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala index d13b9617ae9..53b783581a4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetScan.scala @@ -108,7 +108,7 @@ case class GpuParquetScan( dataFilters: Seq[Expression], rapidsConf: RapidsConf, queryUsesInputFile: Boolean = false) - extends ScanWithMetrics with FileScan with Logging { + extends FileScan with GpuScan with Logging { override def isSplitable(path: Path): Boolean = true @@ -146,6 +146,8 @@ case class GpuParquetScan( def withFilters( partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): FileScan = this.copy(partitionFilters = partitionFilters, dataFilters = dataFilters) + + override def withInputFile(): GpuScan = copy(queryUsesInputFile = true) } object GpuParquetScan { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala index 7cf2a9fbd6e..79403bae6b5 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadCSVFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2022, NVIDIA CORPORATION. + * Copyright (c) 2020-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,12 +19,15 @@ package com.nvidia.spark.rapids import com.nvidia.spark.rapids.shims.SparkShimImpl import org.apache.hadoop.conf.Configuration +import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.csv.CSVOptions +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat +import org.apache.spark.sql.rapids.GpuFileSourceScanExec import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -66,6 +69,15 @@ class GpuReadCSVFileFormat extends CSVFileFormat with GpuReadFileFormatWithMetri options) PartitionReaderIterator.buildReader(factory) } + + override def isPerFileReadEnabled(conf: RapidsConf): Boolean = true + + override def createMultiFileReaderFactory( + broadcastedConf: Broadcast[SerializableConfiguration], + pushedFilters: Array[Filter], + fileScan: GpuFileSourceScanExec): PartitionReaderFactory = { + throw new IllegalStateException("CSV format does not support multifile reads") + } } object GpuReadCSVFileFormat { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala index 488aea02039..9b384916ab5 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadFileFormatWithMetrics.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2021, NVIDIA CORPORATION. + * Copyright (c) 2020-2023, NVIDIA CORPORATION. * * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -22,11 +22,15 @@ package com.nvidia.spark.rapids import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapreduce.Job +import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriterFactory, PartitionedFile} +import org.apache.spark.sql.rapids.GpuFileSourceScanExec import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration trait GpuReadFileFormatWithMetrics extends FileFormat { final override def supportBatch(spark: SparkSession, dataSchema: StructType): Boolean = true @@ -62,4 +66,11 @@ trait GpuReadFileFormatWithMetrics extends FileFormat { metrics: Map[String, GpuMetric], alluxioPathReplacementMap: Option[Map[String, String]]) : PartitionedFile => Iterator[InternalRow] + + def isPerFileReadEnabled(conf: RapidsConf): Boolean + + def createMultiFileReaderFactory( + broadcastedConf: Broadcast[SerializableConfiguration], + pushedFilters: Array[Filter], + fileScan: GpuFileSourceScanExec): PartitionReaderFactory } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala index b6dd08856a0..1792f36e09b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadOrcFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2022, NVIDIA CORPORATION. + * Copyright (c) 2020-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,11 +19,14 @@ package com.nvidia.spark.rapids import com.nvidia.spark.rapids.shims.SparkShimImpl import org.apache.hadoop.conf.Configuration +import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat +import org.apache.spark.sql.rapids.GpuFileSourceScanExec import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -58,6 +61,24 @@ class GpuReadOrcFileFormat extends OrcFileFormat with GpuReadFileFormatWithMetri options) PartitionReaderIterator.buildReader(factory) } + + override def isPerFileReadEnabled(conf: RapidsConf): Boolean = conf.isOrcPerFileReadEnabled + + override def createMultiFileReaderFactory( + broadcastedConf: Broadcast[SerializableConfiguration], + pushedFilters: Array[Filter], + fileScan: GpuFileSourceScanExec): PartitionReaderFactory = { + GpuOrcMultiFilePartitionReaderFactory( + fileScan.conf, + broadcastedConf, + fileScan.relation.dataSchema, + fileScan.requiredSchema, + fileScan.readPartitionSchema, + pushedFilters, + fileScan.rapidsConf, + fileScan.allMetrics, + fileScan.queryUsesInputFile) + } } object GpuReadOrcFileFormat { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala index 8718219222c..e9b8beba036 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuReadParquetFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2020-2022, NVIDIA CORPORATION. + * Copyright (c) 2020-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -19,11 +19,14 @@ package com.nvidia.spark.rapids import com.nvidia.spark.rapids.shims.SparkShimImpl import org.apache.hadoop.conf.Configuration +import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.rapids.GpuFileSourceScanExec import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -59,6 +62,25 @@ class GpuReadParquetFileFormat extends ParquetFileFormat with GpuReadFileFormatW alluxioPathReplacementMap) PartitionReaderIterator.buildReader(factory) } + + override def isPerFileReadEnabled(conf: RapidsConf): Boolean = conf.isParquetPerFileReadEnabled + + override def createMultiFileReaderFactory( + broadcastedConf: Broadcast[SerializableConfiguration], + pushedFilters: Array[Filter], + fileScan: GpuFileSourceScanExec): PartitionReaderFactory = { + GpuParquetMultiFilePartitionReaderFactory( + fileScan.conf, + broadcastedConf, + fileScan.relation.dataSchema, + fileScan.requiredSchema, + fileScan.readPartitionSchema, + pushedFilters, + fileScan.rapidsConf, + fileScan.allMetrics, + fileScan.queryUsesInputFile, + fileScan.alluxioPathsMap) + } } object GpuReadParquetFileFormat { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuScan.scala new file mode 100644 index 00000000000..eb3e8a71e2d --- /dev/null +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuScan.scala @@ -0,0 +1,27 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 com.nvidia.spark.rapids + +import org.apache.spark.sql.connector.read.Scan + +trait GpuScan extends Scan with ScanWithMetrics { + /** Create a version of this scan with input file name support */ + def withInputFile(): GpuScan +} + +// Allows use of GpuScan from Java code +abstract class GpuScanWrapper extends GpuScan diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala index b980be75d4f..b80913aa64a 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuTransitionOverrides.scala @@ -35,7 +35,7 @@ import org.apache.spark.sql.execution.command.{DataWritingCommandExec, ExecutedC import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanExecBase, DropTableExec, ShowTablesExec} import org.apache.spark.sql.execution.exchange.{BroadcastExchangeLike, Exchange, ReusedExchangeExec, ShuffleExchangeLike} import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, HashedRelationBroadcastMode} -import org.apache.spark.sql.rapids.{ExternalSource, GpuDataSourceScanExec, GpuFileSourceScanExec, GpuInputFileBlockLength, GpuInputFileBlockStart, GpuInputFileName, GpuShuffleEnv, GpuTaskMetrics} +import org.apache.spark.sql.rapids.{GpuDataSourceScanExec, GpuFileSourceScanExec, GpuInputFileBlockLength, GpuInputFileBlockStart, GpuInputFileName, GpuShuffleEnv, GpuTaskMetrics} import org.apache.spark.sql.rapids.execution.{ExchangeMappingCache, GpuBroadcastExchangeExec, GpuBroadcastExchangeExecBase, GpuBroadcastToRowExec, GpuCustomShuffleReaderExec, GpuHashJoin, GpuShuffleExchangeExecBase} import org.apache.spark.sql.types.StructType @@ -362,20 +362,13 @@ class GpuTransitionOverrides extends Rule[SparkPlan] { disableUntilInput: Boolean = false): SparkPlan = { plan match { case batchScan: GpuBatchScanExec => - if ((batchScan.scan.isInstanceOf[GpuParquetScan] || - batchScan.scan.isInstanceOf[GpuOrcScan] || - ExternalSource.isSupportedScan(batchScan.scan)) && - (disableUntilInput || disableScanUntilInput(batchScan))) { - val scanCopy = batchScan.scan match { - case parquetScan: GpuParquetScan => - parquetScan.copy(queryUsesInputFile = true) - case orcScan: GpuOrcScan => - orcScan.copy(queryUsesInputFile = true) - case eScan if ExternalSource.isSupportedScan(eScan) => - ExternalSource.copyScanWithInputFileTrue(eScan) - case _ => throw new RuntimeException("Wrong format") // never reach here + if (disableUntilInput || disableScanUntilInput(batchScan)) { + val newScan = batchScan.scan.withInputFile() + if (newScan ne batchScan.scan) { + batchScan.copy(scan = newScan) + } else { + batchScan } - batchScan.copy(scan = scanCopy) } else { batchScan } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala index df6376a7b8b..8ed35cffad4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsMeta.scala @@ -503,7 +503,7 @@ abstract class ScanMeta[INPUT <: Scan](scan: INPUT, conf: RapidsConf, parent: Option[RapidsMeta[_, _, _]], rule: DataFromReplacementRule) - extends RapidsMeta[INPUT, Scan, Scan](scan, conf, parent, rule) { + extends RapidsMeta[INPUT, Scan, GpuScan](scan, conf, parent, rule) { override val childPlans: Seq[SparkPlanMeta[_]] = Seq.empty override val childExprs: Seq[BaseExprMeta[_]] = Seq.empty @@ -529,7 +529,7 @@ final class RuleNotFoundScanMeta[INPUT <: Scan]( willNotWorkOnGpu(s"GPU does not currently support the operator ${scan.getClass}") } - override def convertToGpu(): Scan = + override def convertToGpu(): GpuScan = throw new IllegalStateException("Cannot be converted to GPU") } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/iceberg/IcebergProvider.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/iceberg/IcebergProvider.scala index b17a70d303f..e4eea2d92b8 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/iceberg/IcebergProvider.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/iceberg/IcebergProvider.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022, NVIDIA CORPORATION. + * Copyright (c) 2022-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -22,11 +22,7 @@ import org.apache.spark.sql.connector.read.Scan /** Interfaces to avoid accessing the optional Apache Iceberg jars directly in common code. */ trait IcebergProvider { - def isSupportedScan(scan: Scan): Boolean - def getScans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] - - def copyScanWithInputFileTrue(scan: Scan): Scan } object IcebergProvider { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/iceberg/IcebergProviderImpl.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/iceberg/IcebergProviderImpl.scala index 830a1029a80..4a6c2b63fa7 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/iceberg/IcebergProviderImpl.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/iceberg/IcebergProviderImpl.scala @@ -19,14 +19,12 @@ package com.nvidia.spark.rapids.iceberg import scala.reflect.ClassTag import scala.util.{Failure, Try} -import com.nvidia.spark.rapids.{FileFormatChecks, IcebergFormatType, RapidsConf, ReadFileOp, ScanMeta, ScanRule, ShimReflectionUtils} +import com.nvidia.spark.rapids.{FileFormatChecks, GpuScan, IcebergFormatType, RapidsConf, ReadFileOp, ScanMeta, ScanRule, ShimReflectionUtils} import com.nvidia.spark.rapids.iceberg.spark.source.GpuSparkBatchQueryScan import org.apache.spark.sql.connector.read.Scan class IcebergProviderImpl extends IcebergProvider { - override def isSupportedScan(scan: Scan): Boolean = scan.isInstanceOf[GpuSparkBatchQueryScan] - override def getScans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] = { val cpuIcebergScanClass = ShimReflectionUtils.loadClass(IcebergProvider.cpuScanClassName) Seq(new ScanRule[Scan]( @@ -60,17 +58,10 @@ class IcebergProviderImpl extends IcebergProvider { } } - override def convertToGpu(): Scan = convertedScan.get + override def convertToGpu(): GpuScan = convertedScan.get }, "Iceberg scan", ClassTag(cpuIcebergScanClass)) ).map(r => (r.getClassFor.asSubclass(classOf[Scan]), r)).toMap } - - override def copyScanWithInputFileTrue(scan: Scan): Scan = scan match { - case icebergBatchScan: GpuSparkBatchQueryScan => - icebergBatchScan.copyWithInputFileTrue(); - case _ => - throw new RuntimeException(s"Unsupported scan type: ${scan.getClass.getSimpleName}") - } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala index 8f1e5066c85..8ac04f19429 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuJsonScan.scala @@ -198,7 +198,7 @@ case class GpuJsonScan( dataFilters: Seq[Expression], maxReaderBatchSizeRows: Integer, maxReaderBatchSizeBytes: Long) - extends TextBasedFileScan(sparkSession, options) with ScanWithMetrics { + extends TextBasedFileScan(sparkSession, options) with GpuScan { private lazy val parsedOptions: JSONOptions = new JSONOptions( options.asScala.toMap, @@ -222,6 +222,8 @@ case class GpuJsonScan( dataSchema, readDataSchema, readPartitionSchema, parsedOptions, maxReaderBatchSizeRows, maxReaderBatchSizeBytes, metrics, options.asScala.toMap) } + + override def withInputFile(): GpuScan = this } case class GpuJsonPartitionReaderFactory( diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala index 7fe59b983cc..3b85493266e 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/catalyst/json/rapids/GpuReadJsonFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022, NVIDIA CORPORATION. + * Copyright (c) 2022-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,12 +20,15 @@ import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.shims.SparkShimImpl import org.apache.hadoop.conf.Configuration +import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.json.JSONOptionsInRead +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.json.JsonFileFormat +import org.apache.spark.sql.rapids.GpuFileSourceScanExec import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -66,6 +69,15 @@ class GpuReadJsonFileFormat extends JsonFileFormat with GpuReadFileFormatWithMet options) PartitionReaderIterator.buildReader(factory) } + + override def isPerFileReadEnabled(conf: RapidsConf): Boolean = true + + override def createMultiFileReaderFactory( + broadcastedConf: Broadcast[SerializableConfiguration], + pushedFilters: Array[Filter], + fileScan: GpuFileSourceScanExec): PartitionReaderFactory = { + throw new IllegalStateException("JSON format does not support multifile reads") + } } object GpuReadJsonFileFormat { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AvroProviderImpl.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AvroProviderImpl.scala index dd913a04631..0f0618ce8fb 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AvroProviderImpl.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AvroProviderImpl.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022, NVIDIA CORPORATION. + * Copyright (c) 2022-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -99,7 +99,7 @@ class AvroProviderImpl extends AvroProvider { (a, conf, p, r) => new ScanMeta[AvroScan](a, conf, p, r) { override def tagSelfForGpu(): Unit = GpuAvroScan.tagSupport(this) - override def convertToGpu(): Scan = + override def convertToGpu(): GpuScan = GpuAvroScan(a.sparkSession, a.fileIndex, a.dataSchema, @@ -113,13 +113,4 @@ class AvroProviderImpl extends AvroProvider { }) ).map(r => (r.getClassFor.asSubclass(classOf[Scan]), r)).toMap } - - def isSupportedScan(scan: Scan): Boolean = scan.isInstanceOf[GpuAvroScan] - - def copyScanWithInputFileTrue(scan: Scan): Scan = scan match { - case avroScan: GpuAvroScan => - avroScan.copy(queryUsesInputFile=true) - case _ => - throw new RuntimeException(s"Unsupported scan type: ${scan.getClass.getSimpleName}") - } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExternalSource.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExternalSource.scala index b9631377438..edf7317dec3 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExternalSource.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExternalSource.scala @@ -23,14 +23,13 @@ import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.delta.DeltaProvider import com.nvidia.spark.rapids.iceberg.IcebergProvider -import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging -import org.apache.spark.sql.connector.read.{PartitionReaderFactory, Scan} +import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.execution.datasources.FileFormat -import org.apache.spark.sql.sources.{CreatableRelationProvider, Filter} -import org.apache.spark.util.{SerializableConfiguration, Utils} +import org.apache.spark.sql.sources.CreatableRelationProvider +import org.apache.spark.util.Utils /** * The subclass of AvroProvider imports spark-avro classes. This file should not imports @@ -79,12 +78,6 @@ object ExternalSource extends Logging { } else false } - def isPerFileReadEnabledForFormat(format: FileFormat, conf: RapidsConf): Boolean = { - if (hasSparkAvroJar) { - avroProvider.isPerFileReadEnabledForFormat(format, conf) - } else false - } - def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { if (hasSparkAvroJar) { avroProvider.tagSupportForGpuFileSourceScan(meta) @@ -103,23 +96,6 @@ object ExternalSource extends Logging { } } - /** - * Create a multi-file reader factory for the input format. - * Better to check if the format is supported first by calling 'isSupportedFormat' - */ - def createMultiFileReaderFactory( - format: FileFormat, - broadcastedConf: Broadcast[SerializableConfiguration], - pushedFilters: Array[Filter], - fileScan: GpuFileSourceScanExec): PartitionReaderFactory = { - if (hasSparkAvroJar) { - avroProvider.createMultiFileReaderFactory(format, broadcastedConf, pushedFilters, - fileScan) - } else { - throw new RuntimeException(s"File format $format is not supported yet") - } - } - def getScans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] = { var scans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] = Map.empty if (hasSparkAvroJar) { @@ -131,31 +107,6 @@ object ExternalSource extends Logging { scans } - /** If the scan is supported as an external source */ - def isSupportedScan(scan: Scan): Boolean = { - if (hasSparkAvroJar && avroProvider.isSupportedScan(scan)) { - true - } else if (hasIcebergJar && icebergProvider.isSupportedScan(scan)) { - true - } else { - false - } - } - - /** - * Clone the input scan with setting 'true' to the 'queryUsesInputFile'. - * Better to check if the scan is supported first by calling 'isSupportedScan'. - */ - def copyScanWithInputFileTrue(scan: Scan): Scan = { - if (hasSparkAvroJar && avroProvider.isSupportedScan(scan)) { - avroProvider.copyScanWithInputFileTrue(scan) - } else if (hasIcebergJar && icebergProvider.isSupportedScan(scan)) { - icebergProvider.copyScanWithInputFileTrue(scan) - } else { - throw new RuntimeException(s"Unsupported scan type: ${scan.getClass.getSimpleName}") - } - } - def wrapCreatableRelationProvider[INPUT <: CreatableRelationProvider]( provider: INPUT, conf: RapidsConf, diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala index 539cb36438c..a4c64b2b2f4 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuAvroScan.scala @@ -104,7 +104,7 @@ case class GpuAvroScan( rapidsConf: RapidsConf, partitionFilters: Seq[Expression] = Seq.empty, dataFilters: Seq[Expression] = Seq.empty, - queryUsesInputFile: Boolean = false) extends FileScan with ScanWithMetrics { + queryUsesInputFile: Boolean = false) extends FileScan with GpuScan { override def isSplitable(path: Path): Boolean = true override def createReaderFactory(): PartitionReaderFactory = { @@ -144,6 +144,8 @@ case class GpuAvroScan( override def description(): String = { super.description() + ", PushedFilters: " + seqToString(pushedFilters) } + + override def withInputFile(): GpuScan = copy(queryUsesInputFile = true) } /** Avro partition reader factory to build columnar reader */ diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala index fe22478bcb3..2d5d5cae895 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala @@ -37,7 +37,6 @@ import org.apache.spark.sql.execution.{ExecSubqueryExpression, ExplainUtils, Fil import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.json.JsonFileFormat -import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.rapids.shims.FilePartitionShims @@ -91,8 +90,7 @@ case class GpuFileSourceScanExec( dataOutAttrs ++ prunedPartOutAttrs }.getOrElse(originalOutput) - private[rapids] val readPartitionSchema = - requiredPartitionSchema.getOrElse(relation.partitionSchema) + val readPartitionSchema = requiredPartitionSchema.getOrElse(relation.partitionSchema) // this is set only when we either explicitly replaced a path for CONVERT_TIME // or when TASK_TIME if one of the paths will be replaced. @@ -100,14 +98,13 @@ case class GpuFileSourceScanExec( // should update this to None and read directly from s3 to get faster. private var alluxioPathReplacementMap: Option[Map[String, String]] = alluxioPathsMap - private val isPerFileReadEnabled = relation.fileFormat match { - case _: ParquetFileFormat => rapidsConf.isParquetPerFileReadEnabled - case _: OrcFileFormat => rapidsConf.isOrcPerFileReadEnabled - case ef if ExternalSource.isSupportedFormat(ef) => - ExternalSource.isPerFileReadEnabledForFormat(ef, rapidsConf) - case _ => true // For others, default to PERFILE reader + @transient private val gpuFormat = relation.fileFormat match { + case g: GpuReadFileFormatWithMetrics => g + case f => throw new IllegalStateException(s"${f.getClass} is not a GPU format with metrics") } + private val isPerFileReadEnabled = gpuFormat.isPerFileReadEnabled(rapidsConf) + override def otherCopyArgs: Seq[AnyRef] = Seq(rapidsConf) // All expressions are filter expressions used on the CPU. @@ -363,8 +360,7 @@ case class GpuFileSourceScanExec( lazy val inputRDD: RDD[InternalRow] = { val readFile: Option[(PartitionedFile) => Iterator[InternalRow]] = if (isPerFileReadEnabled) { - val fileFormat = relation.fileFormat.asInstanceOf[GpuReadFileFormatWithMetrics] - val reader = fileFormat.buildReaderWithPartitionValuesAndMetrics( + val reader = gpuFormat.buildReaderWithPartitionValuesAndMetrics( sparkSession = relation.sparkSession, dataSchema = relation.dataSchema, partitionSchema = readPartitionSchema, @@ -620,44 +616,13 @@ case class GpuFileSourceScanExec( lazy val readerFactory: PartitionReaderFactory = { // here we are making an optimization to read more then 1 file at a time on the CPU side // if they are small files before sending it down to the GPU - val sqlConf = relation.sparkSession.sessionState.conf val hadoopConf = relation.sparkSession.sessionState.newHadoopConfWithOptions(relation.options) val broadcastedHadoopConf = relation.sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) - - relation.fileFormat match { - case _: ParquetFileFormat => - GpuParquetMultiFilePartitionReaderFactory( - sqlConf, - broadcastedHadoopConf, - relation.dataSchema, - requiredSchema, - readPartitionSchema, - pushedDownFilters.toArray, - rapidsConf, - allMetrics, - queryUsesInputFile, - alluxioPathReplacementMap) - case _: OrcFileFormat => - GpuOrcMultiFilePartitionReaderFactory( - sqlConf, - broadcastedHadoopConf, - relation.dataSchema, - requiredSchema, - readPartitionSchema, - pushedDownFilters.toArray, - rapidsConf, - allMetrics, - queryUsesInputFile) - case ef if ExternalSource.isSupportedFormat(ef) => - ExternalSource.createMultiFileReaderFactory( - ef, - broadcastedHadoopConf, - pushedDownFilters.toArray, - this) - case other => - throw new IllegalArgumentException(s"${other.getClass.getCanonicalName} is not supported") - } + gpuFormat.createMultiFileReaderFactory( + broadcastedHadoopConf, + pushedDownFilters.toArray, + this) } // Filters unused DynamicPruningExpression expressions - one which has been replaced diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala index 0386602de0a..d50b0b184ca 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuReadAvroFileFormat.scala @@ -1,5 +1,5 @@ /* - * Copyright (c) 2022, NVIDIA CORPORATION. + * Copyright (c) 2022-2023, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -20,9 +20,11 @@ import com.nvidia.spark.rapids.{GpuMetric, GpuReadFileFormatWithMetrics, Partiti import com.nvidia.spark.rapids.shims.SparkShimImpl import org.apache.hadoop.conf.Configuration +import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.SparkSession import org.apache.spark.sql.avro.{AvroFileFormat, AvroOptions} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.sources.Filter @@ -64,6 +66,25 @@ class GpuReadAvroFileFormat extends AvroFileFormat with GpuReadFileFormatWithMet options) PartitionReaderIterator.buildReader(factory) } + + override def isPerFileReadEnabled(conf: RapidsConf): Boolean = conf.isAvroPerFileReadEnabled + + override def createMultiFileReaderFactory( + broadcastedConf: Broadcast[SerializableConfiguration], + pushedFilters: Array[Filter], + fileScan: GpuFileSourceScanExec): PartitionReaderFactory = { + GpuAvroMultiFilePartitionReaderFactory( + fileScan.relation.sparkSession.sessionState.conf, + fileScan.rapidsConf, + broadcastedConf, + fileScan.relation.dataSchema, + fileScan.requiredSchema, + fileScan.readPartitionSchema, + new AvroOptions(fileScan.relation.options, broadcastedConf.value.value), + fileScan.allMetrics, + pushedFilters, + fileScan.queryUsesInputFile) + } } object GpuReadAvroFileFormat { diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index 82ec332f283..e7d5d452abb 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -21,7 +21,7 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, ScanWithMetrics} +import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, GpuScan} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -33,7 +33,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuBatchScanExec( output: Seq[AttributeReference], - @transient scan: Scan) extends DataSourceV2ScanExecBase with GpuBatchScanExecMetrics { + @transient scan: GpuScan) extends DataSourceV2ScanExecBase with GpuBatchScanExecMetrics { @transient lazy val batch: Batch = scan.toBatch @transient override lazy val partitions: Seq[InputPartition] = batch.planInputPartitions() @@ -41,11 +41,7 @@ case class GpuBatchScanExec( override lazy val readerFactory: PartitionReaderFactory = batch.createReaderFactory() override lazy val inputRDD: RDD[InternalRow] = { - scan match { - case s: ScanWithMetrics => s.metrics = allMetrics - case _ => - } - + scan.metrics = allMetrics new GpuDataSourceRDD(sparkContext, partitions, readerFactory) } diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala index b7e8c758622..34bd8d7cc4f 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala @@ -21,9 +21,8 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuOrcScan, RapidsConf, RapidsMeta, ScanMeta} +import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuOrcScan, GpuScan, RapidsConf, RapidsMeta, ScanMeta} -import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan class RapidsOrcScanMeta( @@ -37,7 +36,7 @@ class RapidsOrcScanMeta( GpuOrcScan.tagSupport(this) } - override def convertToGpu(): Scan = + override def convertToGpu(): GpuScan = GpuOrcScan(oScan.sparkSession, oScan.hadoopConf, oScan.fileIndex, diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala index 7c3f57574de..42a905c9c45 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala @@ -21,9 +21,8 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuParquetScan, RapidsConf, RapidsMeta, ScanMeta} +import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuParquetScan, GpuScan, RapidsConf, RapidsMeta, ScanMeta} -import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan class RapidsParquetScanMeta( @@ -37,7 +36,7 @@ class RapidsParquetScanMeta( GpuParquetScan.tagSupport(this) } - override def convertToGpu(): Scan = { + override def convertToGpu(): GpuScan = { GpuParquetScan(pScan.sparkSession, pScan.hadoopConf, pScan.fileIndex, diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index 79e3758d543..c1cac965df5 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -26,7 +26,7 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims import com.google.common.base.Objects -import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, ScanWithMetrics} +import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, GpuScan} import org.apache.spark.SparkException import org.apache.spark.rdd.RDD @@ -42,7 +42,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuBatchScanExec( output: Seq[AttributeReference], - @transient scan: Scan, + @transient scan: GpuScan, runtimeFilters: Seq[Expression] = Seq.empty) extends DataSourceV2ScanExecBase with GpuBatchScanExecMetrics { @transient lazy val batch: Batch = scan.toBatch @@ -97,11 +97,7 @@ case class GpuBatchScanExec( override lazy val readerFactory: PartitionReaderFactory = batch.createReaderFactory() override lazy val inputRDD: RDD[InternalRow] = { - scan match { - case s: ScanWithMetrics => s.metrics = allMetrics - case _ => - } - + scan.metrics = allMetrics if (filteredPartitions.isEmpty && outputPartitioning == SinglePartition) { // return an empty RDD with 1 partition if dynamic filtering removed the only split sparkContext.parallelize(Array.empty[InternalRow], 1) diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala index fcac077990c..9b5c7c3fdda 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsCsvScanMeta.scala @@ -35,9 +35,8 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuCSVScan, RapidsConf, RapidsMeta, ScanMeta} +import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuCSVScan, GpuScan, RapidsConf, RapidsMeta, ScanMeta} -import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.v2.csv.CSVScan class RapidsCsvScanMeta( @@ -53,7 +52,7 @@ class RapidsCsvScanMeta( TagScanForRuntimeFiltering.tagScanForRuntimeFiltering(this, cScan) } - override def convertToGpu(): Scan = + override def convertToGpu(): GpuScan = GpuCSVScan(cScan.sparkSession, cScan.fileIndex, cScan.dataSchema, diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala index f9dfe5ef158..a713529cb83 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala @@ -25,9 +25,8 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuOrcScan, RapidsConf, RapidsMeta, ScanMeta} +import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuOrcScan, GpuScan, RapidsConf, RapidsMeta, ScanMeta} -import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan class RapidsOrcScanMeta( @@ -43,7 +42,7 @@ class RapidsOrcScanMeta( TagScanForRuntimeFiltering.tagScanForRuntimeFiltering(this, oScan) } - override def convertToGpu(): Scan = + override def convertToGpu(): GpuScan = GpuOrcScan(oScan.sparkSession, oScan.hadoopConf, oScan.fileIndex, diff --git a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala index 536fb420b97..5f5c8efd46d 100644 --- a/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala +++ b/sql-plugin/src/main/spark320/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala @@ -25,9 +25,8 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuParquetScan, RapidsConf, RapidsMeta, ScanMeta} +import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuParquetScan, GpuScan, RapidsConf, RapidsMeta, ScanMeta} -import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan class RapidsParquetScanMeta( @@ -43,7 +42,7 @@ class RapidsParquetScanMeta( TagScanForRuntimeFiltering.tagScanForRuntimeFiltering(this, pScan) } - override def convertToGpu(): Scan = { + override def convertToGpu(): GpuScan = { GpuParquetScan(pScan.sparkSession, pScan.hadoopConf, pScan.fileIndex, diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index fce9532168a..c3ab21954e6 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -24,7 +24,7 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims import com.google.common.base.Objects -import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, ScanWithMetrics} +import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, GpuScan} import org.apache.spark.SparkException import org.apache.spark.rdd.RDD @@ -40,7 +40,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuBatchScanExec( output: Seq[AttributeReference], - @transient scan: Scan, + @transient scan: GpuScan, runtimeFilters: Seq[Expression] = Seq.empty, keyGroupedPartitioning: Option[Seq[Expression]] = None) extends DataSourceV2ScanExecBase with GpuBatchScanExecMetrics { @@ -116,11 +116,7 @@ case class GpuBatchScanExec( override lazy val readerFactory: PartitionReaderFactory = batch.createReaderFactory() override lazy val inputRDD: RDD[InternalRow] = { - scan match { - case s: ScanWithMetrics => s.metrics = allMetrics - case _ => - } - + scan.metrics = allMetrics if (filteredPartitions.isEmpty && outputPartitioning == SinglePartition) { // return an empty RDD with 1 partition if dynamic filtering removed the only split sparkContext.parallelize(Array.empty[InternalRow], 1) diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala index e78338eb72c..319df49a17a 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsOrcScanMeta.scala @@ -28,9 +28,8 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuOrcScan, RapidsConf, RapidsMeta, ScanMeta} +import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuOrcScan, GpuScan, RapidsConf, RapidsMeta, ScanMeta} -import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan class RapidsOrcScanMeta( @@ -50,7 +49,7 @@ class RapidsOrcScanMeta( } } - override def convertToGpu(): Scan = + override def convertToGpu(): GpuScan = GpuOrcScan(oScan.sparkSession, oScan.hadoopConf, oScan.fileIndex, diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala index 464a4dcb73e..b9322f15b30 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/RapidsParquetScanMeta.scala @@ -28,9 +28,8 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims -import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuParquetScan, RapidsConf, RapidsMeta, ScanMeta} +import com.nvidia.spark.rapids.{DataFromReplacementRule, GpuParquetScan, GpuScan, RapidsConf, RapidsMeta, ScanMeta} -import org.apache.spark.sql.connector.read.Scan import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan class RapidsParquetScanMeta( @@ -51,7 +50,7 @@ class RapidsParquetScanMeta( } } - override def convertToGpu(): Scan = { + override def convertToGpu(): GpuScan = { GpuParquetScan(pScan.sparkSession, pScan.hadoopConf, pScan.fileIndex, diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index cd2b738e927..8fbb493ddb3 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -20,7 +20,7 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims import com.google.common.base.Objects -import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, ScanWithMetrics} +import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, GpuScan} import org.apache.spark.SparkException import org.apache.spark.rdd.RDD @@ -37,7 +37,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuBatchScanExec( output: Seq[AttributeReference], - @transient scan: Scan, + @transient scan: GpuScan, runtimeFilters: Seq[Expression], keyGroupedPartitioning: Option[Seq[Expression]], ordering: Option[Seq[SortOrder]], @@ -116,11 +116,7 @@ case class GpuBatchScanExec( override lazy val readerFactory: PartitionReaderFactory = batch.createReaderFactory() override lazy val inputRDD: RDD[InternalRow] = { - scan match { - case s: ScanWithMetrics => s.metrics = allMetrics - case _ => - } - + scan.metrics = allMetrics if (filteredPartitions.isEmpty && outputPartitioning == SinglePartition) { // return an empty RDD with 1 partition if dynamic filtering removed the only split sparkContext.parallelize(Array.empty[InternalRow], 1) diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index b4dc5d8ef17..3cbdd49a47c 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -20,7 +20,7 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims import com.google.common.base.Objects -import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, ScanWithMetrics} +import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, GpuScan} import org.apache.spark.SparkException import org.apache.spark.rdd.RDD @@ -37,7 +37,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuBatchScanExec( output: Seq[AttributeReference], - @transient scan: Scan, + @transient scan: GpuScan, runtimeFilters: Seq[Expression], keyGroupedPartitioning: Option[Seq[Expression]], ordering: Option[Seq[SortOrder]], @@ -116,11 +116,7 @@ case class GpuBatchScanExec( override lazy val readerFactory: PartitionReaderFactory = batch.createReaderFactory() override lazy val inputRDD: RDD[InternalRow] = { - scan match { - case s: ScanWithMetrics => s.metrics = allMetrics - case _ => - } - + scan.metrics = allMetrics if (filteredPartitions.isEmpty && outputPartitioning == SinglePartition) { // return an empty RDD with 1 partition if dynamic filtering removed the only split sparkContext.parallelize(Array.empty[InternalRow], 1) diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index 237d375e593..2a951cb9500 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -22,7 +22,7 @@ spark-rapids-shim-json-lines ***/ package com.nvidia.spark.rapids.shims import com.google.common.base.Objects -import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, ScanWithMetrics} +import com.nvidia.spark.rapids.{GpuBatchScanExecMetrics, GpuScan} import org.apache.spark.SparkException import org.apache.spark.rdd.RDD @@ -40,7 +40,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch case class GpuBatchScanExec( output: Seq[AttributeReference], - @transient scan: Scan, + @transient scan: GpuScan, runtimeFilters: Seq[Expression] = Seq.empty, keyGroupedPartitioning: Option[Seq[Expression]] = None, ordering: Option[Seq[SortOrder]] = None, @@ -140,10 +140,7 @@ case class GpuBatchScanExec( override lazy val readerFactory: PartitionReaderFactory = batch.createReaderFactory() override lazy val inputRDD: RDD[InternalRow] = { - scan match { - case s: ScanWithMetrics => s.metrics = allMetrics - case _ => - } + scan.metrics = allMetrics val rdd = if (filteredPartitions.isEmpty && outputPartitioning == SinglePartition) { // return an empty RDD with 1 partition if dynamic filtering removed the only split sparkContext.parallelize(Array.empty[InternalRow], 1) From f68cb944ca4fa1890608cf0f1f1c462b80290945 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Fri, 22 Sep 2023 16:22:17 -0500 Subject: [PATCH 24/46] Fix file format checks to be exact and handle Delta Lake column mapping [databricks] (#9279) * Check for exact file format class rather than an instance of the class Signed-off-by: Jason Lowe * Add GpuDeltaParquetFileFormat Signed-off-by: Jason Lowe * Fix Parquet field IDs not being written when column ID mapping requested --------- Signed-off-by: Jason Lowe --- .../rapids/delta/DeltaProviderImpl.scala | 24 +++++- .../delta/GpuDeltaParquetFileFormatBase.scala | 84 +++++++++++++++++++ .../spark/rapids/delta/RapidsDeltaUtils.scala | 10 +-- .../spark/rapids/delta/DeltaIOProvider.scala | 8 +- .../delta/GpuDeltaParquetFileFormat.scala | 84 +++++++++++++++++++ .../spark/rapids/delta/RapidsDeltaUtils.scala | 10 +-- .../delta/delta20x/Delta20xProvider.scala | 19 ++++- .../GpuDelta20xParquetFileFormat.scala | 27 ++++++ .../delta20x/GpuOptimisticTransaction.scala | 8 +- .../delta/delta21x/Delta21xProvider.scala | 19 ++++- .../GpuDelta21xParquetFileFormat.scala | 27 ++++++ .../delta21x/GpuOptimisticTransaction.scala | 8 +- .../delta/delta22x/Delta22xProvider.scala | 19 ++++- .../GpuDelta22xParquetFileFormat.scala | 38 +++++++++ .../delta22x/GpuOptimisticTransaction.scala | 8 +- .../delta/delta24x/Delta24xProvider.scala | 33 +++++++- .../GpuDelta24xParquetFileFormat.scala | 48 +++++++++++ .../delta24x/GpuOptimisticTransaction.scala | 8 +- .../rapids/GpuOptimisticTransaction.scala | 13 ++- .../delta/GpuDeltaParquetFileFormat.scala | 36 ++++++++ .../rapids/GpuOptimisticTransaction.scala | 13 ++- .../delta/GpuDeltaParquetFileFormat.scala | 36 ++++++++ .../rapids/GpuOptimisticTransaction.scala | 8 +- .../delta/GpuDeltaParquetFileFormat.scala | 65 ++++++++++++++ .../src/main/python/delta_lake_merge_test.py | 45 +++++----- .../src/main/python/delta_lake_test.py | 66 ++++++++++++++- .../src/main/python/delta_lake_write_test.py | 34 +++++++- .../nvidia/spark/rapids/AvroProvider.scala | 2 +- .../nvidia/spark/rapids/GpuOverrides.scala | 35 ++++---- .../spark/rapids/GpuParquetFileFormat.scala | 2 +- .../spark/rapids/delta/DeltaProvider.scala | 19 ++++- .../spark/sql/rapids/AvroProviderImpl.scala | 53 ++++-------- .../spark/sql/rapids/ExternalSource.scala | 23 +++-- .../sql/rapids/GpuFileSourceScanExec.scala | 44 ++++++---- .../spark/sql/rapids/GpuOrcFileFormat.scala | 6 +- ...aSourceTableAsSelectCommandMetaShims.scala | 18 ++-- .../rapids/shims/ParquetFieldIdShims.scala | 3 +- .../rapids/shims/ParquetFieldIdShims.scala | 44 ++++++++++ .../rapids/shims/ParquetFieldIdShims.scala | 5 +- .../rapids/shims/ParquetFieldIdShims.scala | 45 ++++++++++ ...aSourceTableAsSelectCommandMetaShims.scala | 17 ++-- 41 files changed, 941 insertions(+), 173 deletions(-) create mode 100644 delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormatBase.scala create mode 100644 delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala create mode 100644 delta-lake/delta-20x/src/main/scala/com/nvidia/spark/rapids/delta/delta20x/GpuDelta20xParquetFileFormat.scala create mode 100644 delta-lake/delta-21x/src/main/scala/com/nvidia/spark/rapids/delta/delta21x/GpuDelta21xParquetFileFormat.scala create mode 100644 delta-lake/delta-22x/src/main/scala/com/nvidia/spark/rapids/delta/delta22x/GpuDelta22xParquetFileFormat.scala create mode 100644 delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/GpuDelta24xParquetFileFormat.scala create mode 100644 delta-lake/delta-spark321db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala create mode 100644 delta-lake/delta-spark330db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala create mode 100644 delta-lake/delta-spark332db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala create mode 100644 sql-plugin/src/main/spark321db/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala create mode 100644 sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala diff --git a/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/DeltaProviderImpl.scala b/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/DeltaProviderImpl.scala index 6f225810b6e..9f7e0896ec6 100644 --- a/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/DeltaProviderImpl.scala +++ b/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/DeltaProviderImpl.scala @@ -16,14 +16,15 @@ package com.nvidia.spark.rapids.delta -import com.databricks.sql.transaction.tahoe.DeltaLog +import com.databricks.sql.transaction.tahoe.{DeltaLog, DeltaParquetFileFormat} import com.databricks.sql.transaction.tahoe.commands.{DeleteCommand, DeleteCommandEdge, MergeIntoCommand, MergeIntoCommandEdge, UpdateCommand, UpdateCommandEdge} import com.databricks.sql.transaction.tahoe.sources.DeltaDataSource import com.nvidia.spark.rapids._ import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.command.RunnableCommand -import org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand +import org.apache.spark.sql.execution.datasources.{FileFormat, SaveIntoDataSourceCommand} import org.apache.spark.sql.rapids.ExternalSource import org.apache.spark.sql.sources.CreatableRelationProvider @@ -72,6 +73,25 @@ object DeltaProviderImpl extends DeltaProviderImplBase { .disabledByDefault("Delta Lake update support is experimental") ).map(r => (r.getClassFor.asSubclass(classOf[RunnableCommand]), r)).toMap } + + override def isSupportedFormat(format: Class[_ <: FileFormat]): Boolean = { + format == classOf[DeltaParquetFileFormat] + } + + override def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { + val format = meta.wrapped.relation.fileFormat + if (format.getClass == classOf[DeltaParquetFileFormat]) { + GpuReadParquetFileFormat.tagSupport(meta) + GpuDeltaParquetFileFormat.tagSupportForGpuFileSourceScan(meta) + } else { + meta.willNotWorkOnGpu(s"format ${format.getClass} is not supported") + } + } + + override def getReadFileFormat(format: FileFormat): FileFormat = { + val cpuFormat = format.asInstanceOf[DeltaParquetFileFormat] + GpuDeltaParquetFileFormat.convertToGpu(cpuFormat) + } } class DeltaCreatableRelationProviderMeta( diff --git a/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormatBase.scala b/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormatBase.scala new file mode 100644 index 00000000000..598cf8248f4 --- /dev/null +++ b/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormatBase.scala @@ -0,0 +1,84 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 com.nvidia.spark.rapids.delta + +import com.databricks.sql.transaction.tahoe.{DeltaColumnMapping, DeltaColumnMappingMode, NoMapping} +import com.nvidia.spark.rapids.{GpuMetric, GpuParquetMultiFilePartitionReaderFactory, GpuReadParquetFileFormat} +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.PartitionReaderFactory +import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.rapids.GpuFileSourceScanExec +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration + +abstract class GpuDeltaParquetFileFormatBase extends GpuReadParquetFileFormat { + val columnMappingMode: DeltaColumnMappingMode + val referenceSchema: StructType + + def prepareSchema(inputSchema: StructType): StructType = { + DeltaColumnMapping.createPhysicalSchema(inputSchema, referenceSchema, columnMappingMode) + } + + override def createMultiFileReaderFactory( + broadcastedConf: Broadcast[SerializableConfiguration], + pushedFilters: Array[Filter], + fileScan: GpuFileSourceScanExec): PartitionReaderFactory = { + GpuParquetMultiFilePartitionReaderFactory( + fileScan.conf, + broadcastedConf, + prepareSchema(fileScan.relation.dataSchema), + prepareSchema(fileScan.requiredSchema), + prepareSchema(fileScan.readPartitionSchema), + pushedFilters, + fileScan.rapidsConf, + fileScan.allMetrics, + fileScan.queryUsesInputFile, + fileScan.alluxioPathsMap) + } + + override def buildReaderWithPartitionValuesAndMetrics( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration, + metrics: Map[String, GpuMetric], + alluxioPathReplacementMap: Option[Map[String, String]]) + : PartitionedFile => Iterator[InternalRow] = { + super.buildReaderWithPartitionValuesAndMetrics( + sparkSession, + prepareSchema(dataSchema), + prepareSchema(partitionSchema), + prepareSchema(requiredSchema), + filters, + options, + hadoopConf, + metrics, + alluxioPathReplacementMap) + } + + override def supportFieldName(name: String): Boolean = { + if (columnMappingMode != NoMapping) true else super.supportFieldName(name) + } +} diff --git a/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala b/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala index 22f5485092d..17511390fbe 100644 --- a/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala +++ b/delta-lake/common/src/main/databricks/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala @@ -33,11 +33,11 @@ object RapidsDeltaUtils { options: Map[String, String], spark: SparkSession): Unit = { FileFormatChecks.tag(meta, schema, DeltaFormatType, WriteFileOp) - DeltaLogShim.fileFormat(deltaLog) match { - case _: DeltaParquetFileFormat => - GpuParquetFileFormat.tagGpuSupport(meta, spark, options, schema) - case f => - meta.willNotWorkOnGpu(s"file format $f is not supported") + val format = DeltaLogShim.fileFormat(deltaLog) + if (format.getClass == classOf[DeltaParquetFileFormat]) { + GpuParquetFileFormat.tagGpuSupport(meta, spark, options, schema) + } else { + meta.willNotWorkOnGpu(s"file format $format is not supported") } checkIncompatibleConfs(meta, schema, deltaLog, spark.sessionState.conf, options) } diff --git a/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/DeltaIOProvider.scala b/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/DeltaIOProvider.scala index 0b7da564a93..64ad89490dd 100644 --- a/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/DeltaIOProvider.scala +++ b/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/DeltaIOProvider.scala @@ -21,10 +21,10 @@ import scala.util.Try import com.nvidia.spark.rapids._ import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.delta.DeltaLog +import org.apache.spark.sql.delta.{DeltaLog, DeltaParquetFileFormat} import org.apache.spark.sql.delta.rapids.DeltaRuntimeShim import org.apache.spark.sql.delta.sources.DeltaDataSource -import org.apache.spark.sql.execution.datasources.SaveIntoDataSourceCommand +import org.apache.spark.sql.execution.datasources.{FileFormat, SaveIntoDataSourceCommand} import org.apache.spark.sql.rapids.ExternalSource import org.apache.spark.sql.rapids.execution.UnshimmedTrampolineUtil import org.apache.spark.sql.sources.CreatableRelationProvider @@ -44,6 +44,10 @@ abstract class DeltaIOProvider extends DeltaProviderImplBase { }) ).map(r => (r.getClassFor.asSubclass(classOf[CreatableRelationProvider]), r)).toMap } + + override def isSupportedFormat(format: Class[_ <: FileFormat]): Boolean = { + format == classOf[DeltaParquetFileFormat] + } } class DeltaCreatableRelationProviderMeta( diff --git a/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala b/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala new file mode 100644 index 00000000000..96cb4b6d552 --- /dev/null +++ b/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala @@ -0,0 +1,84 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 com.nvidia.spark.rapids.delta + +import com.nvidia.spark.rapids.{GpuMetric, GpuParquetMultiFilePartitionReaderFactory, GpuReadParquetFileFormat} +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.connector.read.PartitionReaderFactory +import org.apache.spark.sql.delta.{DeltaColumnMapping, DeltaColumnMappingMode, NoMapping} +import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.rapids.GpuFileSourceScanExec +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration + +trait GpuDeltaParquetFileFormat extends GpuReadParquetFileFormat { + val columnMappingMode: DeltaColumnMappingMode + val referenceSchema: StructType + + def prepareSchema(inputSchema: StructType): StructType = { + DeltaColumnMapping.createPhysicalSchema(inputSchema, referenceSchema, columnMappingMode) + } + + override def createMultiFileReaderFactory( + broadcastedConf: Broadcast[SerializableConfiguration], + pushedFilters: Array[Filter], + fileScan: GpuFileSourceScanExec): PartitionReaderFactory = { + GpuParquetMultiFilePartitionReaderFactory( + fileScan.conf, + broadcastedConf, + prepareSchema(fileScan.relation.dataSchema), + prepareSchema(fileScan.requiredSchema), + prepareSchema(fileScan.readPartitionSchema), + pushedFilters, + fileScan.rapidsConf, + fileScan.allMetrics, + fileScan.queryUsesInputFile, + fileScan.alluxioPathsMap) + } + + override def buildReaderWithPartitionValuesAndMetrics( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration, + metrics: Map[String, GpuMetric], + alluxioPathReplacementMap: Option[Map[String, String]]) + : PartitionedFile => Iterator[InternalRow] = { + super.buildReaderWithPartitionValuesAndMetrics( + sparkSession, + prepareSchema(dataSchema), + prepareSchema(partitionSchema), + prepareSchema(requiredSchema), + filters, + options, + hadoopConf, + metrics, + alluxioPathReplacementMap) + } + + override def supportFieldName(name: String): Boolean = { + if (columnMappingMode != NoMapping) true else super.supportFieldName(name) + } +} diff --git a/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala b/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala index 75a6af75ae4..e0c5725f91d 100644 --- a/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala +++ b/delta-lake/common/src/main/delta-io/scala/com/nvidia/spark/rapids/delta/RapidsDeltaUtils.scala @@ -32,11 +32,11 @@ object RapidsDeltaUtils { options: Map[String, String], spark: SparkSession): Unit = { FileFormatChecks.tag(meta, schema, DeltaFormatType, WriteFileOp) - DeltaRuntimeShim.fileFormatFromLog(deltaLog) match { - case _: DeltaParquetFileFormat => - GpuParquetFileFormat.tagGpuSupport(meta, spark, options, schema) - case f => - meta.willNotWorkOnGpu(s"file format $f is not supported") + val format = DeltaRuntimeShim.fileFormatFromLog(deltaLog) + if (format.getClass == classOf[DeltaParquetFileFormat]) { + GpuParquetFileFormat.tagGpuSupport(meta, spark, options, schema) + } else { + meta.willNotWorkOnGpu(s"file format $format is not supported") } checkIncompatibleConfs(meta, deltaLog, spark.sessionState.conf, options) } diff --git a/delta-lake/delta-20x/src/main/scala/com/nvidia/spark/rapids/delta/delta20x/Delta20xProvider.scala b/delta-lake/delta-20x/src/main/scala/com/nvidia/spark/rapids/delta/delta20x/Delta20xProvider.scala index 256ebd10c48..1e8cef6cca7 100644 --- a/delta-lake/delta-20x/src/main/scala/com/nvidia/spark/rapids/delta/delta20x/Delta20xProvider.scala +++ b/delta-lake/delta-20x/src/main/scala/com/nvidia/spark/rapids/delta/delta20x/Delta20xProvider.scala @@ -16,11 +16,14 @@ package com.nvidia.spark.rapids.delta.delta20x -import com.nvidia.spark.rapids.{GpuOverrides, RunnableCommandRule} +import com.nvidia.spark.rapids.{GpuOverrides, GpuReadParquetFileFormat, RunnableCommandRule, SparkPlanMeta} import com.nvidia.spark.rapids.delta.DeltaIOProvider +import org.apache.spark.sql.delta.DeltaParquetFileFormat import org.apache.spark.sql.delta.commands.{DeleteCommand, MergeIntoCommand, UpdateCommand} +import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.execution.datasources.FileFormat object Delta20xProvider extends DeltaIOProvider { @@ -41,4 +44,18 @@ object Delta20xProvider extends DeltaIOProvider { .disabledByDefault("Delta Lake update support is experimental") ).map(r => (r.getClassFor.asSubclass(classOf[RunnableCommand]), r)).toMap } + + override def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { + val format = meta.wrapped.relation.fileFormat + if (format.getClass == classOf[DeltaParquetFileFormat]) { + GpuReadParquetFileFormat.tagSupport(meta) + } else { + meta.willNotWorkOnGpu(s"format ${format.getClass} is not supported") + } + } + + override def getReadFileFormat(format: FileFormat): FileFormat = { + val cpuFormat = format.asInstanceOf[DeltaParquetFileFormat] + GpuDelta20xParquetFileFormat(cpuFormat.columnMappingMode, cpuFormat.referenceSchema) + } } diff --git a/delta-lake/delta-20x/src/main/scala/com/nvidia/spark/rapids/delta/delta20x/GpuDelta20xParquetFileFormat.scala b/delta-lake/delta-20x/src/main/scala/com/nvidia/spark/rapids/delta/delta20x/GpuDelta20xParquetFileFormat.scala new file mode 100644 index 00000000000..05b764eed22 --- /dev/null +++ b/delta-lake/delta-20x/src/main/scala/com/nvidia/spark/rapids/delta/delta20x/GpuDelta20xParquetFileFormat.scala @@ -0,0 +1,27 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 com.nvidia.spark.rapids.delta.delta20x + +import com.nvidia.spark.rapids.delta.GpuDeltaParquetFileFormat + +import org.apache.spark.sql.delta.DeltaColumnMappingMode +import org.apache.spark.sql.types.StructType + +case class GpuDelta20xParquetFileFormat( + override val columnMappingMode: DeltaColumnMappingMode, + override val referenceSchema: StructType) extends GpuDeltaParquetFileFormat { +} diff --git a/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala b/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala index e8f2a9df685..282368fc95b 100644 --- a/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-20x/src/main/scala/org/apache/spark/sql/delta/rapids/delta20x/GpuOptimisticTransaction.scala @@ -200,9 +200,11 @@ class GpuOptimisticTransaction }.toMap } - val gpuFileFormat = deltaLog.fileFormat(metadata) match { - case _: DeltaParquetFileFormat => new GpuParquetFileFormat - case f => throw new IllegalStateException(s"file format $f is not supported") + val deltaFileFormat = deltaLog.fileFormat(metadata) + val gpuFileFormat = if (deltaFileFormat.getClass == classOf[DeltaParquetFileFormat]) { + new GpuParquetFileFormat + } else { + throw new IllegalStateException(s"file format $deltaFileFormat is not supported") } try { diff --git a/delta-lake/delta-21x/src/main/scala/com/nvidia/spark/rapids/delta/delta21x/Delta21xProvider.scala b/delta-lake/delta-21x/src/main/scala/com/nvidia/spark/rapids/delta/delta21x/Delta21xProvider.scala index 3ada9d2087e..934fe7eb95b 100644 --- a/delta-lake/delta-21x/src/main/scala/com/nvidia/spark/rapids/delta/delta21x/Delta21xProvider.scala +++ b/delta-lake/delta-21x/src/main/scala/com/nvidia/spark/rapids/delta/delta21x/Delta21xProvider.scala @@ -16,11 +16,14 @@ package com.nvidia.spark.rapids.delta.delta21x -import com.nvidia.spark.rapids.{GpuOverrides, RunnableCommandRule} +import com.nvidia.spark.rapids.{GpuOverrides, GpuReadParquetFileFormat, RunnableCommandRule, SparkPlanMeta} import com.nvidia.spark.rapids.delta.DeltaIOProvider +import org.apache.spark.sql.delta.DeltaParquetFileFormat import org.apache.spark.sql.delta.commands.{DeleteCommand, MergeIntoCommand, UpdateCommand} +import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.execution.datasources.FileFormat object Delta21xProvider extends DeltaIOProvider { @@ -41,4 +44,18 @@ object Delta21xProvider extends DeltaIOProvider { .disabledByDefault("Delta Lake update support is experimental") ).map(r => (r.getClassFor.asSubclass(classOf[RunnableCommand]), r)).toMap } + + override def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { + val format = meta.wrapped.relation.fileFormat + if (format.getClass == classOf[DeltaParquetFileFormat]) { + GpuReadParquetFileFormat.tagSupport(meta) + } else { + meta.willNotWorkOnGpu(s"format ${format.getClass} is not supported") + } + } + + override def getReadFileFormat(format: FileFormat): FileFormat = { + val cpuFormat = format.asInstanceOf[DeltaParquetFileFormat] + GpuDelta21xParquetFileFormat(cpuFormat.columnMappingMode, cpuFormat.referenceSchema) + } } diff --git a/delta-lake/delta-21x/src/main/scala/com/nvidia/spark/rapids/delta/delta21x/GpuDelta21xParquetFileFormat.scala b/delta-lake/delta-21x/src/main/scala/com/nvidia/spark/rapids/delta/delta21x/GpuDelta21xParquetFileFormat.scala new file mode 100644 index 00000000000..accbde4de3a --- /dev/null +++ b/delta-lake/delta-21x/src/main/scala/com/nvidia/spark/rapids/delta/delta21x/GpuDelta21xParquetFileFormat.scala @@ -0,0 +1,27 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 com.nvidia.spark.rapids.delta.delta21x + +import com.nvidia.spark.rapids.delta.GpuDeltaParquetFileFormat + +import org.apache.spark.sql.delta.DeltaColumnMappingMode +import org.apache.spark.sql.types.StructType + +case class GpuDelta21xParquetFileFormat( + override val columnMappingMode: DeltaColumnMappingMode, + override val referenceSchema: StructType) extends GpuDeltaParquetFileFormat { +} diff --git a/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala b/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala index b2d9854789e..9acded983d0 100644 --- a/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-21x/src/main/scala/org/apache/spark/sql/delta/rapids/delta21x/GpuOptimisticTransaction.scala @@ -200,9 +200,11 @@ class GpuOptimisticTransaction }.toMap } - val gpuFileFormat = deltaLog.fileFormat(metadata) match { - case _: DeltaParquetFileFormat => new GpuParquetFileFormat - case f => throw new IllegalStateException(s"file format $f is not supported") + val deltaFileFormat = deltaLog.fileFormat(metadata) + val gpuFileFormat = if (deltaFileFormat.getClass == classOf[DeltaParquetFileFormat]) { + new GpuParquetFileFormat + } else { + throw new IllegalStateException(s"file format $deltaFileFormat is not supported") } try { diff --git a/delta-lake/delta-22x/src/main/scala/com/nvidia/spark/rapids/delta/delta22x/Delta22xProvider.scala b/delta-lake/delta-22x/src/main/scala/com/nvidia/spark/rapids/delta/delta22x/Delta22xProvider.scala index db8a8c69f84..d5b5a79c7bc 100644 --- a/delta-lake/delta-22x/src/main/scala/com/nvidia/spark/rapids/delta/delta22x/Delta22xProvider.scala +++ b/delta-lake/delta-22x/src/main/scala/com/nvidia/spark/rapids/delta/delta22x/Delta22xProvider.scala @@ -16,11 +16,14 @@ package com.nvidia.spark.rapids.delta.delta22x -import com.nvidia.spark.rapids.{GpuOverrides, RunnableCommandRule} +import com.nvidia.spark.rapids.{GpuOverrides, GpuReadParquetFileFormat, RunnableCommandRule, SparkPlanMeta} import com.nvidia.spark.rapids.delta.DeltaIOProvider +import org.apache.spark.sql.delta.DeltaParquetFileFormat import org.apache.spark.sql.delta.commands.{DeleteCommand, MergeIntoCommand, UpdateCommand} +import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.execution.datasources.FileFormat object Delta22xProvider extends DeltaIOProvider { @@ -41,4 +44,18 @@ object Delta22xProvider extends DeltaIOProvider { .disabledByDefault("Delta Lake update support is experimental") ).map(r => (r.getClassFor.asSubclass(classOf[RunnableCommand]), r)).toMap } + + override def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { + val format = meta.wrapped.relation.fileFormat + if (format.getClass == classOf[DeltaParquetFileFormat]) { + GpuReadParquetFileFormat.tagSupport(meta) + } else { + meta.willNotWorkOnGpu(s"format ${format.getClass} is not supported") + } + } + + override def getReadFileFormat(format: FileFormat): FileFormat = { + val cpuFormat = format.asInstanceOf[DeltaParquetFileFormat] + GpuDelta22xParquetFileFormat(cpuFormat.columnMappingMode, cpuFormat.referenceSchema) + } } diff --git a/delta-lake/delta-22x/src/main/scala/com/nvidia/spark/rapids/delta/delta22x/GpuDelta22xParquetFileFormat.scala b/delta-lake/delta-22x/src/main/scala/com/nvidia/spark/rapids/delta/delta22x/GpuDelta22xParquetFileFormat.scala new file mode 100644 index 00000000000..3dfb55eb209 --- /dev/null +++ b/delta-lake/delta-22x/src/main/scala/com/nvidia/spark/rapids/delta/delta22x/GpuDelta22xParquetFileFormat.scala @@ -0,0 +1,38 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 com.nvidia.spark.rapids.delta.delta22x + +import com.nvidia.spark.rapids.delta.GpuDeltaParquetFileFormat + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.delta.{DeltaColumnMappingMode, IdMapping} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType + +case class GpuDelta22xParquetFileFormat( + override val columnMappingMode: DeltaColumnMappingMode, + override val referenceSchema: StructType) extends GpuDeltaParquetFileFormat { + + if (columnMappingMode == IdMapping) { + val requiredReadConf = SQLConf.PARQUET_FIELD_ID_READ_ENABLED + require(SparkSession.getActiveSession.exists(_.sessionState.conf.getConf(requiredReadConf)), + s"${requiredReadConf.key} must be enabled to support Delta id column mapping mode") + val requiredWriteConf = SQLConf.PARQUET_FIELD_ID_WRITE_ENABLED + require(SparkSession.getActiveSession.exists(_.sessionState.conf.getConf(requiredWriteConf)), + s"${requiredWriteConf.key} must be enabled to support Delta id column mapping mode") + } +} diff --git a/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala b/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala index 178f35969cf..3ec00c5cb62 100644 --- a/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-22x/src/main/scala/org/apache/spark/sql/delta/rapids/delta22x/GpuOptimisticTransaction.scala @@ -222,9 +222,11 @@ class GpuOptimisticTransaction }.toMap } - val gpuFileFormat = deltaLog.fileFormat(metadata) match { - case _: DeltaParquetFileFormat => new GpuParquetFileFormat - case f => throw new IllegalStateException(s"file format $f is not supported") + val deltaFileFormat = deltaLog.fileFormat(metadata) + val gpuFileFormat = if (deltaFileFormat.getClass == classOf[DeltaParquetFileFormat]) { + new GpuParquetFileFormat + } else { + throw new IllegalStateException(s"file format $deltaFileFormat is not supported") } try { diff --git a/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/Delta24xProvider.scala b/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/Delta24xProvider.scala index 369e54af4bb..1f21c2a3e02 100644 --- a/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/Delta24xProvider.scala +++ b/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/Delta24xProvider.scala @@ -16,11 +16,15 @@ package com.nvidia.spark.rapids.delta.delta24x -import com.nvidia.spark.rapids.{GpuOverrides, RunnableCommandRule} +import com.nvidia.spark.rapids.{GpuOverrides, GpuReadParquetFileFormat, RunnableCommandRule, SparkPlanMeta} import com.nvidia.spark.rapids.delta.DeltaIOProvider +import org.apache.spark.sql.delta.DeltaParquetFileFormat +import org.apache.spark.sql.delta.DeltaParquetFileFormat.{IS_ROW_DELETED_COLUMN_NAME, ROW_INDEX_COLUMN_NAME} import org.apache.spark.sql.delta.commands.{DeleteCommand, MergeIntoCommand, UpdateCommand} +import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.execution.datasources.FileFormat object Delta24xProvider extends DeltaIOProvider { @@ -41,4 +45,31 @@ object Delta24xProvider extends DeltaIOProvider { .disabledByDefault("Delta Lake update support is experimental") ).map(r => (r.getClassFor.asSubclass(classOf[RunnableCommand]), r)).toMap } + + override def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { + val format = meta.wrapped.relation.fileFormat + if (format.getClass == classOf[DeltaParquetFileFormat]) { + val deltaFormat = format.asInstanceOf[DeltaParquetFileFormat] + val requiredSchema = meta.wrapped.requiredSchema + if (requiredSchema.exists(_.name == IS_ROW_DELETED_COLUMN_NAME)) { + meta.willNotWorkOnGpu( + s"reading metadata column $IS_ROW_DELETED_COLUMN_NAME is not supported") + } + if (requiredSchema.exists(_.name == ROW_INDEX_COLUMN_NAME)) { + meta.willNotWorkOnGpu( + s"reading metadata column $ROW_INDEX_COLUMN_NAME is not supported") + } + if (deltaFormat.hasDeletionVectorMap()) { + meta.willNotWorkOnGpu("deletion vectors are not supported") + } + GpuReadParquetFileFormat.tagSupport(meta) + } else { + meta.willNotWorkOnGpu(s"format ${format.getClass} is not supported") + } + } + + override def getReadFileFormat(format: FileFormat): FileFormat = { + val cpuFormat = format.asInstanceOf[DeltaParquetFileFormat] + GpuDelta24xParquetFileFormat(cpuFormat.metadata, cpuFormat.isSplittable) + } } diff --git a/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/GpuDelta24xParquetFileFormat.scala b/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/GpuDelta24xParquetFileFormat.scala new file mode 100644 index 00000000000..ab8b6afe1ce --- /dev/null +++ b/delta-lake/delta-24x/src/main/scala/com/nvidia/spark/rapids/delta/delta24x/GpuDelta24xParquetFileFormat.scala @@ -0,0 +1,48 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 com.nvidia.spark.rapids.delta.delta24x + +import com.nvidia.spark.rapids.delta.GpuDeltaParquetFileFormat +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.delta.{DeltaColumnMappingMode, IdMapping} +import org.apache.spark.sql.delta.actions.Metadata +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType + +case class GpuDelta24xParquetFileFormat( + metadata: Metadata, + isSplittable: Boolean) extends GpuDeltaParquetFileFormat { + + override val columnMappingMode: DeltaColumnMappingMode = metadata.columnMappingMode + override val referenceSchema: StructType = metadata.schema + + if (columnMappingMode == IdMapping) { + val requiredReadConf = SQLConf.PARQUET_FIELD_ID_READ_ENABLED + require(SparkSession.getActiveSession.exists(_.sessionState.conf.getConf(requiredReadConf)), + s"${requiredReadConf.key} must be enabled to support Delta id column mapping mode") + val requiredWriteConf = SQLConf.PARQUET_FIELD_ID_WRITE_ENABLED + require(SparkSession.getActiveSession.exists(_.sessionState.conf.getConf(requiredWriteConf)), + s"${requiredWriteConf.key} must be enabled to support Delta id column mapping mode") + } + + override def isSplitable( + sparkSession: SparkSession, + options: Map[String, String], + path: Path): Boolean = isSplittable +} diff --git a/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala b/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala index b781235aba6..bc95a656271 100644 --- a/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-24x/src/main/scala/org/apache/spark/sql/delta/rapids/delta24x/GpuOptimisticTransaction.scala @@ -224,9 +224,11 @@ class GpuOptimisticTransaction }.toMap } - val gpuFileFormat = DeltaRuntimeShim.fileFormatFromLog(deltaLog) match { - case _: DeltaParquetFileFormat => new GpuParquetFileFormat - case f => throw new IllegalStateException(s"file format $f is not supported") + val deltaFileFormat = DeltaRuntimeShim.fileFormatFromLog(deltaLog) + val gpuFileFormat = if (deltaFileFormat.getClass == classOf[DeltaParquetFileFormat]) { + new GpuParquetFileFormat + } else { + throw new IllegalStateException(s"file format $deltaFileFormat is not supported") } try { diff --git a/delta-lake/delta-spark321db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala b/delta-lake/delta-spark321db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala index f585d3a3214..03f3592af5a 100644 --- a/delta-lake/delta-spark321db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-spark321db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala @@ -33,6 +33,7 @@ import com.databricks.sql.transaction.tahoe.schema.InvariantViolationException import com.databricks.sql.transaction.tahoe.sources.DeltaSQLConf import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.delta._ +import com.nvidia.spark.rapids.shims.ParquetFieldIdShims import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.fs.Path @@ -198,6 +199,10 @@ class GpuOptimisticTransaction( val hadoopConf = spark.sessionState.newHadoopConfWithOptions( metadata.configuration ++ deltaLog.options) + if (metadata.columnMappingMode == IdMapping) { + // Need Parquet field IDs when doing column ID mapping + ParquetFieldIdShims.setWriteIdOverride(hadoopConf, true) + } if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) { val serializableHadoopConf = new SerializableConfiguration(hadoopConf) @@ -218,9 +223,11 @@ class GpuOptimisticTransaction( case Some(writeOptions) => writeOptions.options } - val gpuFileFormat = deltaLog.fileFormat(metadata) match { - case _: DeltaParquetFileFormat => new GpuParquetFileFormat - case f => throw new IllegalStateException(s"file format $f is not supported") + val deltaFileFormat = deltaLog.fileFormat(metadata) + val gpuFileFormat = if (deltaFileFormat.getClass == classOf[DeltaParquetFileFormat]) { + new GpuParquetFileFormat + } else { + throw new IllegalStateException(s"file format $deltaFileFormat is not supported") } try { diff --git a/delta-lake/delta-spark321db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala b/delta-lake/delta-spark321db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala new file mode 100644 index 00000000000..773da5eb0bf --- /dev/null +++ b/delta-lake/delta-spark321db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala @@ -0,0 +1,36 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 com.nvidia.spark.rapids.delta + +import com.databricks.sql.transaction.tahoe.{DeltaColumnMappingMode, DeltaParquetFileFormat} +import com.nvidia.spark.rapids.SparkPlanMeta + +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.types.StructType + +case class GpuDeltaParquetFileFormat( + override val columnMappingMode: DeltaColumnMappingMode, + override val referenceSchema: StructType) extends GpuDeltaParquetFileFormatBase { +} + +object GpuDeltaParquetFileFormat { + def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = {} + + def convertToGpu(format: DeltaParquetFileFormat): GpuDeltaParquetFileFormat = { + GpuDeltaParquetFileFormat(format.columnMappingMode, format.referenceSchema) + } +} diff --git a/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala b/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala index eb2a58251be..9dff950ca5b 100644 --- a/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-spark330db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala @@ -32,6 +32,7 @@ import com.databricks.sql.transaction.tahoe.schema.InvariantViolationException import com.databricks.sql.transaction.tahoe.sources.DeltaSQLConf import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.delta._ +import com.nvidia.spark.rapids.shims.ParquetFieldIdShims import org.apache.commons.lang3.exception.ExceptionUtils import org.apache.hadoop.fs.Path @@ -210,6 +211,10 @@ class GpuOptimisticTransaction( val hadoopConf = spark.sessionState.newHadoopConfWithOptions( metadata.configuration ++ deltaLog.options) + if (metadata.columnMappingMode == IdMapping) { + // Need Parquet field IDs when doing column ID mapping + ParquetFieldIdShims.setWriteIdOverride(hadoopConf, true) + } if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) { val serializableHadoopConf = new SerializableConfiguration(hadoopConf) @@ -236,9 +241,11 @@ class GpuOptimisticTransaction( }.toMap } - val gpuFileFormat = deltaLog.fileFormat(metadata) match { - case _: DeltaParquetFileFormat => new GpuParquetFileFormat - case f => throw new IllegalStateException(s"file format $f is not supported") + val deltaFileFormat = deltaLog.fileFormat(metadata) + val gpuFileFormat = if (deltaFileFormat.getClass == classOf[DeltaParquetFileFormat]) { + new GpuParquetFileFormat + } else { + throw new IllegalStateException(s"file format $deltaFileFormat is not supported") } try { diff --git a/delta-lake/delta-spark330db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala b/delta-lake/delta-spark330db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala new file mode 100644 index 00000000000..773da5eb0bf --- /dev/null +++ b/delta-lake/delta-spark330db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala @@ -0,0 +1,36 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 com.nvidia.spark.rapids.delta + +import com.databricks.sql.transaction.tahoe.{DeltaColumnMappingMode, DeltaParquetFileFormat} +import com.nvidia.spark.rapids.SparkPlanMeta + +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.types.StructType + +case class GpuDeltaParquetFileFormat( + override val columnMappingMode: DeltaColumnMappingMode, + override val referenceSchema: StructType) extends GpuDeltaParquetFileFormatBase { +} + +object GpuDeltaParquetFileFormat { + def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = {} + + def convertToGpu(format: DeltaParquetFileFormat): GpuDeltaParquetFileFormat = { + GpuDeltaParquetFileFormat(format.columnMappingMode, format.referenceSchema) + } +} diff --git a/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala index e1693e11d09..fb9d3840917 100644 --- a/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala +++ b/delta-lake/delta-spark332db/src/main/scala/com/databricks/sql/transaction/tahoe/rapids/GpuOptimisticTransaction.scala @@ -238,9 +238,11 @@ class GpuOptimisticTransaction( }.toMap } - val gpuFileFormat = deltaLog.fileFormat(metadata) match { - case _: DeltaParquetFileFormat => new GpuParquetFileFormat - case f => throw new IllegalStateException(s"file format $f is not supported") + val deltaFileFormat = deltaLog.fileFormat(metadata) + val gpuFileFormat = if (deltaFileFormat.getClass == classOf[DeltaParquetFileFormat]) { + new GpuParquetFileFormat + } else { + throw new IllegalStateException(s"file format $deltaFileFormat is not supported") } try { diff --git a/delta-lake/delta-spark332db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala b/delta-lake/delta-spark332db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala new file mode 100644 index 00000000000..969d005b573 --- /dev/null +++ b/delta-lake/delta-spark332db/src/main/scala/com/nvidia/spark/rapids/delta/GpuDeltaParquetFileFormat.scala @@ -0,0 +1,65 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 com.nvidia.spark.rapids.delta + +import com.databricks.sql.transaction.tahoe.{DeltaColumnMappingMode, DeltaParquetFileFormat, IdMapping} +import com.databricks.sql.transaction.tahoe.DeltaParquetFileFormat.IS_ROW_DELETED_COLUMN_NAME +import com.nvidia.spark.rapids.SparkPlanMeta +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType + +case class GpuDeltaParquetFileFormat( + override val columnMappingMode: DeltaColumnMappingMode, + override val referenceSchema: StructType, + isSplittable: Boolean) extends GpuDeltaParquetFileFormatBase { + + if (columnMappingMode == IdMapping) { + val requiredReadConf = SQLConf.PARQUET_FIELD_ID_READ_ENABLED + require(SparkSession.getActiveSession.exists(_.sessionState.conf.getConf(requiredReadConf)), + s"${requiredReadConf.key} must be enabled to support Delta id column mapping mode") + val requiredWriteConf = SQLConf.PARQUET_FIELD_ID_WRITE_ENABLED + require(SparkSession.getActiveSession.exists(_.sessionState.conf.getConf(requiredWriteConf)), + s"${requiredWriteConf.key} must be enabled to support Delta id column mapping mode") + } + + override def isSplitable( + sparkSession: SparkSession, + options: Map[String, String], + path: Path): Boolean = isSplittable +} + +object GpuDeltaParquetFileFormat { + def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { + val format = meta.wrapped.relation.fileFormat.asInstanceOf[DeltaParquetFileFormat] + val requiredSchema = meta.wrapped.requiredSchema + if (requiredSchema.exists(_.name == IS_ROW_DELETED_COLUMN_NAME)) { + meta.willNotWorkOnGpu( + s"reading metadata column $IS_ROW_DELETED_COLUMN_NAME is not supported") + } + if (format.hasDeletionVectorMap()) { + meta.willNotWorkOnGpu("deletion vectors are not supported") + } + } + + def convertToGpu(fmt: DeltaParquetFileFormat): GpuDeltaParquetFileFormat = { + GpuDeltaParquetFileFormat(fmt.columnMappingMode, fmt.referenceSchema, fmt.isSplittable) + } +} diff --git a/integration_tests/src/main/python/delta_lake_merge_test.py b/integration_tests/src/main/python/delta_lake_merge_test.py index 27a9575a95e..2f03de2a5b6 100644 --- a/integration_tests/src/main/python/delta_lake_merge_test.py +++ b/integration_tests/src/main/python/delta_lake_merge_test.py @@ -48,30 +48,33 @@ def make_df(spark, gen, num_slices): return three_col_df(spark, gen, SetValuesGen(StringType(), string.ascii_lowercase), SetValuesGen(StringType(), string.ascii_uppercase), num_slices=num_slices) +def setup_dest_table(spark, path, dest_table_func, use_cdf, partition_columns=None, enable_deletion_vectors=False): + dest_df = dest_table_func(spark) + writer = dest_df.write.format("delta") + ddl = schema_to_ddl(spark, dest_df.schema) + table_properties = {} + if use_cdf: + table_properties['delta.enableChangeDataFeed'] = 'true' + if enable_deletion_vectors: + table_properties['delta.enableDeletionVectors'] = 'true' + if len(table_properties) > 0: + # if any table properties are specified then we need to use SQL to define the table + sql_text = "CREATE TABLE delta.`{path}` ({ddl}) USING DELTA".format(path=path, ddl=ddl) + if partition_columns: + sql_text += " PARTITIONED BY ({})".format(",".join(partition_columns)) + properties = ', '.join(key + ' = ' + value for key, value in table_properties.items()) + sql_text += " TBLPROPERTIES ({})".format(properties) + spark.sql(sql_text) + elif partition_columns: + writer = writer.partitionBy(*partition_columns) + if use_cdf or enable_deletion_vectors: + writer = writer.mode("append") + writer.save(path) + def setup_dest_tables(spark, data_path, dest_table_func, use_cdf, partition_columns=None, enable_deletion_vectors=False): for name in ["CPU", "GPU"]: path = "{}/{}".format(data_path, name) - dest_df = dest_table_func(spark) - writer = dest_df.write.format("delta") - ddl = schema_to_ddl(spark, dest_df.schema) - table_properties = {} - if use_cdf: - table_properties['delta.enableChangeDataFeed'] = 'true' - if enable_deletion_vectors: - table_properties['delta.enableDeletionVectors'] = 'true' - if len(table_properties) > 0: - # if any table properties are specified then we need to use SQL to define the table - sql_text = "CREATE TABLE delta.`{path}` ({ddl}) USING DELTA".format(path=path, ddl=ddl) - if partition_columns: - sql_text += " PARTITIONED BY ({})".format(",".join(partition_columns)) - properties = ', '.join(key + ' = ' + value for key, value in table_properties.items()) - sql_text += " TBLPROPERTIES ({})".format(properties) - spark.sql(sql_text) - elif partition_columns: - writer = writer.partitionBy(*partition_columns) - if use_cdf or enable_deletion_vectors: - writer = writer.mode("append") - writer.save(path) + setup_dest_table(spark, path, dest_table_func, use_cdf, partition_columns, enable_deletion_vectors) def delta_sql_merge_test(spark_tmp_path, spark_tmp_table_factory, use_cdf, src_table_func, dest_table_func, merge_sql, check_func, diff --git a/integration_tests/src/main/python/delta_lake_test.py b/integration_tests/src/main/python/delta_lake_test.py index ceba311b174..a7ca70dc64c 100644 --- a/integration_tests/src/main/python/delta_lake_test.py +++ b/integration_tests/src/main/python/delta_lake_test.py @@ -14,9 +14,14 @@ import pytest from pyspark.sql import Row -from asserts import assert_gpu_fallback_collect -from marks import allow_non_gpu, delta_lake -from spark_session import with_cpu_session, with_gpu_session, is_databricks_runtime, spark_version, is_spark_320_or_later +from asserts import assert_gpu_fallback_collect, assert_gpu_and_cpu_are_equal_collect +from data_gen import * +from delta_lake_merge_test import setup_dest_table +from delta_lake_write_test import delta_meta_allow +from marks import allow_non_gpu, delta_lake, ignore_order +from parquet_test import reader_opt_confs_no_native +from spark_session import with_cpu_session, with_gpu_session, is_databricks_runtime, \ + is_spark_320_or_later, is_spark_340_or_later, supports_delta_lake_deletion_vectors _conf = {'spark.rapids.sql.explain': 'ALL'} @@ -62,4 +67,57 @@ def merge(spark): with_gpu_session(lambda spark : merge(spark), conf = _conf) # check the results on CPU result = with_cpu_session(lambda spark: spark.sql("SELECT * FROM t1 ORDER BY c0").collect(), conf=_conf) - assert [Row(c0='a', c1=40), Row(c0='b', c1=20), Row(c0='c', c1=30)] == result \ No newline at end of file + assert [Row(c0='a', c1=40), Row(c0='b', c1=20), Row(c0='c', c1=30)] == result + +@allow_non_gpu("FileSourceScanExec", "ColumnarToRowExec", *delta_meta_allow) +@delta_lake +@ignore_order(local=True) +@pytest.mark.parametrize("use_cdf", [True, False], ids=idfn) +@pytest.mark.skipif(not supports_delta_lake_deletion_vectors(), + reason="Delta Lake deletion vector support is required") +def test_delta_deletion_vector_read_fallback(spark_tmp_path, use_cdf): + data_path = spark_tmp_path + "/DELTA_DATA" + conf = {"spark.databricks.delta.delete.deletionVectors.persistent": "true"} + def setup_tables(spark): + setup_dest_table(spark, data_path, + dest_table_func=lambda spark: unary_op_df(spark, int_gen), + use_cdf=use_cdf, enable_deletion_vectors=True) + spark.sql("INSERT INTO delta.`{}` VALUES(1)".format(data_path)) + spark.sql("DELETE FROM delta.`{}` WHERE a = 1".format(data_path)) + with_cpu_session(setup_tables, conf=conf) + assert_gpu_fallback_collect( + lambda spark: spark.sql("SELECT * FROM delta.`{}`".format(data_path)), + "FileSourceScanExec", + conf=conf) + +# ID mapping is supported starting in Delta Lake 2.2, but currently cannot distinguish +# Delta Lake 2.1 from 2.2 in tests. https://github.com/NVIDIA/spark-rapids/issues/9276 +column_mappings = ["name", "id"] +if is_spark_340_or_later() or is_databricks_runtime(): + column_mappings.append("id") + +@allow_non_gpu(*delta_meta_allow) +@delta_lake +@ignore_order(local=True) +@pytest.mark.parametrize("reader_confs", reader_opt_confs_no_native, ids=idfn) +@pytest.mark.parametrize("mapping", column_mappings, ids=idfn) +def test_delta_read_column_mapping(spark_tmp_path, reader_confs, mapping): + data_path = spark_tmp_path + "/DELTA_DATA" + gen_list = [("a", int_gen), + ("b", SetValuesGen(StringType(), ["x", "y", "z"])), + ("c", string_gen), + ("d", SetValuesGen(IntegerType(), [1, 2, 3])), + ("e", long_gen)] + confs = copy_and_update(reader_confs, { + "spark.databricks.delta.properties.defaults.columnMapping.mode": mapping, + "spark.databricks.delta.properties.defaults.minReaderVersion": "2", + "spark.databricks.delta.properties.defaults.minWriterVersion": "5", + "spark.sql.parquet.fieldId.read.enabled": "true" + }) + with_cpu_session( + lambda spark: gen_df(spark, gen_list).coalesce(1).write.format("delta") \ + .partitionBy("b", "d") \ + .save(data_path), + conf=confs) + assert_gpu_and_cpu_are_equal_collect(lambda spark: spark.read.format("delta").load(data_path), + conf=confs) diff --git a/integration_tests/src/main/python/delta_lake_write_test.py b/integration_tests/src/main/python/delta_lake_write_test.py index 2dceb0fe275..94e7c529bb1 100644 --- a/integration_tests/src/main/python/delta_lake_write_test.py +++ b/integration_tests/src/main/python/delta_lake_write_test.py @@ -25,7 +25,7 @@ from marks import * from parquet_write_test import parquet_part_write_gens, parquet_write_gens_list, writer_confs from pyspark.sql.types import * -from spark_session import is_before_spark_320, is_before_spark_330, is_databricks122_or_later, with_cpu_session +from spark_session import is_before_spark_320, is_before_spark_330, is_databricks122_or_later, is_spark_340_or_later, with_cpu_session delta_meta_allow = [ "DeserializeToObjectExec", @@ -932,3 +932,35 @@ def test_delta_write_partial_overwrite_replace_where(spark_tmp_path): data_path, conf=copy_and_update(writer_confs, delta_writes_enabled_conf)) with_cpu_session(lambda spark: assert_gpu_and_cpu_delta_logs_equivalent(spark, data_path)) + +# ID mapping is supported starting in Delta Lake 2.2, but currently cannot distinguish +# Delta Lake 2.1 from 2.2 in tests. https://github.com/NVIDIA/spark-rapids/issues/9276 +column_mappings = ["name"] +if is_spark_340_or_later() or is_databricks_runtime(): + column_mappings.append("id") + +@allow_non_gpu(*delta_meta_allow) +@delta_lake +@ignore_order +@pytest.mark.parametrize("mapping", column_mappings) +@pytest.mark.skipif(is_before_spark_320(), reason="Delta Lake writes are not supported before Spark 3.2.x") +def test_delta_write_column_name_mapping(spark_tmp_path, mapping): + gen_list = [("a", int_gen), + ("b", SetValuesGen(StringType(), ["x", "y", "z"])), + ("c", string_gen), + ("d", SetValuesGen(IntegerType(), [1, 2, 3])), + ("e", long_gen)] + data_path = spark_tmp_path + "/DELTA_DATA" + confs = copy_and_update(writer_confs, delta_writes_enabled_conf, { + "spark.databricks.delta.properties.defaults.columnMapping.mode": mapping, + "spark.databricks.delta.properties.defaults.minReaderVersion": "2", + "spark.databricks.delta.properties.defaults.minWriterVersion": "5", + "spark.sql.parquet.fieldId.read.enabled": "true" + }) + assert_gpu_and_cpu_writes_are_equal_collect( + lambda spark, path: gen_df(spark, gen_list).coalesce(1).write.format("delta") \ + .partitionBy("b", "d") \ + .save(path), + lambda spark, path: spark.read.format("delta").load(path), + data_path, + conf=confs) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AvroProvider.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AvroProvider.scala index 05ea12d086e..3355a270b77 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AvroProvider.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/AvroProvider.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.execution.datasources.FileFormat trait AvroProvider { /** If the file format is supported as an external source */ - def isSupportedFormat(format: FileFormat): Boolean + def isSupportedFormat(format: Class[_ <: FileFormat]): Boolean def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index b3db1b6168c..a5bda29670f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -324,24 +324,23 @@ final class InsertIntoHadoopFsRelationCommandMeta( } val spark = SparkSession.active - - fileFormat = cmd.fileFormat match { - case _: CSVFileFormat => - willNotWorkOnGpu("CSV output is not supported") - None - case _: JsonFileFormat => - willNotWorkOnGpu("JSON output is not supported") - None - case f if GpuOrcFileFormat.isSparkOrcFormat(f) => - GpuOrcFileFormat.tagGpuSupport(this, spark, cmd.options, cmd.query.schema) - case _: ParquetFileFormat => - GpuParquetFileFormat.tagGpuSupport(this, spark, cmd.options, cmd.query.schema) - case _: TextFileFormat => - willNotWorkOnGpu("text output is not supported") - None - case f => - willNotWorkOnGpu(s"unknown file format: ${f.getClass.getCanonicalName}") - None + val formatCls = cmd.fileFormat.getClass + fileFormat = if (formatCls == classOf[CSVFileFormat]) { + willNotWorkOnGpu("CSV output is not supported") + None + } else if (formatCls == classOf[JsonFileFormat]) { + willNotWorkOnGpu("JSON output is not supported") + None + } else if (GpuOrcFileFormat.isSparkOrcFormat(formatCls)) { + GpuOrcFileFormat.tagGpuSupport(this, spark, cmd.options, cmd.query.schema) + } else if (formatCls == classOf[ParquetFileFormat]) { + GpuParquetFileFormat.tagGpuSupport(this, spark, cmd.options, cmd.query.schema) + } else if (formatCls == classOf[TextFileFormat]) { + willNotWorkOnGpu("text output is not supported") + None + } else { + willNotWorkOnGpu(s"unknown file format: ${formatCls.getCanonicalName}") + None } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala index 0bed9b47b4f..cff38e3e3ec 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuParquetFileFormat.scala @@ -250,7 +250,7 @@ class GpuParquetFileFormat extends ColumnarFileFormat with Logging { conf.set(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key, outputTimestampType.toString) ParquetFieldIdShims.setupParquetFieldIdWriteConfig(conf, sqlConf) - val parquetFieldIdWriteEnabled = ParquetFieldIdShims.getParquetIdWriteEnabled(sqlConf) + val parquetFieldIdWriteEnabled = ParquetFieldIdShims.getParquetIdWriteEnabled(conf, sqlConf) ParquetTimestampNTZShims.setupTimestampNTZConfig(conf, sqlConf) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/delta/DeltaProvider.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/delta/DeltaProvider.scala index 9301f992565..7e521bd4829 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/delta/DeltaProvider.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/delta/DeltaProvider.scala @@ -16,11 +16,12 @@ package com.nvidia.spark.rapids.delta -import com.nvidia.spark.rapids.{CreatableRelationProviderRule, ExecRule, RunnableCommandRule, ShimLoader} +import com.nvidia.spark.rapids.{CreatableRelationProviderRule, ExecRule, RunnableCommandRule, ShimLoader, SparkPlanMeta} import org.apache.spark.sql.Strategy -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.command.RunnableCommand +import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.sources.CreatableRelationProvider /** Probe interface to determine which Delta Lake provider to use. */ @@ -39,6 +40,12 @@ trait DeltaProvider { RunnableCommandRule[_ <: RunnableCommand]] def getStrategyRules: Seq[Strategy] + + def isSupportedFormat(format: Class[_ <: FileFormat]): Boolean + + def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit + + def getReadFileFormat(format: FileFormat): FileFormat } object DeltaProvider { @@ -59,4 +66,12 @@ object NoDeltaProvider extends DeltaProvider { RunnableCommandRule[_ <: RunnableCommand]] = Map.empty override def getStrategyRules: Seq[Strategy] = Nil + + override def isSupportedFormat(format: Class[_ <: FileFormat]): Boolean = false + + override def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = + throw new IllegalStateException("unsupported format") + + override def getReadFileFormat(format: FileFormat): FileFormat = + throw new IllegalStateException("unsupported format") } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AvroProviderImpl.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AvroProviderImpl.scala index 0f0618ce8fb..2d8dd1e2a43 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AvroProviderImpl.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AvroProviderImpl.scala @@ -30,26 +30,16 @@ import org.apache.spark.util.SerializableConfiguration class AvroProviderImpl extends AvroProvider { /** If the file format is supported as an external source */ - def isSupportedFormat(format: FileFormat): Boolean = { - format match { - case _: AvroFileFormat => true - case _ => false - } + def isSupportedFormat(format: Class[_ <: FileFormat]): Boolean = { + format == classOf[AvroFileFormat] } def isPerFileReadEnabledForFormat(format: FileFormat, conf: RapidsConf): Boolean = { - format match { - case _: AvroFileFormat => conf.isAvroPerFileReadEnabled - case _ => false - } + conf.isAvroPerFileReadEnabled } def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { - meta.wrapped.relation.fileFormat match { - case _: AvroFileFormat => GpuReadAvroFileFormat.tagSupport(meta) - case f => - meta.willNotWorkOnGpu(s"unsupported file format: ${f.getClass.getCanonicalName}") - } + GpuReadAvroFileFormat.tagSupport(meta) } /** @@ -57,11 +47,8 @@ class AvroProviderImpl extends AvroProvider { * Better to check if the format is supported first by calling 'isSupportedFormat' */ def getReadFileFormat(format: FileFormat): FileFormat = { - format match { - case _: AvroFileFormat => new GpuReadAvroFileFormat - case f => - throw new IllegalArgumentException(s"${f.getClass.getCanonicalName} is not supported") - } + require(isSupportedFormat(format.getClass), s"unexpected format: $format") + new GpuReadAvroFileFormat } /** @@ -73,23 +60,17 @@ class AvroProviderImpl extends AvroProvider { broadcastedConf: Broadcast[SerializableConfiguration], pushedFilters: Array[Filter], fileScan: GpuFileSourceScanExec): PartitionReaderFactory = { - format match { - case _: AvroFileFormat => - GpuAvroMultiFilePartitionReaderFactory( - fileScan.relation.sparkSession.sessionState.conf, - fileScan.rapidsConf, - broadcastedConf, - fileScan.relation.dataSchema, - fileScan.requiredSchema, - fileScan.readPartitionSchema, - new AvroOptions(fileScan.relation.options, broadcastedConf.value.value), - fileScan.allMetrics, - pushedFilters, - fileScan.queryUsesInputFile) - case _ => - // never reach here - throw new RuntimeException(s"File format $format is not supported yet") - } + GpuAvroMultiFilePartitionReaderFactory( + fileScan.relation.sparkSession.sessionState.conf, + fileScan.rapidsConf, + broadcastedConf, + fileScan.relation.dataSchema, + fileScan.requiredSchema, + fileScan.readPartitionSchema, + new AvroOptions(fileScan.relation.options, broadcastedConf.value.value), + fileScan.allMetrics, + pushedFilters, + fileScan.queryUsesInputFile) } def getScans: Map[Class[_ <: Scan], ScanRule[_ <: Scan]] = { diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExternalSource.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExternalSource.scala index edf7317dec3..fe560551aa3 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExternalSource.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/ExternalSource.scala @@ -72,15 +72,24 @@ object ExternalSource extends Logging { deltaProvider.getExecRules /** If the file format is supported as an external source */ - def isSupportedFormat(format: FileFormat): Boolean = { - if (hasSparkAvroJar) { - avroProvider.isSupportedFormat(format) - } else false + def isSupportedFormat(format: Class[_ <: FileFormat]): Boolean = { + if (hasSparkAvroJar && avroProvider.isSupportedFormat(format)) { + true + } else if (deltaProvider.isSupportedFormat(format)) { + true + } else { + false + } } def tagSupportForGpuFileSourceScan(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { - if (hasSparkAvroJar) { + val format = meta.wrapped.relation.fileFormat + if (hasSparkAvroJar && avroProvider.isSupportedFormat(format.getClass)) { avroProvider.tagSupportForGpuFileSourceScan(meta) + } else if (deltaProvider.isSupportedFormat(format.getClass)) { + deltaProvider.tagSupportForGpuFileSourceScan(meta) + } else { + meta.willNotWorkOnGpu(s"unsupported file format: ${format.getClass.getCanonicalName}") } } @@ -89,8 +98,10 @@ object ExternalSource extends Logging { * Better to check if the format is supported first by calling 'isSupportedFormat' */ def getReadFileFormat(format: FileFormat): FileFormat = { - if (hasSparkAvroJar) { + if (hasSparkAvroJar && avroProvider.isSupportedFormat(format.getClass)) { avroProvider.getReadFileFormat(format) + } else if (deltaProvider.isSupportedFormat(format.getClass)) { + deltaProvider.getReadFileFormat(format) } else { throw new IllegalArgumentException(s"${format.getClass.getCanonicalName} is not supported") } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala index 2d5d5cae895..80acf2b0f9e 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuFileSourceScanExec.scala @@ -651,28 +651,36 @@ case class GpuFileSourceScanExec( object GpuFileSourceScanExec { def tagSupport(meta: SparkPlanMeta[FileSourceScanExec]): Unit = { - meta.wrapped.relation.fileFormat match { - case _: CSVFileFormat => GpuReadCSVFileFormat.tagSupport(meta) - case f if GpuOrcFileFormat.isSparkOrcFormat(f) => GpuReadOrcFileFormat.tagSupport(meta) - case _: ParquetFileFormat => GpuReadParquetFileFormat.tagSupport(meta) - case _: JsonFileFormat => GpuReadJsonFileFormat.tagSupport(meta) - case ef if ExternalSource.isSupportedFormat(ef) => - ExternalSource.tagSupportForGpuFileSourceScan(meta) - case other => - meta.willNotWorkOnGpu(s"unsupported file format: ${other.getClass.getCanonicalName}") + val cls = meta.wrapped.relation.fileFormat.getClass + if (cls == classOf[CSVFileFormat]) { + GpuReadCSVFileFormat.tagSupport(meta) + } else if (GpuOrcFileFormat.isSparkOrcFormat(cls)) { + GpuReadOrcFileFormat.tagSupport(meta) + } else if (cls == classOf[ParquetFileFormat]) { + GpuReadParquetFileFormat.tagSupport(meta) + } else if (cls == classOf[JsonFileFormat]) { + GpuReadJsonFileFormat.tagSupport(meta) + } else if (ExternalSource.isSupportedFormat(cls)) { + ExternalSource.tagSupportForGpuFileSourceScan(meta) + } else { + meta.willNotWorkOnGpu(s"unsupported file format: ${cls.getCanonicalName}") } } def convertFileFormat(format: FileFormat): FileFormat = { - format match { - case _: CSVFileFormat => new GpuReadCSVFileFormat - case f if GpuOrcFileFormat.isSparkOrcFormat(f) => new GpuReadOrcFileFormat - case _: ParquetFileFormat => new GpuReadParquetFileFormat - case _: JsonFileFormat => new GpuReadJsonFileFormat - case ef if ExternalSource.isSupportedFormat(ef) => ExternalSource.getReadFileFormat(ef) - case other => - throw new IllegalArgumentException(s"${other.getClass.getCanonicalName} is not supported") - + val cls = format.getClass + if (cls == classOf[CSVFileFormat]) { + new GpuReadCSVFileFormat + } else if (GpuOrcFileFormat.isSparkOrcFormat(cls)) { + new GpuReadOrcFileFormat + } else if (cls == classOf[ParquetFileFormat]) { + new GpuReadParquetFileFormat + } else if (cls == classOf[JsonFileFormat]) { + new GpuReadJsonFileFormat + } else if (ExternalSource.isSupportedFormat(cls)) { + ExternalSource.getReadFileFormat(format) + } else { + throw new IllegalArgumentException(s"${cls.getCanonicalName} is not supported") } } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala index 354245e2a75..9e50a993537 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuOrcFileFormat.scala @@ -37,10 +37,8 @@ object GpuOrcFileFormat extends Logging { // Spark is not always compiled with Hive support so we cannot import from Spark jars directly. private val HIVE_IMPL_CLASS = "org.apache.spark.sql.hive.orc.OrcFileFormat" - def isSparkOrcFormat(format: FileFormat): Boolean = format match { - case _: OrcFileFormat => true - case f if f.getClass.getCanonicalName.equals(HIVE_IMPL_CLASS) => true - case _ => false + def isSparkOrcFormat(cls: Class[_ <: FileFormat]): Boolean = { + cls == classOf[OrcFileFormat] || cls.getCanonicalName.equals(HIVE_IMPL_CLASS) } def tagGpuSupport(meta: RapidsMeta[_, _, _], diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala index a8f054e60c2..ae510d803c5 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala @@ -67,15 +67,15 @@ final class CreateDataSourceTableAsSelectCommandMeta( cmd.table.provider.get, spark.sessionState.conf) // Note that the data source V2 always fallsback to the V1 currently. // If that changes then this will start failing because we don't have a mapping. - gpuProvider = origProvider.getConstructor().newInstance() match { - case f: FileFormat if GpuOrcFileFormat.isSparkOrcFormat(f) => - GpuOrcFileFormat.tagGpuSupport(this, spark, cmd.table.storage.properties, cmd.query.schema) - case _: ParquetFileFormat => - GpuParquetFileFormat.tagGpuSupport(this, spark, - cmd.table.storage.properties, cmd.query.schema) - case ds => - willNotWorkOnGpu(s"Data source class not supported: ${ds}") - None + gpuProvider = if (classOf[FileFormat].isAssignableFrom(origProvider) && + GpuOrcFileFormat.isSparkOrcFormat(origProvider.asInstanceOf[Class[_ <: FileFormat]])) { + GpuOrcFileFormat.tagGpuSupport(this, spark, cmd.table.storage.properties, cmd.query.schema) + } else if (origProvider == classOf[ParquetFileFormat]) { + GpuParquetFileFormat.tagGpuSupport(this, spark, + cmd.table.storage.properties, cmd.query.schema) + } else { + willNotWorkOnGpu(s"Data source class not supported: $origProvider") + None } } diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala index 3368dcd6e46..340dce31388 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala @@ -21,7 +21,6 @@ {"spark": "320"} {"spark": "321"} {"spark": "321cdh"} -{"spark": "321db"} {"spark": "322"} {"spark": "323"} {"spark": "324"} @@ -39,5 +38,5 @@ object ParquetFieldIdShims { } /** Parquet field ID support configs are not supported until Spark 3.3 */ - def getParquetIdWriteEnabled(sqlConf: SQLConf): Boolean = false + def getParquetIdWriteEnabled(conf: Configuration, sqlConf: SQLConf): Boolean = false } diff --git a/sql-plugin/src/main/spark321db/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala b/sql-plugin/src/main/spark321db/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala new file mode 100644 index 00000000000..7fffdfb4a9f --- /dev/null +++ b/sql-plugin/src/main/spark321db/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala @@ -0,0 +1,44 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "321db"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.sql.internal.SQLConf + +object ParquetFieldIdShims { + val fieldIdOverrideKey: String = "spark.rapids.sql.parquet.writeFieldIds" + + /** Updates the Hadoop configuration with the Parquet field ID write setting from SQLConf */ + def setupParquetFieldIdWriteConfig(conf: Configuration, sqlConf: SQLConf): Unit = { + // No SQL conf for this config in Spark 3.2.x + } + + /** Get Parquet field ID write enabled configuration value */ + def getParquetIdWriteEnabled(conf: Configuration, sqlConf: SQLConf): Boolean = { + // No SQL conf for this config in Spark 3.2.x + conf.get(fieldIdOverrideKey, "false").toBoolean + } + + /** Set the Parquet field ID write enable override */ + def setWriteIdOverride(conf: Configuration, enabled: Boolean): Unit = { + conf.set(fieldIdOverrideKey, enabled.toString) + } +} diff --git a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala index 22991261f76..1d6e06985a6 100644 --- a/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala +++ b/sql-plugin/src/main/spark330/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala @@ -17,7 +17,6 @@ /*** spark-rapids-shim-json-lines {"spark": "330"} {"spark": "330cdh"} -{"spark": "330db"} {"spark": "331"} {"spark": "332"} {"spark": "332db"} @@ -41,5 +40,7 @@ object ParquetFieldIdShims { } /** Get Parquet field ID write enabled configuration value */ - def getParquetIdWriteEnabled(sqlConf: SQLConf): Boolean = sqlConf.parquetFieldIdWriteEnabled + def getParquetIdWriteEnabled(conf: Configuration, sqlConf: SQLConf): Boolean = { + sqlConf.parquetFieldIdWriteEnabled + } } diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala new file mode 100644 index 00000000000..0b896714903 --- /dev/null +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/ParquetFieldIdShims.scala @@ -0,0 +1,45 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "330db"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.sql.internal.SQLConf + +object ParquetFieldIdShims { + private val fieldIdOverrideKey: String = "spark.rapids.sql.parquet.writeFieldIds" + + /** Updates the Hadoop configuration with the Parquet field ID write setting from SQLConf */ + def setupParquetFieldIdWriteConfig(conf: Configuration, sqlConf: SQLConf): Unit = { + conf.set( + SQLConf.PARQUET_FIELD_ID_WRITE_ENABLED.key, + sqlConf.parquetFieldIdWriteEnabled.toString) + } + + /** Get Parquet field ID write enabled configuration value */ + def getParquetIdWriteEnabled(conf: Configuration, sqlConf: SQLConf): Boolean = { + conf.get(fieldIdOverrideKey, "false").toBoolean || sqlConf.parquetFieldIdWriteEnabled + } + + /** Set the Parquet field ID write enable override */ + def setWriteIdOverride(conf: Configuration, enabled: Boolean): Unit = { + conf.set(fieldIdOverrideKey, enabled.toString) + } +} diff --git a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala index c553f0dada8..8b060743d22 100644 --- a/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala +++ b/sql-plugin/src/main/spark332db/scala/com/nvidia/spark/rapids/shims/CreateDataSourceTableAsSelectCommandMetaShims.scala @@ -52,15 +52,14 @@ final class CreateDataSourceTableAsSelectCommandMeta( origProvider = GpuDataSourceBase.lookupDataSourceWithFallback(cmd.table.provider.get, spark.sessionState.conf) - origProvider.getConstructor().newInstance() match { - case f: FileFormat if GpuOrcFileFormat.isSparkOrcFormat(f) => - GpuOrcFileFormat.tagGpuSupport(this, spark, cmd.table.storage.properties, cmd.query.schema) - case _: ParquetFileFormat => - GpuParquetFileFormat.tagGpuSupport(this, spark, - cmd.table.storage.properties, cmd.query.schema) - case ds => - willNotWorkOnGpu(s"Data source class not supported: ${ds}") - None + if (classOf[FileFormat].isAssignableFrom(origProvider) && + GpuOrcFileFormat.isSparkOrcFormat(origProvider.asInstanceOf[Class[_ <: FileFormat]])) { + GpuOrcFileFormat.tagGpuSupport(this, spark, cmd.table.storage.properties, cmd.query.schema) + } else if (origProvider == classOf[ParquetFileFormat]) { + GpuParquetFileFormat.tagGpuSupport(this, spark, + cmd.table.storage.properties, cmd.query.schema) + } else { + willNotWorkOnGpu(s"Data source class not supported: $origProvider") } } From 056c187b40e13324ad0becd44b861b88d24ae1ca Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Fri, 22 Sep 2023 17:16:33 -0700 Subject: [PATCH 25/46] Add CastOptions to make GpuCast extendible to handle more options [databricks] (#9285) * Refactored GpuCast to take a CastOptions to make it easer to pass in options * singing off --------- Signed-off-by: Raza Jafri --- .../spark/source/GpuIcebergReader.java | 3 +- .../rapids/GpuApproximatePercentile.scala | 8 +- .../com/nvidia/spark/rapids/GpuCast.scala | 212 +++++++++++------- .../com/nvidia/spark/rapids/GpuOrcScan.scala | 3 +- .../spark/sql/rapids/GpuJsonToStructs.scala | 2 +- .../apache/spark/sql/rapids/arithmetic.scala | 27 ++- .../sql/rapids/datetimeExpressions.scala | 2 +- .../apache/spark/sql/rapids/arithmetic.scala | 12 +- 8 files changed, 167 insertions(+), 102 deletions(-) diff --git a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuIcebergReader.java b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuIcebergReader.java index e624852e19d..12236dcd54d 100644 --- a/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuIcebergReader.java +++ b/sql-plugin/src/main/java/com/nvidia/spark/rapids/iceberg/spark/source/GpuIcebergReader.java @@ -23,6 +23,7 @@ import java.util.NoSuchElementException; import ai.rapids.cudf.Scalar; +import com.nvidia.spark.rapids.CastOptions$; import com.nvidia.spark.rapids.GpuCast; import com.nvidia.spark.rapids.GpuColumnVector; import com.nvidia.spark.rapids.GpuScalar; @@ -158,7 +159,7 @@ static ColumnarBatch addUpcastsIfNeeded(ColumnarBatch batch, Schema expectedSche GpuColumnVector oldColumn = columns[i]; columns[i] = GpuColumnVector.from( GpuCast.doCast(oldColumn.getBase(), oldColumn.dataType(), expectedSparkType, - false, false, false), expectedSparkType); + CastOptions$.MODULE$.DEFAULT_CAST_OPTIONS()), expectedSparkType); } ColumnarBatch newBatch = new ColumnarBatch(columns, batch.numRows()); columns = null; diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuApproximatePercentile.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuApproximatePercentile.scala index 2c3c98d20b9..1a188d47660 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuApproximatePercentile.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuApproximatePercentile.scala @@ -144,9 +144,7 @@ case class ApproxPercentileFromTDigestExpr( // array and return that (after converting from Double to finalDataType) withResource(cv.getBase.approxPercentile(Array(p))) { percentiles => withResource(percentiles.extractListElement(0)) { childView => - withResource(doCast(childView, DataTypes.DoubleType, finalDataType, - ansiMode = false, legacyCastToString = false, - stringToDateAnsiModeEnabled = false)) { childCv => + withResource(doCast(childView, DataTypes.DoubleType, finalDataType)) { childCv => GpuColumnVector.from(childCv.copyToColumnVector(), dataType) } } @@ -159,9 +157,7 @@ case class ApproxPercentileFromTDigestExpr( GpuColumnVector.from(percentiles.incRefCount(), dataType) } else { withResource(percentiles.getChildColumnView(0)) { childView => - withResource(doCast(childView, DataTypes.DoubleType, finalDataType, - ansiMode = false, legacyCastToString = false, - stringToDateAnsiModeEnabled = false)) { childCv => + withResource(doCast(childView, DataTypes.DoubleType, finalDataType)) { childCv => withResource(percentiles.replaceListChild(childCv)) { x => GpuColumnVector.from(x.copyToColumnVector(), dataType) } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala index 573f3ced068..7a4bdb592b4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala @@ -163,6 +163,82 @@ final class CastExprMeta[INPUT <: UnaryExpression with TimeZoneAwareExpression w override protected val needTimezoneTagging: Boolean = false } +object CastOptions { + val DEFAULT_CAST_OPTIONS = new CastOptions(false, false, false) + val ARITH_ANSI_OPTIONS = new CastOptions(false, true, false) + + def getArithmeticCastOptions(failOnError: Boolean): CastOptions = + if (failOnError) ARITH_ANSI_OPTIONS else DEFAULT_CAST_OPTIONS +} + +/** + * This class is used to encapsulate parameters to use to help determine how to + * cast + * + * @param legacyCastComplexTypesToString If we should use legacy casting method + * @param ansiMode Whether the cast should be ANSI compliant + * @param stringToDateAnsiMode Whether to cast String to Date using ANSI compliance + */ +class CastOptions( + legacyCastComplexTypesToString: Boolean, + ansiMode: Boolean, + stringToDateAnsiMode: Boolean) extends Serializable { + + /** + * Retuns the left bracket to use when surrounding brackets when converting + * map or struct types to string + * example: + * [ "a" -> "b"] when legacyCastComplexTypesToString is enabled + * otherwise { "a" -> "b" } + */ + val leftBracket: String = if (legacyCastComplexTypesToString) "[" else "{" + + /** + * Returns the right bracket to use when surrounding brackets when converting + * map or struct types to string + * example: + * [ "a" -> "b"] when legacyCastComplexTypesToString is enabled + * otherwise { "a" -> "b" } + */ + val rightBracket: String = if (legacyCastComplexTypesToString) "]" else "}" + + /** + * Returns the string value to use to represent null elements in array/struct/map. + */ + val nullString: String = if (legacyCastComplexTypesToString) "" else "null" + + /** + * Returns whether a decimal value with exponents should be + * converted to a plain string, exactly like Java BigDecimal.toPlainString() + * example: + * plain string value of decimal 1.23E+7 is 12300000 + */ + val useDecimalPlainString: Boolean = ansiMode + + /** + * Returns whether the binary data should be printed as hex values + * instead of ascii values + */ + val useHexFormatForBinary: Boolean = false + + /** + * Returns whether we should cast using ANSI compliance + */ + val isAnsiMode: Boolean = ansiMode + + /** + * Returns whether we should use ANSI compliance when casting a String + * to Date + */ + val useAnsiStringToDateMode: Boolean = stringToDateAnsiMode + + /** + * Returns whether we should use legacy behavior to convert complex types + * like structs/maps to a String + */ + val useLegacyComplexTypesToString: Boolean = legacyCastComplexTypesToString +} + object GpuCast { private val DATE_REGEX_YYYY_MM_DD = "\\A\\d{4}\\-\\d{1,2}\\-\\d{1,2}([ T](:?[\\r\\n]|.)*)?\\Z" @@ -191,14 +267,13 @@ object GpuCast { input: ColumnView, fromDataType: DataType, toDataType: DataType, - ansiMode: Boolean, - legacyCastToString: Boolean, - stringToDateAnsiModeEnabled: Boolean): ColumnVector = { - + options: CastOptions = CastOptions.DEFAULT_CAST_OPTIONS): ColumnVector = { if (DataType.equalsStructurally(fromDataType, toDataType)) { return input.copyToColumnVector() } + val ansiMode = options.isAnsiMode + (fromDataType, toDataType) match { case (NullType, to) => GpuColumnVector.columnVectorFromNull(input.getRowCount.toInt, to) @@ -249,8 +324,7 @@ object GpuCast { castTimestampToString(input) case (StructType(fields), StringType) => - castStructToString(input, fields, ansiMode, legacyCastToString, - stringToDateAnsiModeEnabled) + castStructToString(input, fields, options) // ansi cast from larger-than-long integral-like types, to long case (dt: DecimalType, LongType) if ansiMode => @@ -441,7 +515,7 @@ object GpuCast { case BooleanType => castStringToBool(trimmed, ansiMode) case DateType => - if (stringToDateAnsiModeEnabled) { + if (options.useAnsiStringToDateMode) { castStringToDateAnsi(trimmed, ansiMode) } else { castStringToDate(trimmed) @@ -467,26 +541,22 @@ object GpuCast { case (ArrayType(nestedFrom, _), ArrayType(nestedTo, _)) => withResource(input.getChildColumnView(0)) { childView => - withResource(doCast(childView, nestedFrom, nestedTo, - ansiMode, legacyCastToString, stringToDateAnsiModeEnabled)) { childColumnVector => + withResource(doCast(childView, nestedFrom, nestedTo, options)) { childColumnVector => withResource(input.replaceListChild(childColumnVector))(_.copyToColumnVector()) } } case (ArrayType(elementType, _), StringType) => - castArrayToString( - input, elementType, ansiMode, legacyCastToString, stringToDateAnsiModeEnabled - ) + castArrayToString(input, elementType, options) case (from: StructType, to: StructType) => - castStructToStruct(from, to, input, ansiMode, legacyCastToString, - stringToDateAnsiModeEnabled) + castStructToStruct(from, to, input, options) case (from: MapType, to: MapType) => - castMapToMap(from, to, input, ansiMode, legacyCastToString, stringToDateAnsiModeEnabled) + castMapToMap(from, to, input, options) case (from: MapType, _: StringType) => - castMapToString(input, from, ansiMode, legacyCastToString, stringToDateAnsiModeEnabled) + castMapToString(input, from, options) case (dayTime: DataType, _: StringType) if GpuTypeShims.isSupportedDayTimeType(dayTime) => GpuIntervalUtils.toDayTimeIntervalString(input, dayTime) @@ -684,16 +754,21 @@ object GpuCast { */ private def concatenateStringArrayElements( input: ColumnView, - legacyCastToString: Boolean): ColumnVector = { + options: CastOptions, + castingBinaryData: Boolean = false): ColumnVector = { + + import options._ + val emptyStr = "" val spaceStr = " " - val nullStr = if (legacyCastToString) "" else "null" - val sepStr = if (legacyCastToString) "," else ", " + val sepStr = if (useHexFormatForBinary && castingBinaryData) spaceStr + else if (useLegacyComplexTypesToString) "," else ", " + withResource( - Seq(emptyStr, spaceStr, nullStr, sepStr).safeMap(Scalar.fromString) - ){ case Seq(empty, space, nullRep, sep) => + Seq(emptyStr, spaceStr, nullString, sepStr).safeMap(Scalar.fromString) + ) { case Seq(empty, space, nullRep, sep) => - val withSpacesIfLegacy = if (!legacyCastToString) { + val withSpacesIfLegacy = if (!useLegacyComplexTypesToString) { withResource(input.getChildColumnView(0)) { _.replaceNulls(nullRep) } @@ -724,7 +799,7 @@ object GpuCast { val strCol = withResource(concatenated) { _.replaceNulls(empty) } - if (!legacyCastToString) { + if (!useLegacyComplexTypesToString) { strCol } else { // If the first char of a string is ' ', remove it (only for legacyCastToString = true) @@ -741,26 +816,24 @@ object GpuCast { private def castArrayToString(input: ColumnView, elementType: DataType, - ansiMode: Boolean, - legacyCastToString: Boolean, - stringToDateAnsiModeEnabled: Boolean): ColumnVector = { + options: CastOptions): ColumnVector = { - val (leftStr, rightStr) = ("[", "]") + // We use square brackets for arrays regardless + val (leftStr, rightStr) = ("[", "]") val emptyStr = "" - val nullStr = if (legacyCastToString) "" else "null" val numRows = input.getRowCount.toInt withResource( - Seq(leftStr, rightStr, emptyStr, nullStr).safeMap(Scalar.fromString) + Seq(leftStr, rightStr, emptyStr, options.nullString).safeMap(Scalar.fromString) ){ case Seq(left, right, empty, nullRep) => val strChildContainsNull = withResource(input.getChildColumnView(0)) {child => doCast( - child, elementType, StringType, ansiMode, legacyCastToString, stringToDateAnsiModeEnabled) + child, elementType, StringType, options) } val concatenated = withResource(strChildContainsNull) { _ => withResource(input.replaceListChild(strChildContainsNull)) { - concatenateStringArrayElements(_, legacyCastToString) + concatenateStringArrayElements(_, options) } } @@ -782,45 +855,45 @@ object GpuCast { private def castMapToString( input: ColumnView, from: MapType, - ansiMode: Boolean, - legacyCastToString: Boolean, - stringToDateAnsiModeEnabled: Boolean): ColumnVector = { + options: CastOptions): ColumnVector = { val numRows = input.getRowCount.toInt val (arrowStr, emptyStr, spaceStr) = ("->", "", " ") - val (leftStr, rightStr, nullStr) = - if (legacyCastToString) ("[", "]", "") else ("{", "}", "null") // cast the key column and value column to string columns val (strKey, strValue) = withResource(input.getChildColumnView(0)) { kvStructColumn => val strKey = withResource(kvStructColumn.getChildColumnView(0)) { keyColumn => doCast( - keyColumn, from.keyType, StringType, ansiMode, - legacyCastToString, stringToDateAnsiModeEnabled) + keyColumn, from.keyType, StringType, options) } val strValue = closeOnExcept(strKey) {_ => withResource(kvStructColumn.getChildColumnView(1)) { valueColumn => doCast( - valueColumn, from.valueType, StringType, ansiMode, - legacyCastToString, stringToDateAnsiModeEnabled) + valueColumn, from.valueType, StringType, options) } } (strKey, strValue) } + import options._ // concatenate the key-value pairs to string // Example: ("key", "value") -> "key -> value" withResource( - Seq(leftStr, rightStr, arrowStr, emptyStr, nullStr, spaceStr).safeMap(Scalar.fromString) + Seq(leftBracket, + rightBracket, + arrowStr, + emptyStr, + nullString, + spaceStr).safeMap(Scalar.fromString) ) { case Seq(leftScalar, rightScalar, arrowScalar, emptyScalar, nullScalar, spaceScalar) => val strElements = withResource(Seq(strKey, strValue)) { case Seq(strKey, strValue) => val numElements = strKey.getRowCount.toInt withResource(Seq(spaceScalar, arrowScalar).safeMap(ColumnVector.fromScalar(_, numElements)) - ) {case Seq(spaceCol, arrowCol) => - if (legacyCastToString) { + ) { case Seq(spaceCol, arrowCol) => + if (useLegacyComplexTypesToString) { withResource( spaceCol.mergeAndSetValidity(BinaryOp.BITWISE_AND, strValue) - ) {spaceBetweenSepAndVal => + ) { spaceBetweenSepAndVal => ColumnVector.stringConcatenate( emptyScalar, nullScalar, Array(strKey, spaceCol, arrowCol, spaceBetweenSepAndVal, strValue)) @@ -835,7 +908,7 @@ object GpuCast { // concatenate elements val strCol = withResource(strElements) { _ => withResource(input.replaceListChild(strElements)) { - concatenateStringArrayElements(_, legacyCastToString) + concatenateStringArrayElements(_, options) } } val resPreValidityFix = withResource(strCol) { _ => @@ -855,14 +928,12 @@ object GpuCast { private def castStructToString( input: ColumnView, inputSchema: Array[StructField], - ansiMode: Boolean, - legacyCastToString: Boolean, - stringToDateAnsiModeEnabled: Boolean): ColumnVector = { + options: CastOptions): ColumnVector = { + + import options._ - val (leftStr, rightStr) = if (legacyCastToString) ("[", "]") else ("{", "}") val emptyStr = "" - val nullStr = if (legacyCastToString) "" else "null" - val separatorStr = if (legacyCastToString) "," else ", " + val separatorStr = if (useLegacyComplexTypesToString) "," else ", " val spaceStr = " " val numRows = input.getRowCount.toInt val numInputColumns = input.getNumChildren @@ -879,8 +950,7 @@ object GpuCast { // 3.1+: {firstCol columns += leftColumn.incRefCount() withResource(input.getChildColumnView(0)) { firstColumnView => - columns += doCast(firstColumnView, inputSchema.head.dataType, StringType, - ansiMode, legacyCastToString, stringToDateAnsiModeEnabled) + columns += doCast(firstColumnView, inputSchema.head.dataType, StringType, options) } for (nonFirstIndex <- 1 until numInputColumns) { withResource(input.getChildColumnView(nonFirstIndex)) { nonFirstColumnView => @@ -888,9 +958,8 @@ object GpuCast { // 3.1+: ", " columns += sepColumn.incRefCount() val nonFirstColumn = doCast(nonFirstColumnView, - inputSchema(nonFirstIndex).dataType, StringType, ansiMode, legacyCastToString, - stringToDateAnsiModeEnabled) - if (legacyCastToString) { + inputSchema(nonFirstIndex).dataType, StringType, options) + if (useLegacyComplexTypesToString) { // " " if non-null columns += spaceColumn.mergeAndSetValidity(BinaryOp.BITWISE_AND, nonFirstColumnView) } @@ -905,8 +974,8 @@ object GpuCast { } } - withResource(Seq(emptyStr, nullStr, separatorStr, spaceStr, leftStr, rightStr) - .safeMap(Scalar.fromString)) { + withResource(Seq(emptyStr, nullString, separatorStr, spaceStr, leftBracket, rightBracket) + .safeMap(Scalar.fromString)) { case Seq(emptyScalar, nullScalar, columnScalars@_*) => withResource( @@ -1216,20 +1285,16 @@ object GpuCast { from: MapType, to: MapType, input: ColumnView, - ansiMode: Boolean, - legacyCastToString: Boolean, - stringToDateAnsiModeEnabled: Boolean): ColumnVector = { + options: CastOptions): ColumnVector = { // For cudf a map is a list of (key, value) structs, but lets keep it in ColumnView as much // as possible withResource(input.getChildColumnView(0)) { kvStructColumn => val castKey = withResource(kvStructColumn.getChildColumnView(0)) { keyColumn => - doCast(keyColumn, from.keyType, to.keyType, ansiMode, legacyCastToString, - stringToDateAnsiModeEnabled) + doCast(keyColumn, from.keyType, to.keyType, options) } withResource(castKey) { castKey => val castValue = withResource(kvStructColumn.getChildColumnView(1)) { valueColumn => - doCast(valueColumn, from.valueType, to.valueType, - ansiMode, legacyCastToString, stringToDateAnsiModeEnabled) + doCast(valueColumn, from.valueType, to.valueType, options) } withResource(castValue) { castValue => withResource(ColumnView.makeStructView(castKey, castValue)) { castKvStructColumn => @@ -1248,17 +1313,13 @@ object GpuCast { from: StructType, to: StructType, input: ColumnView, - ansiMode: Boolean, - legacyCastToString: Boolean, - stringToDateAnsiModeEnabled: Boolean): ColumnVector = { + options: CastOptions): ColumnVector = { withResource(new ArrayBuffer[ColumnVector](from.length)) { childColumns => from.indices.foreach { index => childColumns += doCast( input.getChildColumnView(index), from(index).dataType, - to(index).dataType, - ansiMode, - legacyCastToString, stringToDateAnsiModeEnabled) + to(index).dataType, options) } withResource(ColumnView.makeStructView(childColumns: _*)) { casted => if (input.getNullCount == 0) { @@ -1492,12 +1553,15 @@ case class GpuCast( dataType: DataType, ansiMode: Boolean = false, timeZoneId: Option[String] = None, - legacyCastToString: Boolean = false, + legacyCastComplexTypesToString: Boolean = false, stringToDateAnsiModeEnabled: Boolean = false) extends GpuUnaryExpression with TimeZoneAwareExpression with NullIntolerant { import GpuCast._ + private val options: CastOptions = + new CastOptions(legacyCastComplexTypesToString, ansiMode, stringToDateAnsiModeEnabled) + // when ansi mode is enabled, some cast expressions can throw exceptions on invalid inputs override def hasSideEffects: Boolean = super.hasSideEffects || { (child.dataType, dataType) match { @@ -1563,7 +1627,5 @@ case class GpuCast( } override def doColumnar(input: GpuColumnVector): ColumnVector = - doCast(input.getBase, input.dataType(), dataType, ansiMode, legacyCastToString, - stringToDateAnsiModeEnabled) - + doCast(input.getBase, input.dataType(), dataType, options) } \ No newline at end of file diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala index 6e48d7bb8d0..f971ddd2aa4 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOrcScan.scala @@ -383,8 +383,7 @@ object GpuOrcScan { case (f: DType, t: DType) if f.isDecimalType && t.isDecimalType => val fromDataType = DecimalType(f.getDecimalMaxPrecision, -f.getScale) val toDataType = DecimalType(t.getDecimalMaxPrecision, -t.getScale) - GpuCast.doCast(col, fromDataType, toDataType, ansiMode=false, legacyCastToString = false, - stringToDateAnsiModeEnabled = false) + GpuCast.doCast(col, fromDataType, toDataType) case (DType.STRING, DType.STRING) if originalFromDt.isInstanceOf[CharType] => // Trim trailing whitespace off of output strings, to match CPU output. diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala index af9cfc8f717..3b5aea39bf0 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/GpuJsonToStructs.scala @@ -197,7 +197,7 @@ case class GpuJsonToStructs( } else { val col = rawTable.getColumn(i) // getSparkType is only used to get the from type for cast - doCast(col, getSparkType(col), dtype, false, false, false) + doCast(col, getSparkType(col), dtype) } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala index e1cbc6b0c06..2e8d46e6f6f 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/arithmetic.scala @@ -401,13 +401,16 @@ trait GpuDecimalMultiplyBase extends GpuExpression { def regularMultiply(batch: ColumnarBatch): GpuColumnVector = { val castLhs = withResource(left.columnarEval(batch)) { lhs => - GpuCast.doCast(lhs.getBase, lhs.dataType(), intermediateLhsType, ansiMode = failOnError, - legacyCastToString = false, stringToDateAnsiModeEnabled = false) + GpuCast.doCast( + lhs.getBase, + lhs.dataType(), + intermediateLhsType, + CastOptions.getArithmeticCastOptions(failOnError)) } val ret = withResource(castLhs) { castLhs => val castRhs = withResource(right.columnarEval(batch)) { rhs => - GpuCast.doCast(rhs.getBase, rhs.dataType(), intermediateRhsType, ansiMode = failOnError, - legacyCastToString = false, stringToDateAnsiModeEnabled = false) + GpuCast.doCast(rhs.getBase, rhs.dataType(), intermediateRhsType, + CastOptions.getArithmeticCastOptions(failOnError)) } withResource(castRhs) { castRhs => withResource(castLhs.mul(castRhs, @@ -436,7 +439,7 @@ trait GpuDecimalMultiplyBase extends GpuExpression { } withResource(ret) { ret => GpuColumnVector.from(GpuCast.doCast(ret, intermediateResultType, dataType, - ansiMode = failOnError, legacyCastToString = false, stringToDateAnsiModeEnabled = false), + CastOptions.getArithmeticCastOptions(failOnError)), dataType) } } @@ -851,14 +854,18 @@ trait GpuDecimalDivideBase extends GpuExpression { def regularDivide(batch: ColumnarBatch): GpuColumnVector = { val castLhs = withResource(left.columnarEval(batch)) { lhs => - GpuCast.doCast(lhs.getBase, lhs.dataType(), intermediateLhsType, ansiMode = failOnError, - legacyCastToString = false, stringToDateAnsiModeEnabled = false) + GpuCast.doCast( + lhs.getBase, + lhs.dataType(), + intermediateLhsType, + CastOptions.getArithmeticCastOptions(failOnError)) + } val ret = withResource(castLhs) { castLhs => val castRhs = withResource(right.columnarEval(batch)) { rhs => withResource(divByZeroFixes(rhs.getBase)) { fixed => - GpuCast.doCast(fixed, rhs.dataType(), intermediateRhsType, ansiMode = failOnError, - legacyCastToString = false, stringToDateAnsiModeEnabled = false) + GpuCast.doCast(fixed, rhs.dataType(), intermediateRhsType, + CastOptions.getArithmeticCastOptions(failOnError)) } } withResource(castRhs) { castRhs => @@ -871,7 +878,7 @@ trait GpuDecimalDivideBase extends GpuExpression { // in the common case with us. It will also handle rounding the result to the final scale // to match what Spark does. GpuColumnVector.from(GpuCast.doCast(ret, intermediateResultType, dataType, - ansiMode = failOnError, legacyCastToString = false, stringToDateAnsiModeEnabled = false), + CastOptions.getArithmeticCastOptions(failOnError)), dataType) } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala index d373b2459c1..e0fe58b0857 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/datetimeExpressions.scala @@ -460,7 +460,7 @@ case class GpuSecondsToTimestamp(child: Expression) extends GpuNumberToTimestamp } case DoubleType | FloatType => (input: GpuColumnVector) => { - GpuCast.doCast(input.getBase, input.dataType, TimestampType, false, false, false) + GpuCast.doCast(input.getBase, input.dataType, TimestampType) } case dt: DecimalType => (input: GpuColumnVector) => { diff --git a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala index ce978508826..2a4ec2cc902 100644 --- a/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala +++ b/sql-plugin/src/main/spark330db/scala/org/apache/spark/sql/rapids/arithmetic.scala @@ -95,11 +95,11 @@ trait GpuAddSub extends CudfBinaryArithmetic { } else { // eval operands using the output precision val castLhs = withResource(left.columnarEval(batch)) { lhs => - GpuCast.doCast(lhs.getBase(), leftInputType, resultType, false, false, false) + GpuCast.doCast(lhs.getBase(), leftInputType, resultType) } val castRhs = closeOnExcept(castLhs){ _ => withResource(right.columnarEval(batch)) { rhs => - GpuCast.doCast(rhs.getBase(), rightInputType, resultType, false, false, false) + GpuCast.doCast(rhs.getBase(), rightInputType, resultType) } } @@ -342,14 +342,14 @@ case class GpuDecimalRemainder(left: Expression, right: Expression) private def regularRemainder(batch: ColumnarBatch): GpuColumnVector = { val castLhs = withResource(left.columnarEval(batch)) { lhs => - GpuCast.doCast(lhs.getBase, lhs.dataType(), intermediateLhsType, ansiMode = failOnError, - legacyCastToString = false, stringToDateAnsiModeEnabled = false) + GpuCast.doCast(lhs.getBase, lhs.dataType(), intermediateLhsType, + CastOptions.getArithmeticCastOptions(failOnError)) } withResource(castLhs) { castLhs => val castRhs = withResource(right.columnarEval(batch)) { rhs => withResource(divByZeroFixes(rhs.getBase)) { fixed => - GpuCast.doCast(fixed, rhs.dataType(), intermediateRhsType, ansiMode = failOnError, - legacyCastToString = false, stringToDateAnsiModeEnabled = false) + GpuCast.doCast(fixed, rhs.dataType(), intermediateRhsType, + CastOptions.getArithmeticCastOptions(failOnError)) } } withResource(castRhs) { castRhs => From 21474a5e1a9da59ff51f31ba32c45707a7f37481 Mon Sep 17 00:00:00 2001 From: Peixin Date: Mon, 25 Sep 2023 09:36:14 +0800 Subject: [PATCH 26/46] Automerge from 23.10 to 23.12 (#9291) Signed-off-by: Peixin Li --- .github/workflows/auto-merge.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/auto-merge.yml b/.github/workflows/auto-merge.yml index a54b1cae0e1..efcb45a937f 100644 --- a/.github/workflows/auto-merge.yml +++ b/.github/workflows/auto-merge.yml @@ -18,7 +18,7 @@ name: auto-merge HEAD to BASE on: pull_request_target: branches: - - branch-23.08 + - branch-23.10 types: [closed] jobs: @@ -29,13 +29,13 @@ jobs: steps: - uses: actions/checkout@v3 with: - ref: branch-23.08 # force to fetch from latest upstream instead of PR ref + ref: branch-23.10 # force to fetch from latest upstream instead of PR ref - name: auto-merge job uses: ./.github/workflows/auto-merge env: OWNER: NVIDIA REPO_NAME: spark-rapids - HEAD: branch-23.08 - BASE: branch-23.10 + HEAD: branch-23.10 + BASE: branch-23.12 AUTOMERGE_TOKEN: ${{ secrets.AUTOMERGE_TOKEN }} # use to merge PR From 81971d5a425d68454aa4bc27b2802e2a8f1c6e58 Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Mon, 25 Sep 2023 12:29:40 -0500 Subject: [PATCH 27/46] Fix test_delta_read_column_mapping test failures on Spark 3.2.x and 3.3.x (#9294) Signed-off-by: Jason Lowe --- integration_tests/src/main/python/delta_lake_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/integration_tests/src/main/python/delta_lake_test.py b/integration_tests/src/main/python/delta_lake_test.py index a7ca70dc64c..5b00ab20c22 100644 --- a/integration_tests/src/main/python/delta_lake_test.py +++ b/integration_tests/src/main/python/delta_lake_test.py @@ -92,7 +92,7 @@ def setup_tables(spark): # ID mapping is supported starting in Delta Lake 2.2, but currently cannot distinguish # Delta Lake 2.1 from 2.2 in tests. https://github.com/NVIDIA/spark-rapids/issues/9276 -column_mappings = ["name", "id"] +column_mappings = ["name"] if is_spark_340_or_later() or is_databricks_runtime(): column_mappings.append("id") From c5f40d5bc63a722ff17563ab140e5a2d7028c300 Mon Sep 17 00:00:00 2001 From: winningsixnv <146045000+winningsixnv@users.noreply.github.com> Date: Tue, 26 Sep 2023 14:16:45 +0800 Subject: [PATCH 28/46] Append new authorized user to blossom-ci whitelist [skip ci] (#9297) Signed-off-by: Cheng Xu --- .github/workflows/blossom-ci.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/blossom-ci.yml b/.github/workflows/blossom-ci.yml index 514e21386c8..83e102fca21 100644 --- a/.github/workflows/blossom-ci.yml +++ b/.github/workflows/blossom-ci.yml @@ -68,6 +68,7 @@ jobs: YanxuanLiu,\ cindyyuanjiang,\ thirtiseven,\ + winningsixnv,\ ', format('{0},', github.actor)) && github.event.comment.body == 'build' steps: - name: Check if comment is issued by authorized person From e976724c5e1822131a48864344078881040181a6 Mon Sep 17 00:00:00 2001 From: MithunR Date: Tue, 26 Sep 2023 14:47:40 -0700 Subject: [PATCH 29/46] Support negative preceding/following for ROW-based window functions (#9229) This commit adds support for negative values for preceding/following offsets specified for `ROW` based window functions. Prior to this commit, window function queries such as the following were not supported: ```SQL SELECT MIN(x) OVER (PARTITION BY grp ORDER BY oby ROWS BETWEEN 5 PRECEDING AND -1 FOLLOWING) min_x FROM mytable ``` For this query, the window includes all rows between upto 5 rows preceding the current row, and the previous row. This functionality is currently supported only for: 1. `AVG` 2. `COUNT(1)`/`COUNT(*)` 3. `MAX` 4. `MIN` 5. `SUM` 6. `COLLECT_LIST` 7. `COLLECT_SET` Signed-off-by: MithunR --- .../src/main/python/window_function_test.py | 99 ++++++++++++++++++- .../nvidia/spark/rapids/GpuWindowExec.scala | 26 +++-- .../spark/rapids/GpuWindowExpression.scala | 35 +++++-- .../spark/sql/rapids/AggregateFunctions.scala | 19 ++++ 4 files changed, 161 insertions(+), 18 deletions(-) diff --git a/integration_tests/src/main/python/window_function_test.py b/integration_tests/src/main/python/window_function_test.py index 6c81fcb804e..5a5347c70bf 100644 --- a/integration_tests/src/main/python/window_function_test.py +++ b/integration_tests/src/main/python/window_function_test.py @@ -21,7 +21,7 @@ from pyspark.sql.types import NumericType from pyspark.sql.window import Window import pyspark.sql.functions as f -from spark_session import is_before_spark_320, is_databricks113_or_later +from spark_session import is_before_spark_320, is_databricks113_or_later, spark_version import warnings _grpkey_longs_with_no_nulls = [ @@ -1486,6 +1486,103 @@ def test_to_date_with_window_functions(): """ ) + +@ignore_order(local=True) +@approximate_float +@pytest.mark.parametrize('batch_size', ['1000', '1g'], ids=idfn) +@pytest.mark.parametrize('data_gen', [_grpkey_longs_with_no_nulls, + _grpkey_longs_with_nulls, + _grpkey_longs_with_dates, + _grpkey_longs_with_nullable_dates, + _grpkey_longs_with_decimals, + _grpkey_longs_with_nullable_decimals, + _grpkey_longs_with_nullable_larger_decimals + ], ids=idfn) +@pytest.mark.parametrize('window_spec', ["3 PRECEDING AND -1 FOLLOWING", + "-2 PRECEDING AND 4 FOLLOWING", + "UNBOUNDED PRECEDING AND -1 FOLLOWING", + "-1 PRECEDING AND UNBOUNDED FOLLOWING", + "10 PRECEDING AND -1 FOLLOWING", + "5 PRECEDING AND -2 FOLLOWING"], ids=idfn) +def test_window_aggs_for_negative_rows_partitioned(data_gen, batch_size, window_spec): + conf = {'spark.rapids.sql.batchSizeBytes': batch_size, + 'spark.rapids.sql.castFloatToDecimal.enabled': True} + assert_gpu_and_cpu_are_equal_sql( + lambda spark: gen_df(spark, data_gen, length=2048), + "window_agg_table", + 'SELECT ' + ' SUM(c) OVER ' + ' (PARTITION BY a ORDER BY b,c ASC ROWS BETWEEN {window}) AS sum_c_asc, ' + ' MAX(c) OVER ' + ' (PARTITION BY a ORDER BY b DESC, c DESC ROWS BETWEEN {window}) AS max_c_desc, ' + ' MIN(c) OVER ' + ' (PARTITION BY a ORDER BY b,c ROWS BETWEEN {window}) AS min_c_asc, ' + ' COUNT(1) OVER ' + ' (PARTITION BY a ORDER BY b,c ROWS BETWEEN {window}) AS count_1, ' + ' COUNT(c) OVER ' + ' (PARTITION BY a ORDER BY b,c ROWS BETWEEN {window}) AS count_c, ' + ' AVG(c) OVER ' + ' (PARTITION BY a ORDER BY b,c ROWS BETWEEN {window}) AS avg_c, ' + ' COLLECT_LIST(c) OVER ' + ' (PARTITION BY a ORDER BY b,c ROWS BETWEEN {window}) AS list_c, ' + ' SORT_ARRAY(COLLECT_SET(c) OVER ' + ' (PARTITION BY a ORDER BY b,c ROWS BETWEEN {window})) AS sorted_set_c ' + 'FROM window_agg_table '.format(window=window_spec), + conf=conf) + + +def spark_bugs_in_decimal_sorting(): + """ + Checks whether Apache Spark version has a bug in sorting Decimal columns correctly. + See https://issues.apache.org/jira/browse/SPARK-40089. + :return: True, if Apache Spark version does not sort Decimal(>20, >2) correctly. False, otherwise. + """ + v = spark_version() + return v < "3.1.4" or v < "3.3.1" or v < "3.2.3" or v < "3.4.0" + + +@ignore_order(local=True) +@approximate_float +@pytest.mark.parametrize('batch_size', ['1g'], ids=idfn) +@pytest.mark.parametrize('data_gen', [_grpkey_longs_with_no_nulls, + _grpkey_longs_with_nulls, + _grpkey_longs_with_dates, + _grpkey_longs_with_nullable_dates, + _grpkey_longs_with_decimals, + _grpkey_longs_with_nullable_decimals, + pytest.param(_grpkey_longs_with_nullable_larger_decimals, + marks=pytest.mark.skipif( + condition=spark_bugs_in_decimal_sorting(), + reason='https://github.com/NVIDIA/spark-rapids/issues/7429'))], + ids=idfn) +def test_window_aggs_for_negative_rows_unpartitioned(data_gen, batch_size): + conf = {'spark.rapids.sql.batchSizeBytes': batch_size, + 'spark.rapids.sql.castFloatToDecimal.enabled': True} + + assert_gpu_and_cpu_are_equal_sql( + lambda spark: gen_df(spark, data_gen, length=2048), + "window_agg_table", + 'SELECT ' + ' SUM(c) OVER ' + ' (ORDER BY b,c,a ROWS BETWEEN 3 PRECEDING AND -1 FOLLOWING) AS sum_c_asc, ' + ' MAX(c) OVER ' + ' (ORDER BY b DESC, c DESC, a DESC ROWS BETWEEN -2 PRECEDING AND 4 FOLLOWING) AS max_c_desc, ' + ' min(c) OVER ' + ' (ORDER BY b,c,a ROWS BETWEEN UNBOUNDED PRECEDING AND -1 FOLLOWING) AS min_c_asc, ' + ' COUNT(1) OVER ' + ' (ORDER BY b,c,a ROWS BETWEEN -1 PRECEDING AND UNBOUNDED FOLLOWING) AS count_1, ' + ' COUNT(c) OVER ' + ' (ORDER BY b,c,a ROWS BETWEEN 10 PRECEDING AND -1 FOLLOWING) AS count_c, ' + ' AVG(c) OVER ' + ' (ORDER BY b,c,a ROWS BETWEEN -1 PRECEDING AND UNBOUNDED FOLLOWING) AS avg_c, ' + ' COLLECT_LIST(c) OVER ' + ' (PARTITION BY a ORDER BY b,c,a ROWS BETWEEN 5 PRECEDING AND -2 FOLLOWING) AS list_c, ' + ' SORT_ARRAY(COLLECT_SET(c) OVER ' + ' (PARTITION BY a ORDER BY b,c,a ROWS BETWEEN 5 PRECEDING AND -2 FOLLOWING)) AS set_c ' + 'FROM window_agg_table ', + conf=conf) + + def test_lru_cache_datagen(): # log cache info at the end of integration tests, not related to window functions info = gen_df_help.cache_info() diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala index c5d9a8bce9f..b52670d6c1e 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExec.scala @@ -689,12 +689,13 @@ object GroupedAggregations { private def getWindowOptions( orderSpec: Seq[SortOrder], orderPositions: Seq[Int], - frame: GpuSpecifiedWindowFrame): WindowOptions = { + frame: GpuSpecifiedWindowFrame, + minPeriods: Int): WindowOptions = { frame.frameType match { case RowFrame => withResource(getRowBasedLower(frame)) { lower => withResource(getRowBasedUpper(frame)) { upper => - val builder = WindowOptions.builder().minPeriods(1) + val builder = WindowOptions.builder().minPeriods(minPeriods) if (isUnbounded(frame.lower)) builder.unboundedPreceding() else builder.preceding(lower) if (isUnbounded(frame.upper)) builder.unboundedFollowing() else builder.following(upper) builder.build @@ -718,7 +719,7 @@ object GroupedAggregations { withResource(asScalarRangeBoundary(orderType, lower)) { preceding => withResource(asScalarRangeBoundary(orderType, upper)) { following => val windowOptionBuilder = WindowOptions.builder() - .minPeriods(1) + .minPeriods(1) // Does not currently support custom minPeriods. .orderByColumnIndex(orderByIndex) if (preceding.isEmpty) { @@ -929,13 +930,18 @@ class GroupedAggregations { if (frameSpec.frameType == frameType) { // For now I am going to assume that we don't need to combine calls across frame specs // because it would just not help that much - val result = withResource( - getWindowOptions(boundOrderSpec, orderByPositions, frameSpec)) { windowOpts => - val allAggs = functions.map { - case (winFunc, _) => winFunc.aggOverWindow(inputCb, windowOpts) - }.toSeq - withResource(GpuColumnVector.from(inputCb)) { initProjTab => - aggIt(initProjTab.groupBy(partByPositions: _*), allAggs) + val result = { + val allWindowOpts = functions.map { f => + getWindowOptions(boundOrderSpec, orderByPositions, frameSpec, + f._1.windowFunc.getMinPeriods) + } + withResource(allWindowOpts.toSeq) { allWindowOpts => + val allAggs = allWindowOpts.zip(functions).map { case (windowOpt, f) => + f._1.aggOverWindow(inputCb, windowOpt) + } + withResource(GpuColumnVector.from(inputCb)) { initProjTab => + aggIt(initProjTab.groupBy(partByPositions: _*), allAggs) + } } } withResource(result) { result => diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala index 974647d1046..14b41e18eaf 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala @@ -28,6 +28,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{TypeCheckFailure, TypeCheckSuccess} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, Average, CollectList, CollectSet, Count, Max, Min, Sum} import org.apache.spark.sql.rapids.{AddOverflowChecks, GpuAggregateExpression, GpuCount, GpuCreateNamedStruct, GpuDivide, GpuSubtract} import org.apache.spark.sql.rapids.shims.RapidsErrorUtils import org.apache.spark.sql.types._ @@ -81,13 +82,25 @@ abstract class GpuWindowExpressionMetaBase( case _: Lead | _: Lag => // ignored we are good case _ => // need to be sure that the lower/upper are acceptable - if (lower > 0) { - willNotWorkOnGpu(s"lower-bounds ahead of current row is not supported. " + - s"Found $lower") + // Negative bounds are allowed, so long as lower does not exceed upper. + if (upper < lower) { + willNotWorkOnGpu("upper-bounds must equal or exceed the lower bounds. " + + s"Found lower=$lower, upper=$upper ") } - if (upper < 0) { - willNotWorkOnGpu(s"upper-bounds behind the current row is not supported. " + - s"Found $upper") + // Also check for negative offsets. + if (upper < 0 || lower > 0) { + windowFunction.asInstanceOf[AggregateExpression].aggregateFunction match { + case _: Average => // Supported + case _: CollectList => // Supported + case _: CollectSet => // Supported + case _: Count => // Supported + case _: Max => // Supported + case _: Min => // Supported + case _: Sum => // Supported + case f: AggregateFunction => + willNotWorkOnGpu("negative row bounds unsupported for specified " + + s"aggregation: ${f.prettyName}") + } } } case RangeFrame => @@ -649,7 +662,15 @@ case class GpuSpecialFrameBoundary(boundary : SpecialFrameBoundary) // This is here for now just to tag an expression as being a GpuWindowFunction and match // Spark. This may expand in the future if other types of window functions show up. -trait GpuWindowFunction extends GpuUnevaluable with ShimExpression +trait GpuWindowFunction extends GpuUnevaluable with ShimExpression { + /** + * Get "min-periods" value, i.e. the minimum number of periods/rows + * above which a non-null value is returned for the function. + * Otherwise, null is returned. + * @return Non-negative value for min-periods. + */ + def getMinPeriods: Int = 1 +} /** * This is a special window function that simply replaces itself with one or more diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala index 78811fef08e..618ba14b792 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/AggregateFunctions.scala @@ -1586,6 +1586,13 @@ case class GpuCount(children: Seq[Expression], override def newUnboundedToUnboundedFixer: BatchedUnboundedToUnboundedWindowFixer = new CountUnboundedToUnboundedFixer(failOnError) + + // minPeriods should be 0. + // Consider the following rows: + // v = [ 0, 1, 2, 3, 4, 5 ] + // A `COUNT` window aggregation over (2, -1) should yield 0, not null, + // for the first row. + override def getMinPeriods: Int = 0 } object GpuAverage { @@ -1971,6 +1978,12 @@ case class GpuCollectList( override def windowAggregation( inputs: Seq[(ColumnVector, Int)]): RollingAggregationOnColumn = RollingAggregation.collectList().onColumn(inputs.head._2) + + // minPeriods should be 0. + // Consider the following rows: v = [ 0, 1, 2, 3, 4, 5 ] + // A `COLLECT_LIST` window aggregation over (2, -1) should yield an empty array [], + // not null, for the first row. + override def getMinPeriods: Int = 0 } /** @@ -2005,6 +2018,12 @@ case class GpuCollectSet( RollingAggregation.collectSet(NullPolicy.EXCLUDE, NullEquality.EQUAL, NaNEquality.ALL_EQUAL).onColumn(inputs.head._2) } + + // minPeriods should be 0. + // Consider the following rows: v = [ 0, 1, 2, 3, 4, 5 ] + // A `COLLECT_SET` window aggregation over (2, -1) should yield an empty array [], + // not null, for the first row. + override def getMinPeriods: Int = 0 } trait CpuToGpuAggregateBufferConverter { From 8e84c8eedfd2819876fa4e5afb98eef9da1c6a6a Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 27 Sep 2023 13:15:14 -0600 Subject: [PATCH 30/46] [FEA][AUDIT][SPARK-44641] Incorrect result in certain scenarios when SPJ is not triggered (#9286) * Update code with fix for SPARK-44641 * signoff Signed-off-by: Andy Grove * add KeyGroupedPartitioningShim and remove duplicate copy of GpuBatchScanExec --------- Signed-off-by: Andy Grove --- .../spark/rapids/shims/GpuBatchScanExec.scala | 7 +++- .../shims/KeyGroupedPartitioningShim.scala | 33 +++++++++++++++++++ .../shims/KeyGroupedPartitioningShim.scala | 28 ++++++++++++++++ 3 files changed, 67 insertions(+), 1 deletion(-) create mode 100644 sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala create mode 100644 sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala index 2a951cb9500..4b7a984144e 100644 --- a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/GpuBatchScanExec.scala @@ -202,7 +202,12 @@ case class GpuBatchScanExec( val partitionMapping = groupedPartitions.map { case (row, parts) => InternalRowComparableWrapper(row, p.expressions) -> parts }.toMap - finalPartitions = p.partitionValues.map { partValue => + + // In case `commonPartitionValues` is not defined (e.g., SPJ is not used), there + // could exist duplicated partition values, as partition grouping is not done + // at the beginning and postponed to this method. It is important to use unique + // partition values here so that grouped partitions won't get duplicated. + finalPartitions = KeyGroupedPartitioningShim.getUniquePartitions(p).map { partValue => // Use empty partition for those partition values that are not present partitionMapping.getOrElse( InternalRowComparableWrapper(partValue, p.expressions), Seq.empty) diff --git a/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala new file mode 100644 index 00000000000..53644a02804 --- /dev/null +++ b/sql-plugin/src/main/spark340/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala @@ -0,0 +1,33 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/*** spark-rapids-shim-json-lines +{"spark": "340"} +{"spark": "341"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.plans.physical.KeyGroupedPartitioning +import org.apache.spark.sql.catalyst.util.InternalRowComparableWrapper + +object KeyGroupedPartitioningShim { + def getUniquePartitions(p: KeyGroupedPartitioning): Seq[InternalRow] = { + p.partitionValues + .map(InternalRowComparableWrapper(_, p.expressions)) + .distinct + .map(_.row) + } +} \ No newline at end of file diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala new file mode 100644 index 00000000000..af88498cd34 --- /dev/null +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/KeyGroupedPartitioningShim.scala @@ -0,0 +1,28 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/*** spark-rapids-shim-json-lines +{"spark": "350"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.plans.physical.KeyGroupedPartitioning + +object KeyGroupedPartitioningShim { + def getUniquePartitions(p: KeyGroupedPartitioning): Seq[InternalRow] = { + p.uniquePartitionValues + } +} \ No newline at end of file From 95d67a9cc704a0c772ae23cc4f0aeacc94768b1c Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Wed, 27 Sep 2023 18:04:14 -0500 Subject: [PATCH 31/46] Sort results of collect_list test before comparing since it is not guaranteed (#9313) Signed-off-by: Jason Lowe --- .../src/main/python/hash_aggregate_test.py | 43 ++++++++++++++----- 1 file changed, 33 insertions(+), 10 deletions(-) diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index 9ccdf3ef8fa..288cf3ebc07 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -595,8 +595,7 @@ def test_hash_pivot_groupby_duplicates_fallback(data_gen): _repeat_agg_column_for_collect_list_op = [ RepeatSeqGen(ArrayGen(int_gen), length=15), RepeatSeqGen(all_basic_struct_gen, length=15), - RepeatSeqGen(StructGen([['c0', all_basic_struct_gen]]), length=15), - RepeatSeqGen(simple_string_to_string_map_gen, length=15)] + RepeatSeqGen(StructGen([['c0', all_basic_struct_gen]]), length=15)] _gen_data_for_collect_list_op = _full_gen_data_for_collect_op + [[ ('a', RepeatSeqGen(LongGen(), length=20)), @@ -666,19 +665,43 @@ def test_min_max_group_by(data_gen): .groupby('a') .agg(f.min('b'), f.max('b'))) -# to avoid ordering issues with collect_list we do it all in a single task +# To avoid ordering issues with collect_list, sorting the arrays that are returned. +# Note, using sort_array() on the CPU, because sort_array() does not yet +# support sorting certain nested/arbitrary types on the GPU +# See https://github.com/NVIDIA/spark-rapids/issues/3715 +# and https://github.com/rapidsai/cudf/issues/11222 +@allow_non_gpu("ProjectExec", "SortArray") @ignore_order(local=True) @pytest.mark.parametrize('data_gen', _gen_data_for_collect_list_op, ids=idfn) @pytest.mark.parametrize('use_obj_hash_agg', [True, False], ids=idfn) def test_hash_groupby_collect_list(data_gen, use_obj_hash_agg): + def doit(spark): + df = gen_df(spark, data_gen, length=100)\ + .groupby('a')\ + .agg(f.collect_list('b').alias("blist")) + # pull out the rdd and schema and create a new dataframe to run SortArray + # to handle Spark 3.3.0+ optimization that moves SortArray from ProjectExec + # to ObjectHashAggregateExec + return spark.createDataFrame(df.rdd, schema=df.schema).select("a", f.sort_array("blist")) assert_gpu_and_cpu_are_equal_collect( - lambda spark: gen_df(spark, data_gen, length=100, num_slices=1) - .groupby('a') - .agg(f.collect_list('b')), - conf={'spark.sql.execution.useObjectHashAggregateExec': str(use_obj_hash_agg).lower(), - # Disable RADIX sort as the CPU sort is not stable if it is - 'spark.sql.sort.enableRadixSort': False, - 'spark.sql.shuffle.partitions': '1'}) + doit, + conf={'spark.sql.execution.useObjectHashAggregateExec': str(use_obj_hash_agg).lower()}) + +@ignore_order(local=True) +@pytest.mark.parametrize('use_obj_hash_agg', [True, False], ids=idfn) +def test_hash_groupby_collect_list_of_maps(use_obj_hash_agg): + gens = [("a", RepeatSeqGen(LongGen(), length=20)), ("b", simple_string_to_string_map_gen)] + def doit(spark): + df = gen_df(spark, gens, length=100) \ + .groupby('a') \ + .agg(f.collect_list('b').alias("blist")) + # Spark cannot sort maps, so explode the list back into rows. Yes, this is essentially + # testing whether after a collect_list we can get back to the original dataframe with + # an explode. + return spark.createDataFrame(df.rdd, schema=df.schema).select("a", f.explode("blist")) + assert_gpu_and_cpu_are_equal_collect( + doit, + conf={'spark.sql.execution.useObjectHashAggregateExec': str(use_obj_hash_agg).lower()}) @ignore_order(local=True) @pytest.mark.parametrize('data_gen', _full_gen_data_for_collect_op, ids=idfn) From d7230b63a01b45c537c0e74029e7c9244fa7188c Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 27 Sep 2023 18:18:13 -0600 Subject: [PATCH 32/46] Update tests and documentation for `spark.sql.timestampType` when reading CSV/JSON (#9159) * add failing test Signed-off-by: Andy Grove * test now reflects current behavior of falling back to CPU for timestamp_ntz * update docs * fix typos Signed-off-by: Andy Grove * add json test * update docs * refactor json test * implement v1/v2 csv tests * csv tests pass --------- Signed-off-by: Andy Grove --- docs/compatibility.md | 9 ++- integration_tests/src/main/python/csv_test.py | 52 +++++++++++++++++ integration_tests/src/main/python/data_gen.py | 10 ++-- .../src/main/python/json_test.py | 56 ++++++++++++++++++- 4 files changed, 118 insertions(+), 9 deletions(-) diff --git a/docs/compatibility.md b/docs/compatibility.md index b5cb01757dd..01f9707e17a 100644 --- a/docs/compatibility.md +++ b/docs/compatibility.md @@ -123,9 +123,9 @@ Only a limited set of formats are supported when parsing dates. ### CSV Timestamps The CSV parser does not support time zones. It will ignore any trailing time zone information, -despite the format asking for a `XXX` or `[XXX]`. As such it is off by default and you can enable it -by setting [`spark.rapids.sql.csvTimestamps.enabled`](configs.md#sql.csvTimestamps.enabled) to -`true`. +despite the format asking for a `XXX` or `[XXX]`. The CSV parser does not support the `TimestampNTZ` +type and will fall back to CPU if `spark.sql.timestampType` is set to `TIMESTAMP_NTZ` or if an +explicit schema is provided that contains the `TimestampNTZ` type. The formats supported for timestamps are limited similar to dates. The first part of the format must be a supported date format. The second part must start with a `'T'` to separate the time @@ -350,6 +350,9 @@ Spark will treat them as invalid inputs and will just return `null`. ### JSON Timestamps +The JSON parser does not support the `TimestampNTZ` type and will fall back to CPU if `spark.sql.timestampType` is +set to `TIMESTAMP_NTZ` or if an explicit schema is provided that contains the `TimestampNTZ` type. + There is currently no support for reading numeric values as timestamps and null values are returned instead ([#4940](https://github.com/NVIDIA/spark-rapids/issues/4940)). A workaround would be to read as longs and then cast to timestamp. diff --git a/integration_tests/src/main/python/csv_test.py b/integration_tests/src/main/python/csv_test.py index 5227dd0a41c..bae076f5a4d 100644 --- a/integration_tests/src/main/python/csv_test.py +++ b/integration_tests/src/main/python/csv_test.py @@ -551,6 +551,58 @@ def test_csv_read_count(spark_tmp_path): assert_gpu_and_cpu_row_counts_equal(lambda spark: spark.read.csv(data_path)) +@allow_non_gpu('FileSourceScanExec', 'ProjectExec', 'CollectLimitExec', 'DeserializeToObjectExec') +@pytest.mark.skipif(is_before_spark_340(), reason='`TIMESTAMP_NTZ` is only supported in Spark 340+') +@pytest.mark.parametrize('date_format', csv_supported_date_formats) +@pytest.mark.parametrize('ts_part', csv_supported_ts_parts) +@pytest.mark.parametrize("timestamp_type", ["TIMESTAMP_LTZ", "TIMESTAMP_NTZ"]) +def test_csv_infer_schema_timestamp_ntz_v1(spark_tmp_path, date_format, ts_part, timestamp_type): + csv_infer_schema_timestamp_ntz(spark_tmp_path, date_format, ts_part, timestamp_type, 'csv', 'FileSourceScanExec') + +@allow_non_gpu('BatchScanExec', 'FileSourceScanExec', 'ProjectExec', 'CollectLimitExec', 'DeserializeToObjectExec') +@pytest.mark.skipif(is_before_spark_340(), reason='`TIMESTAMP_NTZ` is only supported in Spark 340+') +@pytest.mark.parametrize('date_format', csv_supported_date_formats) +@pytest.mark.parametrize('ts_part', csv_supported_ts_parts) +@pytest.mark.parametrize("timestamp_type", ["TIMESTAMP_LTZ", "TIMESTAMP_NTZ"]) +def test_csv_infer_schema_timestamp_ntz_v2(spark_tmp_path, date_format, ts_part, timestamp_type): + csv_infer_schema_timestamp_ntz(spark_tmp_path, date_format, ts_part, timestamp_type, '', 'BatchScanExec') + +def csv_infer_schema_timestamp_ntz(spark_tmp_path, date_format, ts_part, timestamp_type, v1_enabled_list, cpu_scan_class): + full_format = date_format + ts_part + # specify to use no timezone rather than defaulting to UTC + data_gen = TimestampGen(tzinfo=None) + gen = StructGen([('a', data_gen)], nullable=False) + data_path = spark_tmp_path + '/CSV_DATA' + with_cpu_session( + lambda spark : gen_df(spark, gen).write + .option('timestampFormat', full_format) + .csv(data_path)) + + def do_read(spark): + return spark.read.option("inferSchema", "true") \ + .option('timestampFormat', full_format) \ + .csv(data_path) + + conf = { 'spark.sql.timestampType': timestamp_type, + 'spark.sql.sources.useV1SourceList': v1_enabled_list } + + # determine whether Spark CPU infers TimestampType or TimestampNtzType + inferred_type = with_cpu_session( + lambda spark : do_read(spark).schema["_c0"].dataType.typeName(), conf=conf) + + if inferred_type == "timestamp_ntz": + # we fall back to CPU due to "unsupported data types in output: TimestampNTZType" + assert_gpu_fallback_collect( + lambda spark: do_read(spark), + cpu_fallback_class_name = cpu_scan_class, + conf = conf) + else: + assert_cpu_and_gpu_are_equal_collect_with_capture( + lambda spark: do_read(spark), + exist_classes = 'Gpu' + cpu_scan_class, + non_exist_classes = cpu_scan_class, + conf = conf) + @allow_non_gpu('FileSourceScanExec', 'CollectLimitExec', 'DeserializeToObjectExec') @pytest.mark.skipif(is_before_spark_340(), reason='`preferDate` is only supported in Spark 340+') def test_csv_prefer_date_with_infer_schema(spark_tmp_path): diff --git a/integration_tests/src/main/python/data_gen.py b/integration_tests/src/main/python/data_gen.py index ab9890b0712..9f549adfa46 100644 --- a/integration_tests/src/main/python/data_gen.py +++ b/integration_tests/src/main/python/data_gen.py @@ -572,27 +572,28 @@ def start(self, rand): class TimestampGen(DataGen): """Generate Timestamps in a given range. All timezones are UTC by default.""" - def __init__(self, start=None, end=None, nullable=True): - super().__init__(TimestampType(), nullable=nullable) + def __init__(self, start=None, end=None, nullable=True, tzinfo=timezone.utc): + super().__init__(TimestampNTZType() if tzinfo==None else TimestampType(), nullable=nullable) if start is None: # Spark supports times starting at # "0001-01-01 00:00:00.000000" # but it has issues if you get really close to that because it tries to do things # in a different format which causes roundoff, so we have to add a few days, # just to be sure - start = datetime(1, 1, 3, tzinfo=timezone.utc) + start = datetime(1, 1, 3, tzinfo=tzinfo) elif not isinstance(start, datetime): raise RuntimeError('Unsupported type passed in for start {}'.format(start)) if end is None: # Spark supports time through # "9999-12-31 23:59:59.999999" - end = datetime(9999, 12, 31, 23, 59, 59, 999999, tzinfo=timezone.utc) + end = datetime(9999, 12, 31, 23, 59, 59, 999999, tzinfo=tzinfo) elif isinstance(end, timedelta): end = start + end elif not isinstance(start, date): raise RuntimeError('Unsupported type passed in for end {}'.format(end)) + self._epoch = datetime(1970, 1, 1, tzinfo=tzinfo) self._start_time = self._to_us_since_epoch(start) self._end_time = self._to_us_since_epoch(end) if (self._epoch >= start and self._epoch <= end): @@ -601,7 +602,6 @@ def __init__(self, start=None, end=None, nullable=True): def _cache_repr(self): return super()._cache_repr() + '(' + str(self._start_time) + ',' + str(self._end_time) + ')' - _epoch = datetime(1970, 1, 1, tzinfo=timezone.utc) _us = timedelta(microseconds=1) def _to_us_since_epoch(self, val): diff --git a/integration_tests/src/main/python/json_test.py b/integration_tests/src/main/python/json_test.py index 7c8b2499130..181e8f11bd2 100644 --- a/integration_tests/src/main/python/json_test.py +++ b/integration_tests/src/main/python/json_test.py @@ -15,8 +15,10 @@ import pyspark.sql.functions as f import pytest -from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_error, assert_gpu_and_cpu_row_counts_equal, assert_gpu_fallback_collect +from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_error, assert_gpu_and_cpu_row_counts_equal, \ + assert_gpu_fallback_collect, assert_cpu_and_gpu_are_equal_collect_with_capture from data_gen import * +from datetime import timezone from conftest import is_databricks_runtime from marks import approximate_float, allow_non_gpu, ignore_order from spark_session import with_cpu_session, with_gpu_session, is_before_spark_330 @@ -200,6 +202,58 @@ def test_json_ts_formats_round_trip(spark_tmp_path, date_format, ts_part, v1_ena .json(data_path), conf=updated_conf) +@allow_non_gpu('FileSourceScanExec', 'ProjectExec') +@pytest.mark.skipif(is_before_spark_340(), reason='`TIMESTAMP_NTZ` is only supported in Spark 340+') +@pytest.mark.parametrize('ts_part', json_supported_ts_parts) +@pytest.mark.parametrize('date_format', json_supported_date_formats) +@pytest.mark.parametrize("timestamp_type", ["TIMESTAMP_LTZ", "TIMESTAMP_NTZ"]) +def test_json_ts_formats_round_trip_ntz_v1(spark_tmp_path, date_format, ts_part, timestamp_type): + json_ts_formats_round_trip_ntz(spark_tmp_path, date_format, ts_part, timestamp_type, 'json', 'FileSourceScanExec') + +@allow_non_gpu('BatchScanExec', 'ProjectExec') +@pytest.mark.skipif(is_before_spark_340(), reason='`TIMESTAMP_NTZ` is only supported in Spark 340+') +@pytest.mark.parametrize('ts_part', json_supported_ts_parts) +@pytest.mark.parametrize('date_format', json_supported_date_formats) +@pytest.mark.parametrize("timestamp_type", ["TIMESTAMP_LTZ", "TIMESTAMP_NTZ"]) +def test_json_ts_formats_round_trip_ntz_v2(spark_tmp_path, date_format, ts_part, timestamp_type): + json_ts_formats_round_trip_ntz(spark_tmp_path, date_format, ts_part, timestamp_type, '', 'BatchScanExec') + +def json_ts_formats_round_trip_ntz(spark_tmp_path, date_format, ts_part, timestamp_type, v1_enabled_list, cpu_scan_class): + full_format = date_format + ts_part + data_gen = TimestampGen(tzinfo=None if timestamp_type == "TIMESTAMP_NTZ" else timezone.utc) + gen = StructGen([('a', data_gen)], nullable=False) + data_path = spark_tmp_path + '/JSON_DATA' + schema = gen.data_type + with_cpu_session( + lambda spark : gen_df(spark, gen).write \ + .option('timestampFormat', full_format) \ + .json(data_path)) + updated_conf = copy_and_update(_enable_all_types_conf, + { + 'spark.sql.sources.useV1SourceList': v1_enabled_list, + 'spark.sql.timestampType': timestamp_type + }) + + def do_read(spark): + return spark.read \ + .schema(schema) \ + .option('timestampFormat', full_format) \ + .json(data_path) + + + if timestamp_type == "TIMESTAMP_LTZ": + assert_cpu_and_gpu_are_equal_collect_with_capture( + lambda spark : do_read(spark), + exist_classes = 'Gpu' + cpu_scan_class, + non_exist_classes = cpu_scan_class, + conf=updated_conf) + else: + # we fall back to CPU due to "unsupported data types in output: TimestampNTZType" + assert_gpu_fallback_collect( + lambda spark : do_read(spark), + cpu_fallback_class_name = cpu_scan_class, + conf=updated_conf) + @approximate_float @pytest.mark.parametrize('filename', [ 'boolean.json', From 7d5b90419dbd26dea22a488b121e417c5542280f Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 27 Sep 2023 18:18:42 -0600 Subject: [PATCH 33/46] Improve some CSV integration tests [databricks] (#9146) * improve some csv tests * remove unrelated change Signed-off-by: Andy Grove * remove unrelated test * use named parameters to improve readability * remove trailing comma * update comment Signed-off-by: Andy Grove * address feedback (add link to issue for future improvements) --------- Signed-off-by: Andy Grove --- integration_tests/src/main/python/csv_test.py | 31 ++++++++++++++----- 1 file changed, 23 insertions(+), 8 deletions(-) diff --git a/integration_tests/src/main/python/csv_test.py b/integration_tests/src/main/python/csv_test.py index bae076f5a4d..5e5234d7033 100644 --- a/integration_tests/src/main/python/csv_test.py +++ b/integration_tests/src/main/python/csv_test.py @@ -463,7 +463,7 @@ def test_input_meta_fallback(spark_tmp_path, v1_enabled_list, disable_conf): updated_conf = copy_and_update(_enable_all_types_conf, { 'spark.sql.sources.useV1SourceList': v1_enabled_list, disable_conf: 'false'}) - assert_gpu_and_cpu_are_equal_collect( + assert_gpu_fallback_collect( lambda spark : spark.read.schema(gen.data_type)\ .csv(data_path)\ .filter(f.col('a') > 0)\ @@ -471,6 +471,7 @@ def test_input_meta_fallback(spark_tmp_path, v1_enabled_list, disable_conf): 'input_file_name()', 'input_file_block_start()', 'input_file_block_length()'), + cpu_fallback_class_name = 'FileSourceScanExec' if v1_enabled_list == 'csv' else 'BatchScanExec', conf=updated_conf) @allow_non_gpu('DataWritingCommandExec,ExecutedCommandExec,WriteFilesExec') @@ -529,16 +530,18 @@ def test_round_trip_for_interval(spark_tmp_path, v1_enabled_list): lambda spark: spark.read.schema(schema).csv(data_path), conf=updated_conf) -@allow_non_gpu(any = True) +@allow_non_gpu('FileSourceScanExec', 'CollectLimitExec', 'DeserializeToObjectExec') def test_csv_read_case_insensitivity(spark_tmp_path): gen_list = [('one', int_gen), ('tWo', byte_gen), ('THREE', boolean_gen)] data_path = spark_tmp_path + '/CSV_DATA' with_cpu_session(lambda spark: gen_df(spark, gen_list).write.option('header', True).csv(data_path)) - assert_gpu_and_cpu_are_equal_collect( + assert_cpu_and_gpu_are_equal_collect_with_capture( lambda spark: spark.read.option('header', True).csv(data_path).select('one', 'two', 'three'), - {'spark.sql.caseSensitive': 'false'} + exist_classes = 'GpuFileSourceScanExec', + non_exist_classes = 'FileSourceScanExec', + conf = {'spark.sql.caseSensitive': 'false'} ) @allow_non_gpu('FileSourceScanExec', 'CollectLimitExec', 'DeserializeToObjectExec') @@ -549,7 +552,13 @@ def test_csv_read_count(spark_tmp_path): with_cpu_session(lambda spark: gen_df(spark, gen_list).write.csv(data_path)) - assert_gpu_and_cpu_row_counts_equal(lambda spark: spark.read.csv(data_path)) + # TODO This does not really test that the GPU count actually runs on the GPU + # because this test has @allow_non_gpu for operators that fall back to CPU + # when Spark performs an initial scan to infer the schema. To resolve this + # we would need a new `assert_gpu_and_cpu_row_counts_equal_with_capture` function. + # Tracking issue: https://github.com/NVIDIA/spark-rapids/issues/9199 + assert_gpu_and_cpu_row_counts_equal(lambda spark: spark.read.csv(data_path), + conf = {'spark.rapids.sql.explain': 'ALL'}) @allow_non_gpu('FileSourceScanExec', 'ProjectExec', 'CollectLimitExec', 'DeserializeToObjectExec') @pytest.mark.skipif(is_before_spark_340(), reason='`TIMESTAMP_NTZ` is only supported in Spark 340+') @@ -613,12 +622,18 @@ def test_csv_prefer_date_with_infer_schema(spark_tmp_path): with_cpu_session(lambda spark: gen_df(spark, gen_list).write.csv(data_path)) - assert_gpu_and_cpu_are_equal_collect(lambda spark: spark.read.option("inferSchema", "true").csv(data_path)) - assert_gpu_and_cpu_are_equal_collect(lambda spark: spark.read.option("inferSchema", "true").option("preferDate", "false").csv(data_path)) + assert_cpu_and_gpu_are_equal_collect_with_capture( + lambda spark: spark.read.option("inferSchema", "true").csv(data_path), + exist_classes = 'GpuFileSourceScanExec', + non_exist_classes = 'FileSourceScanExec') + assert_cpu_and_gpu_are_equal_collect_with_capture( + lambda spark: spark.read.option("inferSchema", "true").option("preferDate", "false").csv(data_path), + exist_classes = 'GpuFileSourceScanExec', + non_exist_classes = 'FileSourceScanExec') @allow_non_gpu('FileSourceScanExec') @pytest.mark.skipif(is_before_spark_340(), reason='enableDateTimeParsingFallback is supported from Spark3.4.0') -@pytest.mark.parametrize('filename,schema',[("date.csv", _date_schema), ("date.csv", _ts_schema,), +@pytest.mark.parametrize('filename,schema',[("date.csv", _date_schema), ("date.csv", _ts_schema), ("ts.csv", _ts_schema)]) def test_csv_datetime_parsing_fallback_cpu_fallback(std_input_path, filename, schema): data_path = std_input_path + "/" + filename From 2db7561661016bdc045a854758ba43e92121144d Mon Sep 17 00:00:00 2001 From: Liangcai Li Date: Thu, 28 Sep 2023 09:53:14 +0800 Subject: [PATCH 34/46] Allow checkpoint and restore on non-deterministic expressions in GpuFilter and GpuProject (#9287) fix #7865 This PR is to support checkpoint and restore on non-deterministic expressions in GpuFilter and GpuProject. 1. Introduce a new interface named Retryable in Java to supprt Java UDF in the future for end users. A non-deterministic expression can implement this interface to make it retryable. 2. Introduce a new class named RapidsXORShiftRandom to provide access to the internal hashed seed. This is used by GpuRand to implement the Retryable interface. 3. Removed the useless CheckpointRestore trait, replaced by the new Retryable interface. 4. Update GpuRand to support checkpoint and restore. 5. Introduce two new memebers in GpuExpression, they are selfNonDeterministic and retryable. retryable is used to tell whether an expression is retryable. It will cover its children. While selfNonDeterministic indicates whether an expression itself is non-deterministic when its "deterministic" is false, excluding its children. An expression is actually a tree, and deterministic being false means there is at least one tree node is non-deterministic, but we need to know the exact nodes which are non-deterministic to check if it implements the Retryable. So selfNonDeterministic is created. --------- Signed-off-by: Firestarman --- dist/unshimmed-common-from-spark311.txt | 1 + .../main/java/com/nvidia/spark/Retryable.java | 49 ++++++ .../spark/rapids/ColumnarOutputWriter.scala | 3 +- .../nvidia/spark/rapids/GpuExpressions.scala | 53 ++++++ .../spark/rapids/GpuUserDefinedFunction.scala | 2 + .../spark/rapids/GpuWindowExpression.scala | 3 +- .../nvidia/spark/rapids/JoinGatherer.scala | 3 +- .../spark/rapids/RmmRapidsRetryIterator.scala | 25 +-- .../spark/rapids/basicPhysicalOperators.scala | 63 +++++--- .../expressions/GpuRandomExpressions.scala | 43 ++++- .../execution/python/GpuPythonUDF.scala | 1 + .../util/random/rapids/XORShiftRandom.scala | 46 ++++++ .../rapids/NonDeterministicRetrySuite.scala | 151 ++++++++++++++++++ .../nvidia/spark/rapids/WithRetrySuite.scala | 3 +- 14 files changed, 391 insertions(+), 55 deletions(-) create mode 100644 sql-plugin/src/main/java/com/nvidia/spark/Retryable.java create mode 100644 sql-plugin/src/main/scala/org/apache/spark/util/random/rapids/XORShiftRandom.scala create mode 100644 tests/src/test/scala/com/nvidia/spark/rapids/NonDeterministicRetrySuite.scala diff --git a/dist/unshimmed-common-from-spark311.txt b/dist/unshimmed-common-from-spark311.txt index b044c2cb4c2..cf67a19590a 100644 --- a/dist/unshimmed-common-from-spark311.txt +++ b/dist/unshimmed-common-from-spark311.txt @@ -5,6 +5,7 @@ com/nvidia/spark/ExclusiveModeGpuDiscoveryPlugin* com/nvidia/spark/GpuCachedBatchSerializer* com/nvidia/spark/ParquetCachedBatchSerializer* com/nvidia/spark/RapidsUDF* +com/nvidia/spark/Retryable* com/nvidia/spark/SQLPlugin* com/nvidia/spark/rapids/ColumnarRdd* com/nvidia/spark/rapids/GpuColumnVectorUtils* diff --git a/sql-plugin/src/main/java/com/nvidia/spark/Retryable.java b/sql-plugin/src/main/java/com/nvidia/spark/Retryable.java new file mode 100644 index 00000000000..db1bb0e1fe6 --- /dev/null +++ b/sql-plugin/src/main/java/com/nvidia/spark/Retryable.java @@ -0,0 +1,49 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 com.nvidia.spark; + +/** + * An interface that can be used to retry the processing on non-deterministic + * expressions on the GPU. + * + * GPU memory is a limited resource. When it runs out the RAPIDS Accelerator + * for Apache Spark will use several different strategies to try and free more + * GPU memory to let the query complete. + * One of these strategies is to roll back the processioning for one task, pause + * the task thread, then retry the task when more memory is available. This + * works transparently for any stateless deterministic processing. But technically + * an expression/UDF can be non-deterministic and/or keep state in between calls. + * This interface provides a checkpoint method to save any needed state, and a + * restore method to reset the state in the case of a retry. + * + * Please note that a retry is not isolated to a single expression, so a restore can + * be called even after the expression returned one or more batches of results. And + * each time checkpoint it called any previously saved state can be overwritten. + */ +public interface Retryable { + /** + * Save the state, so it can be restored in the case of a retry. + * (This is called inside a Spark task context on executors.) + */ + void checkpoint(); + + /** + * Restore the state that was saved by calling to "checkpoint". + * (This is called inside a Spark task context on executors.) + */ + void restore(); +} diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala index 278b19bb661..69157c046b6 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/ColumnarOutputWriter.scala @@ -21,6 +21,7 @@ import java.io.OutputStream import scala.collection.mutable import ai.rapids.cudf.{HostBufferConsumer, HostMemoryBuffer, NvtxColor, NvtxRange, TableWriter} +import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{splitSpillableInHalfByRows, withRestoreOnRetry, withRetry, withRetryNoSplit} @@ -186,7 +187,7 @@ abstract class ColumnarOutputWriter(context: TaskAttemptContext, /** Apply any necessary casts before writing batch out */ def transformAndClose(cb: ColumnarBatch): ColumnarBatch = cb - private val checkpointRestore = new CheckpointRestore { + private val checkpointRestore = new Retryable { override def checkpoint(): Unit = () override def restore(): Unit = dropBufferedData() } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala index 96a50e78cd4..c46862ab2aa 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuExpressions.scala @@ -17,6 +17,7 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.{ast, BinaryOp, BinaryOperable, ColumnVector, DType, Scalar, UnaryOp} +import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.Arm.{withResource, withResourceIfAllowed} import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.shims.{ShimBinaryExpression, ShimExpression, ShimTernaryExpression, ShimUnaryExpression} @@ -90,6 +91,19 @@ object GpuExpressionsUtils { case ga: GpuAlias => extractGpuLit(ga.child) case _ => None } + + /** + * Collect the Retryables from a Seq of expression. + */ + def collectRetryables(expressions: Seq[Expression]): Seq[Retryable] = { + // There should be no dependence between expression and its children for + // the checkpoint and restore operations. + expressions.flatMap { expr => + expr.collect { + case r: Retryable => r + } + } + } } /** @@ -169,10 +183,49 @@ trait GpuExpression extends Expression { * this is seen. */ def disableTieredProjectCombine: Boolean = hasSideEffects + + /** + * Whether an expression itself is non-deterministic when its "deterministic" is false, + * no matter whether it has any non-deterministic children. + * An expression is actually a tree, and deterministic being false means there is at + * least one tree node is non-deterministic, but we need to know the exact nodes which + * are non-deterministic to check if it implements the Retryable. + * + * Default to false because Spark checks only children by default in Expression. So it + * is non-deterministic iff it has non-deterministic children. + * + * NOTE When overriding "deterministic", this should be taken care of. + */ + val selfNonDeterministic: Boolean = false + + /** + * true means this expression can be used inside a retry block, otherwise false. + * An expression is retryable when + * - it is deterministic, or + * - when being non-deterministic, it is a Retryable and its children are all retryable. + */ + lazy val retryable: Boolean = deterministic || { + val childrenAllRetryable = children.forall(_.asInstanceOf[GpuExpression].retryable) + if (selfNonDeterministic || children.forall(_.deterministic)) { + // self is non-deterministic, so need to check if it is a Retryable. + // + // "selfNonDeterministic" should be reliable enough, but it is still good to + // do this check for one case we are 100% sure self is non-deterministic (its + // "deterministic" is false but its children are all deterministic). This can + // minimize the possibility of missing expressions that happen to forget + // overriding "selfNonDeterministic" correctly. + this.isInstanceOf[Retryable] && childrenAllRetryable + } else { + childrenAllRetryable + } + } } abstract class GpuLeafExpression extends GpuExpression with ShimExpression { override final def children: Seq[Expression] = Nil + + /* no children, so only self can be non-deterministic */ + override final val selfNonDeterministic: Boolean = !deterministic } trait GpuUnevaluable extends GpuExpression { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuUserDefinedFunction.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuUserDefinedFunction.scala index 7f362c68746..59f96cf3cdb 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuUserDefinedFunction.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuUserDefinedFunction.scala @@ -45,6 +45,7 @@ trait GpuUserDefinedFunction extends GpuExpression override def hasSideEffects: Boolean = true override lazy val deterministic: Boolean = udfDeterministic && children.forall(_.deterministic) + override val selfNonDeterministic: Boolean = !udfDeterministic private[this] val nvtxRangeName = s"UDF: $name" private[this] lazy val funcCls = TrampolineUtil.getSimpleName(function.getClass) @@ -107,6 +108,7 @@ trait GpuRowBasedUserDefinedFunction extends GpuExpression private[this] lazy val outputType = dataType.catalogString override lazy val deterministic: Boolean = udfDeterministic && children.forall(_.deterministic) + override val selfNonDeterministic: Boolean = !udfDeterministic override def hasSideEffects: Boolean = true override def columnarEval(batch: ColumnarBatch): GpuColumnVector = { diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala index 14b41e18eaf..9d9f4101e26 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuWindowExpression.scala @@ -20,6 +20,7 @@ import java.util.concurrent.TimeUnit import ai.rapids.cudf import ai.rapids.cudf.{BinaryOp, ColumnVector, DType, GroupByScanAggregation, RollingAggregation, RollingAggregationOnColumn, Scalar, ScanAggregation} +import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import com.nvidia.spark.rapids.GpuOverrides.wrapExpr import com.nvidia.spark.rapids.shims.{GpuWindowUtil, ShimExpression} @@ -835,7 +836,7 @@ trait GpuRunningWindowFunction extends GpuWindowFunction { * * which can be output. */ -trait BatchedRunningWindowFixer extends AutoCloseable with CheckpointRestore { +trait BatchedRunningWindowFixer extends AutoCloseable with Retryable { /** * Fix up `windowedColumnOutput` with any stored state from previous batches. * Like all window operations the input data will have been sorted by the partition diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/JoinGatherer.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/JoinGatherer.scala index 57260b61c4a..3a55ad94bfd 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/JoinGatherer.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/JoinGatherer.scala @@ -17,6 +17,7 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.{ColumnVector, ColumnView, DeviceMemoryBuffer, DType, GatherMap, NvtxColor, NvtxRange, OrderByArg, OutOfBoundsPolicy, Scalar, Table} +import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import org.apache.spark.TaskContext @@ -34,7 +35,7 @@ import org.apache.spark.sql.vectorized.ColumnarBatch * If the data is needed after `allowSpilling` is called the implementations should get the data * back and cache it again until allowSpilling is called once more. */ -trait LazySpillable extends AutoCloseable with CheckpointRestore { +trait LazySpillable extends AutoCloseable with Retryable { /** * Indicate that we are done using the data for now and it can be spilled. diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RmmRapidsRetryIterator.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RmmRapidsRetryIterator.scala index 7271cfd533f..8be9f37fa55 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RmmRapidsRetryIterator.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RmmRapidsRetryIterator.scala @@ -20,6 +20,7 @@ import scala.annotation.tailrec import scala.collection.mutable import ai.rapids.cudf.CudfColumnSizeOverflowException +import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import com.nvidia.spark.rapids.RapidsPluginImplicits._ import com.nvidia.spark.rapids.ScalableTaskCompletion.onTaskCompletion @@ -223,7 +224,7 @@ object RmmRapidsRetryIterator extends Logging { } /** - * withRestoreOnRetry for CheckpointRestore. This helper function calls `fn` with no input and + * withRestoreOnRetry for Retryable. This helper function calls `fn` with no input and * returns the result. In the event of an OOM Retry exception, it calls the restore() method * of the input and then throws the oom exception. This is intended to be used within the `fn` * of one of the withRetry* functions. It provides an opportunity to reset state in the case @@ -231,11 +232,11 @@ object RmmRapidsRetryIterator extends Logging { * * @param r a single item T * @param fn the work to perform. Takes no input and produces K - * @tparam T element type that must be a `CheckpointRestore` subclass + * @tparam T element type that must be a `Retryable` subclass * @tparam K `fn` result type * @return a single item of type K */ - def withRestoreOnRetry[T <: CheckpointRestore, K](r: T)(fn: => K): K = { + def withRestoreOnRetry[T <: Retryable, K](r: T)(fn: => K): K = { try { fn } catch { @@ -250,7 +251,7 @@ object RmmRapidsRetryIterator extends Logging { } /** - * withRestoreOnRetry for CheckpointRestore. This helper function calls `fn` with no input and + * withRestoreOnRetry for Retryable. This helper function calls `fn` with no input and * returns the result. In the event of an OOM Retry exception, it calls the restore() method * of the input and then throws the oom exception. This is intended to be used within the `fn` * of one of the withRetry* functions. It provides an opportunity to reset state in the case @@ -258,11 +259,11 @@ object RmmRapidsRetryIterator extends Logging { * * @param r a Seq of item T * @param fn the work to perform. Takes no input and produces K - * @tparam T element type that must be a `CheckpointRestore` subclass + * @tparam T element type that must be a `Retryable` subclass * @tparam K `fn` result type * @return a single item of type K */ - def withRestoreOnRetry[T <: CheckpointRestore, K](r: Seq[T])(fn: => K): K = { + def withRestoreOnRetry[T <: Retryable, K](r: Seq[T])(fn: => K): K = { try { fn } catch { @@ -673,18 +674,6 @@ object RmmRapidsRetryIterator extends Logging { } } -trait CheckpointRestore { - /** - * Save state so it can be restored in case of an OOM Retry. - */ - def checkpoint(): Unit - - /** - * Restore state that was checkpointed. - */ - def restore(): Unit -} - /** * This is a wrapper that turns a target size into an autocloseable to allow it to be used * in withRetry blocks. It is intended to be used to help with cases where the split calculation diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala index c59352473a7..9a66bf237ac 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/basicPhysicalOperators.scala @@ -21,10 +21,11 @@ import scala.collection.mutable.ArrayBuffer import ai.rapids.cudf import ai.rapids.cudf._ +import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} import com.nvidia.spark.rapids.GpuMetric._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ -import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{splitSpillableInHalfByRows, withRetry, withRetryNoSplit} +import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{splitSpillableInHalfByRows, withRestoreOnRetry, withRetry, withRetryNoSplit} import com.nvidia.spark.rapids.ScalableTaskCompletion.onTaskCompletion import com.nvidia.spark.rapids.jni.SplitAndRetryOOM import com.nvidia.spark.rapids.shims._ @@ -136,13 +137,15 @@ object GpuProjectExec { def projectWithRetrySingleBatch(sb: SpillableColumnarBatch, boundExprs: Seq[Expression]): ColumnarBatch = { - // First off we want to find/run all of the expressions that are non-deterministic + // First off we want to find/run all of the expressions that are not retryable, // These cannot be retried. - val (deterministicExprs, nonDeterministicExprs) = boundExprs.partition(_.deterministic) + val (retryableExprs, notRetryableExprs) = boundExprs.partition( + _.asInstanceOf[GpuExpression].retryable) + val retryables = GpuExpressionsUtils.collectRetryables(retryableExprs) - val snd = if (nonDeterministicExprs.nonEmpty) { + val snd = if (notRetryableExprs.nonEmpty) { withResource(sb.getColumnarBatch()) { cb => - Some(SpillableColumnarBatch(project(cb, nonDeterministicExprs), + Some(SpillableColumnarBatch(project(cb, notRetryableExprs), SpillPriorities.ACTIVE_ON_DECK_PRIORITY)) } } else { @@ -150,10 +153,13 @@ object GpuProjectExec { } withResource(snd) { snd => + retryables.foreach(_.checkpoint()) RmmRapidsRetryIterator.withRetryNoSplit { val deterministicResults = withResource(sb.getColumnarBatch()) { cb => - // For now we are just going to run all of these and deal with losing work... - project(cb, deterministicExprs) + withRestoreOnRetry(retryables) { + // For now we are just going to run all of these and deal with losing work... + project(cb, retryableExprs) + } } if (snd.isEmpty) { // We are done and the order should be the same so we don't need to do anything... @@ -493,14 +499,16 @@ case class GpuProjectAstExec( */ case class GpuTieredProject(exprTiers: Seq[Seq[GpuExpression]]) { /** - * Is everything deterministic. This can help with reliability in the common case. + * Is everything retryable. This can help with reliability in the common case. */ - lazy val areAllDeterministic = !exprTiers.exists { tier => + lazy val areAllRetryable = !exprTiers.exists { tier => tier.exists { expr => - !expr.deterministic + !expr.retryable } } + lazy val retryables: Seq[Retryable] = exprTiers.flatMap(GpuExpressionsUtils.collectRetryables) + lazy val outputTypes = exprTiers.last.map(_.dataType).toArray private[this] def getPassThroughIndex(tierIndex: Int, @@ -535,20 +543,22 @@ case class GpuProjectAstExec( private [this] def projectWithRetrySingleBatchInternal(sb: SpillableColumnarBatch, closeInputBatch: Boolean): ColumnarBatch = { - if (areAllDeterministic) { - // If all of the expressions are deterministic we can just run everything and retry it - // at the top level. If some things are non-deterministic we need to split them up and + if (areAllRetryable) { + // If all of the expressions are retryable we can just run everything and retry it + // at the top level. If some things are not retryable we need to split them up and // do the processing in a way that makes it so retries are more likely to succeed. - if (closeInputBatch) { - RmmRapidsRetryIterator.withRetryNoSplit(sb) { _ => - withResource(sb.getColumnarBatch()) { cb => - project(cb) - } - } + val sbToClose = if (closeInputBatch) { + Some(sb) } else { + None + } + withResource(sbToClose) { _ => + retryables.foreach(_.checkpoint()) RmmRapidsRetryIterator.withRetryNoSplit { withResource(sb.getColumnarBatch()) { cb => - project(cb) + withRestoreOnRetry(retryables) { + project(cb) + } } } } @@ -639,16 +649,16 @@ object GpuFilter { numOutputRows: GpuMetric, numOutputBatches: GpuMetric, filterTime: GpuMetric): Iterator[ColumnarBatch] = { - if (boundCondition.areAllDeterministic) { + if (boundCondition.areAllRetryable) { val sb = SpillableColumnarBatch(batch, SpillPriorities.ACTIVE_ON_DECK_PRIORITY) filterAndCloseWithRetry(sb, boundCondition, numOutputRows, numOutputBatches, filterTime) } else { - filterAndCloseNondeterministic(batch, boundCondition, numOutputRows, numOutputBatches, + filterAndCloseNoRetry(batch, boundCondition, numOutputRows, numOutputBatches, filterTime) } } - private def filterAndCloseNondeterministic(batch: ColumnarBatch, + private def filterAndCloseNoRetry(batch: ColumnarBatch, boundCondition: GpuTieredProject, numOutputRows: GpuMetric, numOutputBatches: GpuMetric, @@ -668,10 +678,13 @@ object GpuFilter { numOutputRows: GpuMetric, numOutputBatches: GpuMetric, opTime: GpuMetric): Iterator[ColumnarBatch] = { + boundCondition.retryables.foreach(_.checkpoint()) val ret = withRetry(input, splitSpillableInHalfByRows) { sb => withResource(sb.getColumnarBatch()) { cb => - withResource(new NvtxWithMetrics("filter batch", NvtxColor.YELLOW, opTime)) { _ => - GpuFilter(cb, boundCondition) + withRestoreOnRetry(boundCondition.retryables) { + withResource(new NvtxWithMetrics("filter batch", NvtxColor.YELLOW, opTime)) { _ => + GpuFilter(cb, boundCondition) + } } } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala index d6a0c4ad630..6675f678f6d 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/catalyst/expressions/GpuRandomExpressions.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.rapids.catalyst.expressions import ai.rapids.cudf.{DType, HostColumnVector, NvtxColor, NvtxRange} +import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.{GpuColumnVector, GpuExpression, GpuLiteral} import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.shims.ShimUnaryExpression @@ -27,11 +28,11 @@ import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils -import org.apache.spark.util.random.XORShiftRandom +import org.apache.spark.util.random.rapids.RapidsXORShiftRandom /** Generate a random column with i.i.d. uniformly distributed values in [0, 1). */ case class GpuRand(child: Expression) extends ShimUnaryExpression with GpuExpression - with ExpectsInputTypes with ExpressionWithRandomSeed { + with ExpectsInputTypes with ExpressionWithRandomSeed with Retryable { def this() = this(GpuLiteral(Utils.random.nextLong(), LongType)) @@ -40,12 +41,13 @@ case class GpuRand(child: Expression) extends ShimUnaryExpression with GpuExpres def seedExpression: Expression = child override lazy val deterministic: Boolean = false + override val selfNonDeterministic: Boolean = true /** * Record ID within each partition. By being transient, the Random Number Generator is * reset every time we serialize and deserialize and initialize it. */ - @transient protected var rng: XORShiftRandom = _ + @transient protected var rng: RapidsXORShiftRandom = _ @transient protected lazy val seed: Long = child match { case GpuLiteral(s, IntegerType) => s.asInstanceOf[Int] @@ -55,6 +57,9 @@ case class GpuRand(child: Expression) extends ShimUnaryExpression with GpuExpres } @transient protected var previousPartition: Int = 0 + + @transient protected var curXORShiftRandomSeed: Option[Long] = None + private def wasInitialized: Boolean = rng != null override def nullable: Boolean = false @@ -63,13 +68,23 @@ case class GpuRand(child: Expression) extends ShimUnaryExpression with GpuExpres override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(IntegerType, LongType)) + private def initRandom(): Unit = { + val partId = TaskContext.getPartitionId() + if (partId != previousPartition || !wasInitialized) { + rng = new RapidsXORShiftRandom(seed + partId) + previousPartition = partId + } + } + override def columnarEval(batch: ColumnarBatch): GpuColumnVector = { + if (curXORShiftRandomSeed.isEmpty) { + // checkpoint not called, need to init the random generator here + initRandom() + } else { + // make sure here uses the same random generator with checkpoint + assert(wasInitialized) + } withResource(new NvtxRange("GpuRand", NvtxColor.RED)) { _ => - val partId = TaskContext.getPartitionId() - if (partId != previousPartition || !wasInitialized) { - rng = new XORShiftRandom(seed + partId) - previousPartition = partId - } val numRows = batch.numRows() withResource(HostColumnVector.builder(DType.FLOAT64, numRows)) { builder => (0 until numRows).foreach(_ => builder.append(rng.nextDouble())) @@ -77,4 +92,16 @@ case class GpuRand(child: Expression) extends ShimUnaryExpression with GpuExpres } } } + + override def checkpoint(): Unit = { + // In a task, checkpoint is called before columnarEval, so need to try to + // init the random generator here. + initRandom() + curXORShiftRandomSeed = Some(rng.currentSeed) + } + + override def restore(): Unit = { + assert(wasInitialized && curXORShiftRandomSeed.isDefined) + rng.setHashedSeed(curXORShiftRandomSeed.get) + } } diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuPythonUDF.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuPythonUDF.scala index bc2cb433deb..0421802252d 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuPythonUDF.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/execution/python/GpuPythonUDF.scala @@ -68,6 +68,7 @@ abstract class GpuPythonFunction( with UserDefinedExpression with GpuAggregateWindowFunction with Serializable { override lazy val deterministic: Boolean = udfDeterministic && children.forall(_.deterministic) + override val selfNonDeterministic: Boolean = !udfDeterministic override def toString: String = s"$name(${children.mkString(", ")})" diff --git a/sql-plugin/src/main/scala/org/apache/spark/util/random/rapids/XORShiftRandom.scala b/sql-plugin/src/main/scala/org/apache/spark/util/random/rapids/XORShiftRandom.scala new file mode 100644 index 00000000000..1f32b56a6cd --- /dev/null +++ b/sql-plugin/src/main/scala/org/apache/spark/util/random/rapids/XORShiftRandom.scala @@ -0,0 +1,46 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.random.rapids + +import org.apache.spark.util.random.XORShiftRandom + +/** RAPIDS version of the Spark XORShiftRandom providing access to the internal seed. */ +class RapidsXORShiftRandom(init: Long) extends XORShiftRandom(init) { + + private var curSeed = XORShiftRandom.hashSeed(init) + + // Only override "next", since it will be called by other nextXXX. + override protected def next(bits: Int): Int = { + var nextSeed = curSeed ^ (curSeed << 21) + nextSeed ^= (nextSeed >>> 35) + nextSeed ^= (nextSeed << 4) + curSeed = nextSeed + (nextSeed & ((1L << bits) - 1)).asInstanceOf[Int] + } + + override def setSeed(s: Long): Unit = { + curSeed = XORShiftRandom.hashSeed(s) + } + + /* Set the hashed seed directly. (Not thread-safe) */ + def setHashedSeed(hashedSeed: Long): Unit = { + curSeed = hashedSeed + } + + /* Get the current internal seed. (Not thread-safe) */ + def currentSeed: Long = curSeed +} diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/NonDeterministicRetrySuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/NonDeterministicRetrySuite.scala new file mode 100644 index 00000000000..b0cecd80c87 --- /dev/null +++ b/tests/src/test/scala/com/nvidia/spark/rapids/NonDeterministicRetrySuite.scala @@ -0,0 +1,151 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 com.nvidia.spark.rapids + +import ai.rapids.cudf.{ColumnVector, Table} +import com.nvidia.spark.rapids.Arm.{closeOnExcept, withResource} +import com.nvidia.spark.rapids.RapidsPluginImplicits.AutoCloseableProducingSeq +import com.nvidia.spark.rapids.jni.RmmSpark + +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId} +import org.apache.spark.sql.rapids.GpuGreaterThan +import org.apache.spark.sql.rapids.catalyst.expressions.GpuRand +import org.apache.spark.sql.types.{DoubleType, IntegerType} +import org.apache.spark.sql.vectorized.ColumnarBatch + +class NonDeterministicRetrySuite extends RmmSparkRetrySuiteBase { + private val NUM_ROWS = 500 + private val RAND_SEED = 10 + private val batchAttrs = Seq(AttributeReference("int", IntegerType)(ExprId(10))) + + private def buildBatch(ints: Seq[Int] = 0 until NUM_ROWS): ColumnarBatch = { + new ColumnarBatch( + Array(GpuColumnVector.from(ColumnVector.fromInts(ints: _*), IntegerType)), ints.length) + } + + test("GPU rand outputs the same sequence with checkpoint and restore") { + val gpuRand = GpuRand(GpuLiteral(RAND_SEED, IntegerType)) + withResource(buildBatch()) { inputCB => + // checkpoint the state + gpuRand.checkpoint() + val randHCol1 = withResource(gpuRand.columnarEval(inputCB)) { randCol1 => + randCol1.copyToHost() + } + withResource(randHCol1) { _ => + assert(randHCol1.getRowCount.toInt == NUM_ROWS) + // Restore the state, and generate data again + gpuRand.restore() + val randHCol2 = withResource(gpuRand.columnarEval(inputCB)) { randCol2 => + randCol2.copyToHost() + } + withResource(randHCol2) { _ => + // check the two random columns are equal. + assert(randHCol1.getRowCount == randHCol2.getRowCount) + (0 until randHCol1.getRowCount.toInt).foreach { pos => + assert(randHCol1.getDouble(pos) == randHCol2.getDouble(pos)) + } + } + } + } + } + + test("GPU project retry with GPU rand") { + def projectRand(): Seq[GpuExpression] = Seq( + GpuAlias(GpuRand(GpuLiteral(RAND_SEED)), "rand")()) + + Seq(true, false).foreach { useTieredProject => + // expression should be retryable + val boundProjectRand = GpuBindReferences.bindGpuReferencesTiered(projectRand(), + batchAttrs, useTieredProject) + assert(boundProjectRand.areAllRetryable) + // project with and without retry + val batches = Seq(true, false).safeMap { forceRetry => + val boundProjectList = GpuBindReferences.bindGpuReferencesTiered( + projectRand() ++ batchAttrs, batchAttrs, useTieredProject) + assert(boundProjectList.areAllRetryable) + + val sb = closeOnExcept(buildBatch()) { cb => + SpillableColumnarBatch(cb, SpillPriorities.ACTIVE_ON_DECK_PRIORITY) + } + closeOnExcept(sb) { _ => + if (forceRetry) { + RmmSpark.forceRetryOOM(RmmSpark.getCurrentThreadId) + } + } + boundProjectList.projectAndCloseWithRetrySingleBatch(sb) + } + // check the random columns + val randCols = withResource(batches) { case Seq(retriedBatch, batch) => + assert(retriedBatch.numRows() == batch.numRows()) + assert(retriedBatch.numCols() == batch.numCols()) + batches.safeMap(_.column(0).asInstanceOf[GpuColumnVector].copyToHost()) + } + withResource(randCols) { case Seq(retriedRand, rand) => + (0 until rand.getRowCount.toInt).foreach { pos => + assert(retriedRand.getDouble(pos) == rand.getDouble(pos)) + } + } + } + } + + test("GPU filter retry with GPU rand") { + def filterRand(): Seq[GpuExpression] = Seq( + GpuGreaterThan( + GpuRand(GpuLiteral.create(RAND_SEED, IntegerType)), + GpuLiteral.create(0.1d, DoubleType))) + + Seq(true, false).foreach { useTieredProject => + // filter with and without retry + val tables = Seq(true, false).safeMap { forceRetry => + val boundCondition = GpuBindReferences.bindGpuReferencesTiered(filterRand(), + batchAttrs, useTieredProject) + assert(boundCondition.areAllRetryable) + + val cb = buildBatch() + if (forceRetry) { + RmmSpark.forceSplitAndRetryOOM(RmmSpark.getCurrentThreadId) + } + val batchSeq = GpuFilter.filterAndClose(cb, boundCondition, + NoopMetric, NoopMetric, NoopMetric).toSeq + withResource(batchSeq) { _ => + val tables = batchSeq.safeMap(GpuColumnVector.from) + if (tables.size == 1) { + tables.head + } else { + withResource(tables) { _ => + assert(tables.size > 1) + Table.concatenate(tables: _*) + } + } + } + } + + // check the outputs + val cols = withResource(tables) { case Seq(retriedTable, table) => + assert(retriedTable.getRowCount == table.getRowCount) + assert(retriedTable.getNumberOfColumns == table.getNumberOfColumns) + tables.safeMap(_.getColumn(0).copyToHost()) + } + withResource(cols) { case Seq(retriedInts, ints) => + (0 until ints.getRowCount.toInt).foreach { pos => + assert(retriedInts.getInt(pos) == ints.getInt(pos)) + } + } + } + } + +} diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/WithRetrySuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/WithRetrySuite.scala index 2354544606a..c808a36af85 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/WithRetrySuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/WithRetrySuite.scala @@ -17,6 +17,7 @@ package com.nvidia.spark.rapids import ai.rapids.cudf.{Rmm, RmmAllocationMode, RmmEventHandler, Table} +import com.nvidia.spark.Retryable import com.nvidia.spark.rapids.Arm.withResource import com.nvidia.spark.rapids.RmmRapidsRetryIterator.{splitTargetSizeInHalf, withRestoreOnRetry, withRetry, withRetryNoSplit} import com.nvidia.spark.rapids.jni.{RetryOOM, RmmSpark, SplitAndRetryOOM} @@ -297,7 +298,7 @@ class WithRetrySuite false } } - private class SimpleCheckpointRestore(var value:Int) extends CheckpointRestore { + private class SimpleCheckpointRestore(var value:Int) extends Retryable { private var lastValue:Int = value def setValue(newVal: Int) = { value = newVal From 186ca774af870fb17a56cce293ef9751238285af Mon Sep 17 00:00:00 2001 From: Jason Lowe Date: Thu, 28 Sep 2023 08:44:22 -0500 Subject: [PATCH 35/46] Fix RMM crash in FileCacheIntegrationSuite with ARENA memory allocator (#9314) Signed-off-by: Jason Lowe --- .../filecache/FileCacheIntegrationSuite.scala | 16 ++-------------- 1 file changed, 2 insertions(+), 14 deletions(-) diff --git a/tests/src/test/scala/org/apache/spark/sql/rapids/filecache/FileCacheIntegrationSuite.scala b/tests/src/test/scala/org/apache/spark/sql/rapids/filecache/FileCacheIntegrationSuite.scala index 931e6d95628..97cd4b02277 100644 --- a/tests/src/test/scala/org/apache/spark/sql/rapids/filecache/FileCacheIntegrationSuite.scala +++ b/tests/src/test/scala/org/apache/spark/sql/rapids/filecache/FileCacheIntegrationSuite.scala @@ -16,16 +16,15 @@ package org.apache.spark.sql.rapids.filecache -import com.nvidia.spark.rapids.{RapidsBufferCatalog, RapidsDeviceMemoryStore, SparkQueryCompareTestSuite} +import com.nvidia.spark.rapids.SparkQueryCompareTestSuite import com.nvidia.spark.rapids.shims.GpuBatchScanExec -import org.scalatest.BeforeAndAfterEach import org.apache.spark.SparkConf import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.rapids.GpuFileSourceScanExec -class FileCacheIntegrationSuite extends SparkQueryCompareTestSuite with BeforeAndAfterEach { +class FileCacheIntegrationSuite extends SparkQueryCompareTestSuite { import com.nvidia.spark.rapids.GpuMetric._ private val FILE_SPLITS_PARQUET = "file-splits.parquet" @@ -33,17 +32,6 @@ class FileCacheIntegrationSuite extends SparkQueryCompareTestSuite with BeforeAn private val MAP_OF_STRINGS_PARQUET = "map_of_strings.snappy.parquet" private val SCHEMA_CANT_PRUNE_ORC = "schema-cant-prune.orc" - override def beforeEach(): Unit = { - val deviceStorage = new RapidsDeviceMemoryStore() - val catalog = new RapidsBufferCatalog(deviceStorage) - RapidsBufferCatalog.setDeviceStorage(deviceStorage) - RapidsBufferCatalog.setCatalog(catalog) - } - - override def afterEach(): Unit = { - RapidsBufferCatalog.close() - } - def isFileCacheEnabled(conf: SparkConf): Boolean = { // File cache only supported on Spark 3.2+ assumeSpark320orLater From bfef57274929e8da735c8fd07cca85f99a601823 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Thu, 28 Sep 2023 22:33:34 +0800 Subject: [PATCH 36/46] [DOC] Fix some incorrect config links in doc [skip ci] (#9321) * fix incorrect links Signed-off-by: Haoyang Li * address comment Signed-off-by: Haoyang Li --------- Signed-off-by: Haoyang Li --- docs/FAQ.md | 2 +- docs/compatibility.md | 26 +++++++++---------- .../get-started/getting-started-databricks.md | 2 +- docs/tuning-guide.md | 4 +-- 4 files changed, 17 insertions(+), 17 deletions(-) diff --git a/docs/FAQ.md b/docs/FAQ.md index e2c7242ae46..1d920bfc7cb 100644 --- a/docs/FAQ.md +++ b/docs/FAQ.md @@ -458,7 +458,7 @@ files. Spark tends to prefer sort based joins, and in some cases sort based agg the GPU versions are all hash based. This means that the resulting data can come out in a different order for the CPU and the GPU. This is not wrong, but can make the size of the output data different because of compression. Users can turn on -[spark.rapids.sql.hashOptimizeSort.enabled](configs.md#sql.hashOptimizeSort.enabled) to have +[spark.rapids.sql.hashOptimizeSort.enabled](additional-functionality/advanced_configs.md#sql.hashOptimizeSort.enabled) to have the GPU try to replicate more closely what the output ordering would have been if sort were used, like on the CPU. diff --git a/docs/compatibility.md b/docs/compatibility.md index 01f9707e17a..e72415b634f 100644 --- a/docs/compatibility.md +++ b/docs/compatibility.md @@ -36,7 +36,7 @@ task/partition. The RAPIDS Accelerator does an unstable simply means that the sort algorithm allows for spilling parts of the data if it is larger than can fit in the GPU's memory, but it does not guarantee ordering of rows when the ordering of the keys is ambiguous. If you do rely on a stable sort in your processing you can request this by -setting [spark.rapids.sql.stableSort.enabled](configs.md#sql.stableSort.enabled) to `true` and +setting [spark.rapids.sql.stableSort.enabled](additional-functionality/advanced_configs.md#sql.stableSort.enabled) to `true` and RAPIDS will try to sort all the data for a given task/partition at once on the GPU. This may change in the future to allow for a spillable stable sort. @@ -67,7 +67,7 @@ joins on a floating point value, which is not wise to do anyways, and the value floating point aggregation then the join may fail to work properly with the plugin but would have worked with plain Spark. Starting from 22.06 this is behavior is enabled by default but can be disabled with the config -[`spark.rapids.sql.variableFloatAgg.enabled`](configs.md#sql.variableFloatAgg.enabled). +[`spark.rapids.sql.variableFloatAgg.enabled`](additional-functionality/advanced_configs.md#sql.variableFloatAgg.enabled). ### `0.0` vs `-0.0` @@ -513,13 +513,13 @@ GPU: WrappedArray([0], [19], [19], [19], [19], [19], [19], [19], [19], [19], [19 ``` To enable byte-range windowing on the GPU, set -[`spark.rapids.sql.window.range.byte.enabled`](configs.md#sql.window.range.byte.enabled) to true. +[`spark.rapids.sql.window.range.byte.enabled`](additional-functionality/advanced_configs.md#sql.window.range.byte.enabled) to true. We also provide configurations for other integral range types: -- [`spark.rapids.sql.window.range.short.enabled`](configs.md#sql.window.range.short.enabled) -- [`spark.rapids.sql.window.range.int.enabled`](configs.md#sql.window.range.int.enabled) -- [`spark.rapids.sql.window.range.long.enabled`](configs.md#sql.window.range.short.enabled) +- [`spark.rapids.sql.window.range.short.enabled`](additional-functionality/advanced_configs.md#sql.window.range.short.enabled) +- [`spark.rapids.sql.window.range.int.enabled`](additional-functionality/advanced_configs.md#sql.window.range.int.enabled) +- [`spark.rapids.sql.window.range.long.enabled`](additional-functionality/advanced_configs.md#sql.window.range.long.enabled) The reason why we default the configurations to false for byte/short and to true for int/long is that we think the most real-world queries are based on int or long. @@ -563,7 +563,7 @@ extensively tested and may produce different results compared to the CPU. Known values on GPU where Spark would treat the data as invalid and return null To attempt to use other formats on the GPU, set -[`spark.rapids.sql.incompatibleDateFormats.enabled`](configs.md#sql.incompatibleDateFormats.enabled) +[`spark.rapids.sql.incompatibleDateFormats.enabled`](additional-functionality/advanced_configs.md#sql.incompatibleDateFormats.enabled) to `true`. Formats that contain any of the following characters are unsupported and will fall back to CPU: @@ -585,7 +585,7 @@ Formats that contain any of the following words are unsupported and will fall ba ### LEGACY timeParserPolicy With timeParserPolicy set to `LEGACY` and -[`spark.rapids.sql.incompatibleDateFormats.enabled`](configs.md#sql.incompatibleDateFormats.enabled) +[`spark.rapids.sql.incompatibleDateFormats.enabled`](additional-functionality/advanced_configs.md#sql.incompatibleDateFormats.enabled) set to `true`, and `spark.sql.ansi.enabled` set to `false`, the following formats are supported but not guaranteed to produce the same results as the CPU: @@ -642,7 +642,7 @@ leads to restrictions: Starting from 22.06 this conf is enabled, to disable this operation on the GPU when using Spark 3.1.0 or later, set -[`spark.rapids.sql.castFloatToDecimal.enabled`](configs.md#sql.castFloatToDecimal.enabled) to `false` +[`spark.rapids.sql.castFloatToDecimal.enabled`](additional-functionality/advanced_configs.md#sql.castFloatToDecimal.enabled) to `false` ### Float to Integral Types @@ -653,7 +653,7 @@ starting with 3.1.0 these are now integral types such as `Int.MaxValue` so this affected the valid range of values and now differs slightly from the behavior on GPU in some cases. Starting from 22.06 this conf is enabled, to disable this operation on the GPU when using Spark 3.1.0 or later, set -[`spark.rapids.sql.castFloatToIntegralTypes.enabled`](configs.md#sql.castFloatToIntegralTypes.enabled) +[`spark.rapids.sql.castFloatToIntegralTypes.enabled`](additional-functionality/advanced_configs.md#sql.castFloatToIntegralTypes.enabled) to `false`. This configuration setting is ignored when using Spark versions prior to 3.1.0. @@ -665,7 +665,7 @@ types to strings. The GPU uses a lowercase `e` prefix for an exponent while Spar `E`. As a result the computed string can differ from the default behavior in Spark. Starting from 22.06 this conf is enabled by default, to disable this operation on the GPU, set -[`spark.rapids.sql.castFloatToString.enabled`](configs.md#sql.castFloatToString.enabled) to `false`. +[`spark.rapids.sql.castFloatToString.enabled`](additional-functionality/advanced_configs.md#sql.castFloatToString.enabled) to `false`. ### String to Float @@ -679,7 +679,7 @@ default behavior in Apache Spark is to return `+Infinity` and `-Infinity`, respe Also, the GPU does not support casting from strings containing hex values. Starting from 22.06 this conf is enabled by default, to enable this operation on the GPU, set -[`spark.rapids.sql.castStringToFloat.enabled`](configs.md#sql.castStringToFloat.enabled) to `false`. +[`spark.rapids.sql.castStringToFloat.enabled`](additional-functionality/advanced_configs.md#sql.castStringToFloat.enabled) to `false`. ### String to Date @@ -703,7 +703,7 @@ The following formats/patterns are supported on the GPU. Timezone of UTC is assu ### String to Timestamp To allow casts from string to timestamp on the GPU, enable the configuration property -[`spark.rapids.sql.castStringToTimestamp.enabled`](configs.md#sql.castStringToTimestamp.enabled). +[`spark.rapids.sql.castStringToTimestamp.enabled`](additional-functionality/advanced_configs.md#sql.castStringToTimestamp.enabled). Casting from string to timestamp currently has the following limitations. diff --git a/docs/get-started/getting-started-databricks.md b/docs/get-started/getting-started-databricks.md index f429d361fd9..459a637153c 100644 --- a/docs/get-started/getting-started-databricks.md +++ b/docs/get-started/getting-started-databricks.md @@ -107,7 +107,7 @@ cluster meets the prerequisites above by configuring it as follows: of python for Databricks. On Databricks, the python runtime requires different parameters than the Spark one, so a dedicated python deamon module `rapids.daemon_databricks` is created and should be specified here. Set the config - [`spark.rapids.sql.python.gpu.enabled`](../configs.md#sql.python.gpu.enabled) to `true` to + [`spark.rapids.sql.python.gpu.enabled`](../additional-functionality/advanced_configs.md#sql.python.gpu.enabled) to `true` to enable GPU support for python. Add the path of the plugin jar (supposing it is placed under `/databricks/jars/`) to the `spark.executorEnv.PYTHONPATH` option. For more details please go to [GPU Scheduling For Pandas UDF](../additional-functionality/rapids-udfs.md#gpu-support-for-pandas-udf) diff --git a/docs/tuning-guide.md b/docs/tuning-guide.md index 19935364124..2e61e72c425 100644 --- a/docs/tuning-guide.md +++ b/docs/tuning-guide.md @@ -46,11 +46,11 @@ If there are too many tasks this can increase the memory pressure on the GPU and spilling. ## Pooled Memory -Configuration key: [`spark.rapids.memory.gpu.pooling.enabled`](configs.md#memory.gpu.pooling.enabled) +Configuration key: [`spark.rapids.memory.gpu.pooling.enabled`](additional-functionality/advanced_configs.md#memory.gpu.pooling.enabled) Default value: `true` -Configuration key: [`spark.rapids.memory.gpu.allocFraction`](configs.md#memory.gpu.allocFraction) +Configuration key: [`spark.rapids.memory.gpu.allocFraction`](additional-functionality/advanced_configs.md#memory.gpu.allocFraction) Default value: `1.0` From d8a5e5e400854abee6d60055f937d18503eb270f Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Thu, 28 Sep 2023 13:34:15 -0700 Subject: [PATCH 37/46] Add GPU version of ToPrettyString [databricks] (#9221) * Added a GpuToPrettyString * Added unit tests because integration tests don't cover `DF.show()` --------- Signed-off-by: Raza Jafri --- .../src/main/python/cast_test.py | 7 +- .../com/nvidia/spark/rapids/GpuCast.scala | 170 +++++++++++------- .../com/nvidia/spark/rapids/TypeChecks.scala | 17 +- .../spark/rapids/shims/GpuCastShims.scala | 2 +- .../spark/rapids/shims/GpuCastShims.scala | 4 +- .../rapids/shims/GpuToPrettyString.scala | 67 +++++++ .../spark/rapids/shims/SparkShims.scala | 16 +- .../spark/rapids/GpuBatchUtilsSuite.scala | 101 ++++++----- .../spark/rapids/ToPrettyStringSuite.scala | 116 ++++++++++++ 9 files changed, 380 insertions(+), 120 deletions(-) create mode 100644 sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/GpuToPrettyString.scala create mode 100644 tests/src/test/spark350/scala/com/nvidia/spark/rapids/ToPrettyStringSuite.scala diff --git a/integration_tests/src/main/python/cast_test.py b/integration_tests/src/main/python/cast_test.py index 496002d931d..16c946b2811 100644 --- a/integration_tests/src/main/python/cast_test.py +++ b/integration_tests/src/main/python/cast_test.py @@ -16,8 +16,8 @@ from asserts import assert_gpu_and_cpu_are_equal_collect, assert_gpu_and_cpu_are_equal_sql, assert_gpu_and_cpu_error, assert_gpu_fallback_collect, assert_spark_exception from data_gen import * -from spark_session import is_before_spark_320, is_before_spark_330, is_spark_340_or_later, is_spark_350_or_later, \ - is_databricks113_or_later, with_gpu_session +from spark_session import is_before_spark_320, is_before_spark_330, is_spark_340_or_later, \ + is_databricks113_or_later from marks import allow_non_gpu, approximate_float from pyspark.sql.types import * from spark_init_internal import spark_version @@ -297,7 +297,6 @@ def _assert_cast_to_string_equal (data_gen, conf): @pytest.mark.parametrize('data_gen', all_array_gens_for_cast_to_string, ids=idfn) @pytest.mark.parametrize('legacy', ['true', 'false']) -@pytest.mark.xfail(condition=is_spark_350_or_later(), reason='https://github.com/NVIDIA/spark-rapids/issues/9065') def test_cast_array_to_string(data_gen, legacy): _assert_cast_to_string_equal( data_gen, @@ -317,7 +316,6 @@ def test_cast_array_with_unmatched_element_to_string(data_gen, legacy): @pytest.mark.parametrize('data_gen', basic_map_gens_for_cast_to_string, ids=idfn) @pytest.mark.parametrize('legacy', ['true', 'false']) -@pytest.mark.xfail(condition=is_spark_350_or_later(), reason='https://github.com/NVIDIA/spark-rapids/issues/9065') def test_cast_map_to_string(data_gen, legacy): _assert_cast_to_string_equal( data_gen, @@ -337,7 +335,6 @@ def test_cast_map_with_unmatched_element_to_string(data_gen, legacy): @pytest.mark.parametrize('data_gen', [StructGen([[str(i), gen] for i, gen in enumerate(basic_array_struct_gens_for_cast_to_string)] + [["map", MapGen(ByteGen(nullable=False), null_gen)]])], ids=idfn) @pytest.mark.parametrize('legacy', ['true', 'false']) -@pytest.mark.xfail(condition=is_spark_350_or_later(), reason='https://github.com/NVIDIA/spark-rapids/issues/9065') def test_cast_struct_to_string(data_gen, legacy): _assert_cast_to_string_equal( data_gen, diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala index 7a4bdb592b4..d9dded085c5 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuCast.scala @@ -50,24 +50,46 @@ final class CastExprMeta[INPUT <: UnaryExpression with TimeZoneAwareExpression w // We do not want to create a shim class for this small change stringToAnsiDate: Boolean, toTypeOverride: Option[DataType] = None) - extends UnaryExprMeta[INPUT](cast, conf, parent, rule) { + extends CastExprMetaBase(cast, conf, parent, rule, doFloatToIntCheck) { + + val legacyCastComplexTypesToString: Boolean = + SQLConf.get.getConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING) + override val toType: DataType = toTypeOverride.getOrElse(cast.dataType) + + override def tagExprForGpu(): Unit = { + if (evalMode == GpuEvalMode.TRY) { + willNotWorkOnGpu("try_cast is not supported on the GPU") + } + recursiveTagExprForGpuCheck() + } def withToTypeOverride(newToType: DecimalType): CastExprMeta[INPUT] = new CastExprMeta[INPUT](cast, evalMode, conf, parent, rule, doFloatToIntCheck, stringToAnsiDate, Some(newToType)) + override def convertToGpu(child: Expression): GpuExpression = + GpuCast(child, toType, evalMode == GpuEvalMode.ANSI, cast.timeZoneId, + legacyCastComplexTypesToString, stringToAnsiDate) + +} + +/** Meta-data for cast, ansi_cast and ToPrettyString */ +abstract class CastExprMetaBase[INPUT <: UnaryExpression with TimeZoneAwareExpression]( + cast: INPUT, + conf: RapidsConf, + parent: Option[RapidsMeta[_, _, _]], + rule: DataFromReplacementRule, + doFloatToIntCheck: Boolean = false) + extends UnaryExprMeta[INPUT](cast, conf, parent, rule) { + val fromType: DataType = cast.child.dataType - val toType: DataType = toTypeOverride.getOrElse(cast.dataType) - val legacyCastToString: Boolean = SQLConf.get.getConf(SQLConf.LEGACY_COMPLEX_TYPES_TO_STRING) + val toType: DataType = cast.dataType override def tagExprForGpu(): Unit = { - if (evalMode == GpuEvalMode.TRY) { - willNotWorkOnGpu("try_cast is not supported on the GPU") - } recursiveTagExprForGpuCheck() } - private def recursiveTagExprForGpuCheck( + protected def recursiveTagExprForGpuCheck( fromDataType: DataType = fromType, toDataType: DataType = toType, depth: Int = 0): Unit = { @@ -155,10 +177,6 @@ final class CastExprMeta[INPUT <: UnaryExpression with TimeZoneAwareExpression w s"${entry.doc}. To enable this operation on the GPU, set ${entry.key} to true." } - override def convertToGpu(child: Expression): GpuExpression = - GpuCast(child, toType, evalMode == GpuEvalMode.ANSI, cast.timeZoneId, legacyCastToString, - stringToAnsiDate) - // timezone tagging in type checks is good enough, so always false override protected val needTimezoneTagging: Boolean = false } @@ -166,9 +184,22 @@ final class CastExprMeta[INPUT <: UnaryExpression with TimeZoneAwareExpression w object CastOptions { val DEFAULT_CAST_OPTIONS = new CastOptions(false, false, false) val ARITH_ANSI_OPTIONS = new CastOptions(false, true, false) + val TO_PRETTY_STRING_OPTIONS = ToPrettyStringOptions def getArithmeticCastOptions(failOnError: Boolean): CastOptions = if (failOnError) ARITH_ANSI_OPTIONS else DEFAULT_CAST_OPTIONS + + object ToPrettyStringOptions extends CastOptions(false, false, false) { + override val leftBracket: String = "{" + + override val rightBracket: String = "}" + + override val nullString: String = "NULL" + + override val useDecimalPlainString: Boolean = true + + override val useHexFormatForBinary: Boolean = true + } } /** @@ -281,8 +312,12 @@ object GpuCast { case (DateType, BooleanType | _: NumericType) => // casts from date type to numerics are always null GpuColumnVector.columnVectorFromNull(input.getRowCount.toInt, toDataType) - case (DateType, StringType) => - input.asStrings("%Y-%m-%d") + + // Cast to String + case (DateType | TimestampType | FloatType | DoubleType | BinaryType | + _: DecimalType | _: ArrayType | _: MapType | _: StructType, StringType) => + castToString(input, fromDataType, options) + case (TimestampType, FloatType | DoubleType) => withResource(input.castTo(DType.INT64)) { asLongs => @@ -301,13 +336,13 @@ object GpuCast { toDataType match { case IntegerType => assertValuesInRange[Long](cv, Int.MinValue.toLong, - Int.MaxValue.toLong, errorMsg = GpuCast.OVERFLOW_MESSAGE) + Int.MaxValue.toLong, errorMsg = OVERFLOW_MESSAGE) case ShortType => assertValuesInRange[Long](cv, Short.MinValue.toLong, - Short.MaxValue.toLong, errorMsg = GpuCast.OVERFLOW_MESSAGE) + Short.MaxValue.toLong, errorMsg = OVERFLOW_MESSAGE) case ByteType => assertValuesInRange[Long](cv, Byte.MinValue.toLong, - Byte.MaxValue.toLong, errorMsg = GpuCast.OVERFLOW_MESSAGE) + Byte.MaxValue.toLong, errorMsg = OVERFLOW_MESSAGE) } } cv.castTo(GpuColumnVector.getNonNestedRapidsType(toDataType)) @@ -320,12 +355,6 @@ object GpuCast { asLongs.floorDiv(microsPerSec, GpuColumnVector.getNonNestedRapidsType(toDataType)) } } - case (TimestampType, StringType) => - castTimestampToString(input) - - case (StructType(fields), StringType) => - castStructToString(input, fields, options) - // ansi cast from larger-than-long integral-like types, to long case (dt: DecimalType, LongType) if ansiMode => // This is a work around for https://github.com/rapidsai/cudf/issues/9282 @@ -338,7 +367,7 @@ object GpuCast { withResource(input.max()) { maxInput => if (minInput.isValid && minInput.getBigDecimal().compareTo(min) == -1 || maxInput.isValid && maxInput.getBigDecimal().compareTo(max) == 1) { - throw new ArithmeticException(GpuCast.OVERFLOW_MESSAGE) + throw new ArithmeticException(OVERFLOW_MESSAGE) } } } @@ -501,9 +530,7 @@ object GpuCast { withResource(FloatUtils.nanToZero(input)) { inputWithNansToZero => inputWithNansToZero.castTo(GpuColumnVector.getNonNestedRapidsType(toDataType)) } - case (FloatType | DoubleType, StringType) => - castFloatingTypeToString(input) - case (StringType, ByteType | ShortType | IntegerType | LongType ) => + case (StringType, ByteType | ShortType | IntegerType | LongType) => CastStrings.toInteger(input, ansiMode, GpuColumnVector.getNonNestedRapidsType(toDataType)) case (StringType, FloatType | DoubleType) => @@ -533,12 +560,6 @@ object GpuCast { case (ShortType | IntegerType | LongType | ByteType | StringType, BinaryType) => input.asByteList(true) - case (BinaryType, StringType) => - castBinToString(input) - - case (_: DecimalType, StringType) => - GpuCastShims.CastDecimalToString(input, ansiMode) - case (ArrayType(nestedFrom, _), ArrayType(nestedTo, _)) => withResource(input.getChildColumnView(0)) { childView => withResource(doCast(childView, nestedFrom, nestedTo, options)) { childColumnVector => @@ -546,18 +567,12 @@ object GpuCast { } } - case (ArrayType(elementType, _), StringType) => - castArrayToString(input, elementType, options) - case (from: StructType, to: StructType) => castStructToStruct(from, to, input, options) case (from: MapType, to: MapType) => castMapToMap(from, to, input, options) - case (from: MapType, _: StringType) => - castMapToString(input, from, options) - case (dayTime: DataType, _: StringType) if GpuTypeShims.isSupportedDayTimeType(dayTime) => GpuIntervalUtils.toDayTimeIntervalString(input, dayTime) @@ -618,7 +633,7 @@ object GpuCast { maxValue: T, inclusiveMin: Boolean = true, inclusiveMax: Boolean = true, - errorMsg:String = GpuCast.OVERFLOW_MESSAGE) + errorMsg: String = OVERFLOW_MESSAGE) (implicit ord: Ordering[T]): Unit = { def throwIfAnyNan(): Unit = { @@ -705,6 +720,25 @@ object GpuCast { } } + def castToString( + input: ColumnView, + fromDataType: DataType, options: CastOptions): ColumnVector = fromDataType match { + case StringType => input.copyToColumnVector() + case DateType => input.asStrings("%Y-%m-%d") + case TimestampType => castTimestampToString(input) + case FloatType | DoubleType => castFloatingTypeToString(input) + case BinaryType => castBinToString(input, options) + case _: DecimalType => GpuCastShims.CastDecimalToString(input, options.useDecimalPlainString) + case StructType(fields) => castStructToString(input, fields, options) + + case ArrayType(elementType, _) => + castArrayToString(input, elementType, options) + case from: MapType => + castMapToString(input, from, options) + case _ => + input.castTo(GpuColumnVector.getNonNestedRapidsType(StringType)) + } + private def castTimestampToString(input: ColumnView): ColumnVector = { // the complexity in this function is due to Spark's rules for truncating // the fractional part of the timestamp string. Any trailing decimal place @@ -816,8 +850,8 @@ object GpuCast { private def castArrayToString(input: ColumnView, elementType: DataType, - options: CastOptions): ColumnVector = { - + options: CastOptions, + castingBinaryData: Boolean = false): ColumnVector = { // We use square brackets for arrays regardless val (leftStr, rightStr) = ("[", "]") val emptyStr = "" @@ -833,7 +867,7 @@ object GpuCast { val concatenated = withResource(strChildContainsNull) { _ => withResource(input.replaceListChild(strChildContainsNull)) { - concatenateStringArrayElements(_, options) + concatenateStringArrayElements(_, options, castingBinaryData) } } @@ -863,13 +897,11 @@ object GpuCast { // cast the key column and value column to string columns val (strKey, strValue) = withResource(input.getChildColumnView(0)) { kvStructColumn => val strKey = withResource(kvStructColumn.getChildColumnView(0)) { keyColumn => - doCast( - keyColumn, from.keyType, StringType, options) + castToString(keyColumn, from.keyType, options) } - val strValue = closeOnExcept(strKey) {_ => + val strValue = closeOnExcept(strKey) { _ => withResource(kvStructColumn.getChildColumnView(1)) { valueColumn => - doCast( - valueColumn, from.valueType, StringType, options) + castToString(valueColumn, from.valueType, options) } } (strKey, strValue) @@ -950,7 +982,7 @@ object GpuCast { // 3.1+: {firstCol columns += leftColumn.incRefCount() withResource(input.getChildColumnView(0)) { firstColumnView => - columns += doCast(firstColumnView, inputSchema.head.dataType, StringType, options) + columns += castToString(firstColumnView, inputSchema.head.dataType, options) } for (nonFirstIndex <- 1 until numInputColumns) { withResource(input.getChildColumnView(nonFirstIndex)) { nonFirstColumnView => @@ -1024,7 +1056,7 @@ object GpuCast { if (ansiEnabled) { withResource(validBools.all()) { isAllBool => if (isAllBool.isValid && !isAllBool.getBoolean) { - throw new IllegalStateException(GpuCast.INVALID_INPUT_MESSAGE) + throw new IllegalStateException(INVALID_INPUT_MESSAGE) } } } @@ -1065,8 +1097,8 @@ object GpuCast { } } - /** This method does not close the `input` ColumnVector. */ - def convertDateOr( + /** This method does not close the `input` ColumnVector. */ + def convertDateOr( input: ColumnVector, regex: String, cudfFormat: String, @@ -1335,17 +1367,27 @@ object GpuCast { } } - private def castBinToString(input: ColumnView): ColumnVector = { - // Spark interprets the binary as UTF-8 bytes. So the layout of the - // binary and the layout of the string are the same. We just need to play some games with - // the CPU side metadata to make CUDF think it is a String. - // Sadly there is no simple CUDF API to do this, so for now we pull it apart and put - // it back together again - withResource(input.getChildColumnView(0)) { dataCol => - withResource(new ColumnView(DType.STRING, input.getRowCount, - Optional.of[java.lang.Long](input.getNullCount), - dataCol.getData, input.getValid, input.getOffsets)) { cv => - cv.copyToColumnVector() + private def castBinToString(input: ColumnView, options: CastOptions): ColumnVector = { + if (options.useHexFormatForBinary) { + withResource(input.getChildColumnView(0)) { dataCol => + withResource(dataCol.toHex()) { stringCol => + withResource(input.replaceListChild(stringCol)) { cv => + castArrayToString(cv, DataTypes.StringType, options, true) + } + } + } + } else { + // Spark interprets the binary as UTF-8 bytes. So the layout of the + // binary and the layout of the string are the same. We just need to play some games with + // the CPU side metadata to make CUDF think it is a String. + // Sadly there is no simple CUDF API to do this, so for now we pull it apart and put + // it back together again + withResource(input.getChildColumnView(0)) { dataCol => + withResource(new ColumnView(DType.STRING, input.getRowCount, + Optional.of[java.lang.Long](input.getNullCount), + dataCol.getData, input.getValid, input.getOffsets)) { cv => + cv.copyToColumnVector() + } } } } @@ -1433,7 +1475,7 @@ object GpuCast { if (ansiMode) { withResource(outOfBounds.any()) { isAny => if (isAny.isValid && isAny.getBoolean) { - throw RapidsErrorUtils.arithmeticOverflowError(GpuCast.OVERFLOW_MESSAGE) + throw RapidsErrorUtils.arithmeticOverflowError(OVERFLOW_MESSAGE) } } input.copyToColumnVector() @@ -1534,7 +1576,7 @@ object GpuCast { val cv = withResource(updatedMaxRet) { updatedMax => withResource(Seq(minSeconds, Long.MinValue).safeMap(Scalar.fromLong)) { case Seq(minSecondsS, longMinS) => - withResource(longInput.lessThan(minSecondsS)){ + withResource(longInput.lessThan(minSecondsS)) { _.ifElse(longMinS, updatedMax) } } diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala index 049c0c5d230..b50029ab344 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/TypeChecks.scala @@ -1476,7 +1476,7 @@ class CastChecks extends ExprChecks { } } - private[this] def tagBase(meta: RapidsMeta[_, _, _], willNotWork: String => Unit): Unit = { + protected def tagBase(meta: RapidsMeta[_, _, _], willNotWork: String => Unit): Unit = { val cast = meta.wrapped.asInstanceOf[UnaryExpression] val from = cast.child.dataType val to = cast.dataType @@ -1506,6 +1506,21 @@ class CastChecks extends ExprChecks { } } +/** + * This class is just restricting the 'to' dataType to a StringType in the CastChecks class + */ +class ToPrettyStringChecks extends CastChecks { + + override protected def tagBase(meta: RapidsMeta[_, _, _], willNotWork: String => Unit): Unit = { + val cast = meta.wrapped.asInstanceOf[UnaryExpression] + val from = cast.child.dataType + val to = StringType + if (!gpuCanCast(from, to)) { + willNotWork(s"${meta.wrapped.getClass.getSimpleName} from $from to $to is not supported") + } + } +} + object ExprChecks { /** * A check for an expression that only supports project. diff --git a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala index 3de41de76c2..ffff1d9e2c1 100644 --- a/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala +++ b/sql-plugin/src/main/spark311/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala @@ -38,7 +38,7 @@ import ai.rapids.cudf.{ColumnVector, ColumnView} import com.nvidia.spark.rapids.jni.CastStrings object GpuCastShims { - def CastDecimalToString(decimalInput: ColumnView, ansiMode: Boolean): ColumnVector = { + def CastDecimalToString(decimalInput: ColumnView, usePlainString: Boolean): ColumnVector = { CastStrings.fromDecimal(decimalInput) } } diff --git a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala index 60fdd1e5e72..2493d81d9de 100644 --- a/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala +++ b/sql-plugin/src/main/spark330db/scala/com/nvidia/spark/rapids/shims/GpuCastShims.scala @@ -38,8 +38,8 @@ object GpuCastShims { case _ => throw new IllegalArgumentException(s"Unsupported type in cast $t") } - def CastDecimalToString(decimalInput: ColumnView, ansiMode: Boolean): ColumnVector = { - if (ansiMode) { + def CastDecimalToString(decimalInput: ColumnView, usePlainString: Boolean): ColumnVector = { + if (usePlainString) { // This is equivalent to // https://docs.oracle.com/javase/8/docs/api/java/math/BigDecimal.html#toPlainString-- // except there are a few corner cases, but they are really rare diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/GpuToPrettyString.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/GpuToPrettyString.scala new file mode 100644 index 00000000000..521e2e326b8 --- /dev/null +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/GpuToPrettyString.scala @@ -0,0 +1,67 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "350"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids.shims + +import ai.rapids.cudf.Scalar +import com.nvidia.spark.rapids._ +import com.nvidia.spark.rapids.Arm._ +import com.nvidia.spark.rapids.RapidsPluginImplicits._ + +import org.apache.spark.sql.catalyst.expressions.{Expression, TimeZoneAwareExpression} +import org.apache.spark.sql.types.{DataType, StringType} +import org.apache.spark.sql.vectorized.ColumnarBatch + +case class GpuToPrettyString(child: Expression, timeZoneId: Option[String] = None) + extends ShimUnaryExpression with GpuExpression with TimeZoneAwareExpression { + + override lazy val resolved: Boolean = childrenResolved + + override def dataType: DataType = StringType + + override def nullable: Boolean = false + + override def withTimeZone(timeZoneId: String): GpuToPrettyString = + copy(timeZoneId = Some(timeZoneId)) + + override def withNewChildInternal(newChild: Expression): Expression = + copy(child = newChild) + + override def columnarEval(batch: ColumnarBatch): GpuColumnVector = { + withResource(child.columnarEval(batch)) { evaluatedCol => + withResource(GpuCast.doCast( + evaluatedCol.getBase, + evaluatedCol.dataType(), + StringType, + CastOptions.TO_PRETTY_STRING_OPTIONS)) { possibleStringResult => + if (possibleStringResult.hasNulls) { + withResource(possibleStringResult.isNull) { isNull => + val stringColWithNulls = possibleStringResult + withResource(Scalar.fromString(CastOptions.TO_PRETTY_STRING_OPTIONS.nullString)) { + nullString => + GpuColumnVector.from(isNull.ifElse(nullString, stringColWithNulls), StringType) + } + } + } else { + GpuColumnVector.from(possibleStringResult.incRefCount(), StringType) + } + } + } + } +} \ No newline at end of file diff --git a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/SparkShims.scala b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/SparkShims.scala index 0f4a4bf66d5..75a13143a94 100644 --- a/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/SparkShims.scala +++ b/sql-plugin/src/main/spark350/scala/com/nvidia/spark/rapids/shims/SparkShims.scala @@ -21,13 +21,27 @@ package com.nvidia.spark.rapids.shims import com.nvidia.spark.rapids._ -import org.apache.spark.sql.catalyst.expressions.{Expression, PythonUDAF} +import org.apache.spark.sql.catalyst.expressions.{Expression, PythonUDAF, ToPrettyString} import org.apache.spark.sql.rapids.execution.python.GpuPythonUDAF +import org.apache.spark.sql.types.StringType object SparkShimImpl extends Spark340PlusShims { override def getExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = { val shimExprs: Map[Class[_ <: Expression], ExprRule[_ <: Expression]] = Seq( + GpuOverrides.expr[ToPrettyString]("An internal expressions which is used to " + + "generate pretty string for all kinds of values", + new ToPrettyStringChecks(), + (toPrettyString, conf, p, r) => { + new CastExprMetaBase[ToPrettyString](toPrettyString, conf, p, r) { + + override val toType: StringType.type = StringType + + override def convertToGpu(child: Expression): GpuExpression = { + GpuToPrettyString(child) + } + } + }), GpuOverrides.expr[PythonUDAF]( "UDF run in an external python process. Does not actually run on the GPU, but " + "the transfer of data to/from it can be accelerated", diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala index 5190f4434a4..87b73e1cd10 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/GpuBatchUtilsSuite.scala @@ -25,6 +25,7 @@ import org.scalatest.funsuite.AnyFunSuite import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, GenericRow} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -207,7 +208,12 @@ object GpuBatchUtilsSuite { externalRows.toArray } - private def createValueForType(i: Int, r: Random, dt: DataType, nullable: Boolean): Any = { + private def createValueForType( + i: Int, + r: Random, + dt: DataType, + nullable: Boolean, + isInternal: Boolean = true): Any = { dt match { case DataTypes.BooleanType => maybeNull(nullable, i, r.nextBoolean()) case DataTypes.ByteType => maybeNull(nullable, i, r.nextInt().toByte) @@ -225,30 +231,43 @@ object GpuBatchUtilsSuite { case dataType: DecimalType => val upperBound = (0 until dataType.precision).foldLeft(1L)((x, _) => x * 10) val unScaledValue = r.nextLong() % upperBound - maybeNull(nullable, i, Decimal(unScaledValue, dataType.precision, dataType.scale)) + val d = maybeNull(nullable, i, Decimal(unScaledValue, dataType.precision, dataType.scale)) + if (d != null && !isInternal) { + d.asInstanceOf[Decimal].toJavaBigDecimal + } else { + d + } case dataType@DataTypes.StringType => + val length = if (nullable) dataType.defaultSize * 2 else dataType.defaultSize + val utf8String = createUTF8String(length) + val string = if (!isInternal) utf8String.toString() else utf8String + if (nullable) { // since we want a deterministic test that compares the estimate with actual // usage we need to make sure the average length of strings is `dataType.defaultSize` if (i % 2 == 0) { null } else { - createUTF8String(dataType.defaultSize * 2) + string } } else { - createUTF8String(dataType.defaultSize) + string } case dataType@DataTypes.BinaryType => + val length = if (nullable) dataType.defaultSize * 2 else dataType.defaultSize + val bytes = r.nextString(length).getBytes + val binary = if (!isInternal) bytes.toSeq else bytes + if (nullable) { // since we want a deterministic test that compares the estimate with actual usage we // need to make sure the average length of binary values is `dataType.defaultSize` if (i % 2 == 0) { null } else { - r.nextString(dataType.defaultSize * 2).getBytes + binary } } else { - r.nextString(dataType.defaultSize).getBytes + binary } case ArrayType(elementType, containsNull) => if (nullable && i % 2 == 0) { @@ -256,33 +275,48 @@ object GpuBatchUtilsSuite { } else { val arrayValues = new mutable.ArrayBuffer[Any]() for (_ <- 0 to r.nextInt(10)) { - arrayValues.append(createValueForType(i, r, elementType, containsNull)) + arrayValues.append(createValueForType(i, r, elementType, containsNull, isInternal)) + } + val array = ArrayData.toArrayData(arrayValues) + if (!isInternal && array != null) { + array.toSeq(elementType) + } else { + array } - arrayValues.toArray.toSeq } - case MapType(_, _, valueContainsNull) => + case MapType(keyType, valueType, valueContainsNull) => if (nullable && i % 2 == 0) { null } else { - // TODO: add other types - val map = mutable.Map[String, String]() - for ( j <- 0 until 10) { + val map = mutable.Map[Any, Any]() + for (j <- 0 until 10) { if (valueContainsNull && j % 2 == 0) { - map += (createUTF8String(10).toString -> null) + map += (createValueForType(i, r, keyType, nullable = false, isInternal) -> null) } else { - map += (createUTF8String(10).toString -> createUTF8String(10).toString) + map += (createValueForType(i, r, keyType, nullable = false, isInternal) -> + createValueForType(i, r, valueType, nullable = false, isInternal)) } } - map + val mapData = ArrayBasedMapData(map) + if (mapData != null && !isInternal) { + ArrayBasedMapData.toScalaMap(mapData) + } else { + mapData + } } case StructType(fields) => - new GenericRow(fields.map(f => createValueForType(i, r, f.dataType, nullable))) - case unknown => throw new UnsupportedOperationException( + if (!isInternal) { + new GenericRow(fields.map(f => + createValueForType(i, r, f.dataType, nullable = f.nullable, isInternal = false))) + } else { + InternalRow(fields.map(f => createValueForType(i, r, f.dataType, nullable)): _*) + } + + case unknown => throw new UnsupportedOperationException( s"Type $unknown not supported") } } - private def createRowValues(i: Int, r: Random, fields: Array[StructField]) = { val values: Array[Any] = fields.map(field => { createValueForType(i, r, field.dataType, field.nullable) @@ -291,34 +325,9 @@ object GpuBatchUtilsSuite { } private def createExternalRowValues(i: Int, r: Random, fields: Array[StructField]): Array[Any] = { - val values: Array[Any] = fields.map(field => { - field.dataType match { - // Since it's using the createUTF8String method for InternalRow case, need to convert to - // String for Row case. - case StringType => - val utf8StringOrNull = createValueForType(i, r, field.dataType, field.nullable) - if (utf8StringOrNull != null) { - utf8StringOrNull.asInstanceOf[UTF8String].toString - } else { - utf8StringOrNull - } - case BinaryType => - val b = createValueForType(i, r, field.dataType, field.nullable) - if (b != null) { - b.asInstanceOf[Array[Byte]].toSeq - } else { - b - } - case DecimalType() => - val d = createValueForType(i, r, field.dataType, field.nullable) - if (d != null) { - d.asInstanceOf[Decimal].toJavaBigDecimal - } else { - d - } - case _ => createValueForType(i, r, field.dataType, field.nullable) - } - }) + val values: Array[Any] = fields.map { field => + createValueForType(i, r, field.dataType, field.nullable, isInternal = false) + } values } diff --git a/tests/src/test/spark350/scala/com/nvidia/spark/rapids/ToPrettyStringSuite.scala b/tests/src/test/spark350/scala/com/nvidia/spark/rapids/ToPrettyStringSuite.scala new file mode 100644 index 00000000000..b13ebaa8297 --- /dev/null +++ b/tests/src/test/spark350/scala/com/nvidia/spark/rapids/ToPrettyStringSuite.scala @@ -0,0 +1,116 @@ +/* + * Copyright (c) 2023, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/*** spark-rapids-shim-json-lines +{"spark": "350"} +spark-rapids-shim-json-lines ***/ +package com.nvidia.spark.rapids + +import ai.rapids.cudf.ColumnVector +import com.nvidia.spark.rapids.Arm._ +import com.nvidia.spark.rapids.GpuColumnVector.GpuColumnarBatchBuilder +import com.nvidia.spark.rapids.shims.GpuToPrettyString +import org.scalatest.exceptions.TestFailedException + +import org.apache.spark.sql.catalyst.expressions.{BoundReference, NamedExpression, ToPrettyString} +import org.apache.spark.sql.types.{ArrayType, DataType, DataTypes, DecimalType, MapType, StructField, StructType} +import org.apache.spark.unsafe.types.UTF8String + +class ToPrettyStringSuite extends GpuUnitTests { + + private def testDataType(dataType: DataType): Unit = { + val schema = (new StructType) + .add(StructField("a", dataType, true)) + val numRows = 100 + val inputRows = GpuBatchUtilsSuite.createRows(schema, numRows) + val cpuOutput: Array[String] = inputRows.map { + input => + ToPrettyString(BoundReference(0, dataType, true), Some("UTC")) + .eval(input).asInstanceOf[UTF8String].toString() + } + val child = GpuBoundReference(0, dataType, true)(NamedExpression.newExprId, "arg") + val gpuToPrettyStr = GpuToPrettyString(child, Some("UTC")) + + withResource(new GpuColumnarBatchBuilder(schema, numRows)) { batchBuilder => + val r2cConverter = new GpuRowToColumnConverter(schema) + inputRows.foreach(r2cConverter.convert(_, batchBuilder)) + withResource(batchBuilder.build(numRows)) { columnarBatch => + withResource(GpuColumnVector.from(ColumnVector.fromStrings(cpuOutput: _*), + DataTypes.StringType)) { expected => + checkEvaluation(gpuToPrettyStr, expected, columnarBatch) + } + } + } + } + + test("test show() on booleans") { + testDataType(DataTypes.BooleanType) + } + + test("test show() on bytes") { + testDataType(DataTypes.ByteType) + } + + test("test show() on shorts") { + testDataType(DataTypes.ShortType) + } + + test("test show() on ints") { + testDataType(DataTypes.IntegerType) + } + + test("test show() on longs") { + testDataType(DataTypes.LongType) + } + + test("test show() on floats") { + // This test is expected to fail until https://github.com/NVIDIA/spark-rapids/issues/4204 + // is resolved + assertThrows[TestFailedException](testDataType(DataTypes.FloatType)) + } + + test("test show() on doubles") { + // This test is expected to fail until https://github.com/NVIDIA/spark-rapids/issues/4204 + // is resolved + assertThrows[TestFailedException](testDataType(DataTypes.DoubleType)) + } + + test("test show() on strings") { + testDataType(DataTypes.StringType) + } + + test("test show() on decimals") { + testDataType(DecimalType(8,2)) + } + + test("test show() on binary") { + testDataType(DataTypes.BinaryType) + } + + test("test show() on array") { + testDataType(ArrayType(DataTypes.IntegerType)) + } + + test("test show() on map") { + testDataType(MapType(DataTypes.IntegerType, DataTypes.IntegerType)) + } + + test("test show() on struct") { + testDataType(StructType(Seq(StructField("a", DataTypes.IntegerType), + StructField("b", DataTypes.IntegerType), + StructField("c", DataTypes.IntegerType)))) + } +} From 6d8785a771fa0c4365186864c3708d6dc6f71ba6 Mon Sep 17 00:00:00 2001 From: Ferdinand Xu Date: Fri, 29 Sep 2023 05:09:31 +0800 Subject: [PATCH 38/46] Update authorized user in blossom-ci whitelist [skip ci] (#9318) Signed-off-by: Cheng Xu --- .github/workflows/blossom-ci.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/blossom-ci.yml b/.github/workflows/blossom-ci.yml index 83e102fca21..0320c686046 100644 --- a/.github/workflows/blossom-ci.yml +++ b/.github/workflows/blossom-ci.yml @@ -68,7 +68,7 @@ jobs: YanxuanLiu,\ cindyyuanjiang,\ thirtiseven,\ - winningsixnv,\ + winningsix,\ ', format('{0},', github.actor)) && github.event.comment.body == 'build' steps: - name: Check if comment is issued by authorized person From 1399d80889e71906e3bb71e3be9034ee13b3e003 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Thu, 28 Sep 2023 17:54:29 -0600 Subject: [PATCH 39/46] Temporarily skip failing tests (#9335) Signed-off-by: Andy Grove --- integration_tests/src/main/python/csv_test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/integration_tests/src/main/python/csv_test.py b/integration_tests/src/main/python/csv_test.py index 5e5234d7033..f07f1213cd1 100644 --- a/integration_tests/src/main/python/csv_test.py +++ b/integration_tests/src/main/python/csv_test.py @@ -561,6 +561,7 @@ def test_csv_read_count(spark_tmp_path): conf = {'spark.rapids.sql.explain': 'ALL'}) @allow_non_gpu('FileSourceScanExec', 'ProjectExec', 'CollectLimitExec', 'DeserializeToObjectExec') +@pytest.mark.skip(reason="https://github.com/NVIDIA/spark-rapids/issues/9325") @pytest.mark.skipif(is_before_spark_340(), reason='`TIMESTAMP_NTZ` is only supported in Spark 340+') @pytest.mark.parametrize('date_format', csv_supported_date_formats) @pytest.mark.parametrize('ts_part', csv_supported_ts_parts) @@ -569,6 +570,7 @@ def test_csv_infer_schema_timestamp_ntz_v1(spark_tmp_path, date_format, ts_part, csv_infer_schema_timestamp_ntz(spark_tmp_path, date_format, ts_part, timestamp_type, 'csv', 'FileSourceScanExec') @allow_non_gpu('BatchScanExec', 'FileSourceScanExec', 'ProjectExec', 'CollectLimitExec', 'DeserializeToObjectExec') +@pytest.mark.skip(reason="https://github.com/NVIDIA/spark-rapids/issues/9325") @pytest.mark.skipif(is_before_spark_340(), reason='`TIMESTAMP_NTZ` is only supported in Spark 340+') @pytest.mark.parametrize('date_format', csv_supported_date_formats) @pytest.mark.parametrize('ts_part', csv_supported_ts_parts) From 7bffb165559b2a7ce20ed1b94d3fd5f1ba06c997 Mon Sep 17 00:00:00 2001 From: Haoyang Li Date: Fri, 29 Sep 2023 09:35:53 +0800 Subject: [PATCH 40/46] Support `format_number` (#9281) * wip * wip * support format_number for integral and decimal type Signed-off-by: Haoyang Li * support double/float normal cases * support scientific notation double/float with positive exp * support scientific notation double/float with negative exp * bug fixed and clean up * refactor and memory leak fix * Handle resource pair as a whole * fix more memory leak * address some comments * add a config to control float/double enabling * fixed a bug in neg exp get parts * fixed another bug and add float scala test * add some comments and use lstrip to remove neg sign * fix memory leaks Signed-off-by: Haoyang Li * minor changes Signed-off-by: Haoyang Li * fallback decimal with high scale Signed-off-by: Haoyang Li * Update sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala Co-authored-by: Liangcai Li --------- Signed-off-by: Haoyang Li Co-authored-by: Liangcai Li --- .../advanced_configs.md | 2 + docs/compatibility.md | 4 + docs/supported_ops.md | 150 ++-- .../src/main/python/string_test.py | 54 ++ .../nvidia/spark/rapids/GpuOverrides.scala | 29 + .../com/nvidia/spark/rapids/RapidsConf.scala | 8 + .../spark/sql/rapids/stringFunctions.scala | 665 +++++++++++++++++- .../spark/rapids/StringFunctionSuite.scala | 29 + tools/generated_files/operatorsScore.csv | 1 + tools/generated_files/supportedExprs.csv | 3 + 10 files changed, 899 insertions(+), 46 deletions(-) diff --git a/docs/additional-functionality/advanced_configs.md b/docs/additional-functionality/advanced_configs.md index 1b2216c59e7..1b3ac623bbd 100644 --- a/docs/additional-functionality/advanced_configs.md +++ b/docs/additional-functionality/advanced_configs.md @@ -109,6 +109,7 @@ Name | Description | Default Value | Applicable at spark.rapids.sql.format.parquet.reader.type|Sets the Parquet reader type. We support different types that are optimized for different environments. The original Spark style reader can be selected by setting this to PERFILE which individually reads and copies files to the GPU. Loading many small files individually has high overhead, and using either COALESCING or MULTITHREADED is recommended instead. The COALESCING reader is good when using a local file system where the executors are on the same nodes or close to the nodes the data is being read on. This reader coalesces all the files assigned to a task into a single host buffer before sending it down to the GPU. It copies blocks from a single file into a host buffer in separate threads in parallel, see spark.rapids.sql.multiThreadedRead.numThreads. MULTITHREADED is good for cloud environments where you are reading from a blobstore that is totally separate and likely has a higher I/O read cost. Many times the cloud environments also get better throughput when you have multiple readers in parallel. This reader uses multiple threads to read each file in parallel and each file is sent to the GPU separately. This allows the CPU to keep reading while GPU is also doing work. See spark.rapids.sql.multiThreadedRead.numThreads and spark.rapids.sql.format.parquet.multiThreadedRead.maxNumFilesParallel to control the number of threads and amount of memory used. By default this is set to AUTO so we select the reader we think is best. This will either be the COALESCING or the MULTITHREADED based on whether we think the file is in the cloud. See spark.rapids.cloudSchemes.|AUTO|Runtime spark.rapids.sql.format.parquet.write.enabled|When set to false disables parquet output acceleration|true|Runtime spark.rapids.sql.format.parquet.writer.int96.enabled|When set to false, disables accelerated parquet write if the spark.sql.parquet.outputTimestampType is set to INT96|true|Runtime +spark.rapids.sql.formatNumberFloat.enabled|format_number with floating point types on the GPU returns results that have a different precision than the default results of Spark.|false|Runtime spark.rapids.sql.hasExtendedYearValues|Spark 3.2.0+ extended parsing of years in dates and timestamps to support the full range of possible values. Prior to this it was limited to a positive 4 digit year. The Accelerator does not support the extended range yet. This config indicates if your data includes this extended range or not, or if you don't care about getting the correct values on values with the extended range.|true|Runtime spark.rapids.sql.hashOptimizeSort.enabled|Whether sorts should be inserted after some hashed operations to improve output ordering. This can improve output file sizes when saving to columnar formats.|false|Runtime spark.rapids.sql.improvedFloatOps.enabled|For some floating point operations spark uses one way to compute the value and the underlying cudf implementation can use an improved algorithm. In some cases this can result in cudf producing an answer when spark overflows.|true|Runtime @@ -234,6 +235,7 @@ Name | SQL Function(s) | Description | Default Value | Notes spark.rapids.sql.expression.Expm1|`expm1`|Euler's number e raised to a power minus 1|true|None| spark.rapids.sql.expression.Flatten|`flatten`|Creates a single array from an array of arrays|true|None| spark.rapids.sql.expression.Floor|`floor`|Floor of a number|true|None| +spark.rapids.sql.expression.FormatNumber|`format_number`|Formats the number x like '#,###,###.##', rounded to d decimal places.|true|None| spark.rapids.sql.expression.FromUTCTimestamp|`from_utc_timestamp`|Render the input UTC timestamp in the input timezone|true|None| spark.rapids.sql.expression.FromUnixTime|`from_unixtime`|Get the string from a unix timestamp|true|None| spark.rapids.sql.expression.GetArrayItem| |Gets the field at `ordinal` in the Array|true|None| diff --git a/docs/compatibility.md b/docs/compatibility.md index e72415b634f..de4ee77496e 100644 --- a/docs/compatibility.md +++ b/docs/compatibility.md @@ -664,6 +664,10 @@ The GPU will use different precision than Java's toString method when converting types to strings. The GPU uses a lowercase `e` prefix for an exponent while Spark uses uppercase `E`. As a result the computed string can differ from the default behavior in Spark. +The `format_number` function will retain 10 digits of precision for the GPU when the input is a floating +point number, but Spark will retain up to 17 digits of precision, i.e. `format_number(1234567890.1234567890, 5)` +will return `1,234,567,890.00000` on the GPU and `1,234,567,890.12346` on the CPU. To enable this on the GPU, set [`spark.rapids.sql.formatNumberFloat.enabled`](additional-functionality/advanced_configs.md#sql.formatNumberFloat.enabled) to `true`. + Starting from 22.06 this conf is enabled by default, to disable this operation on the GPU, set [`spark.rapids.sql.castFloatToString.enabled`](additional-functionality/advanced_configs.md#sql.castFloatToString.enabled) to `false`. diff --git a/docs/supported_ops.md b/docs/supported_ops.md index ee22d2de6c3..4e2be930b49 100644 --- a/docs/supported_ops.md +++ b/docs/supported_ops.md @@ -6461,23 +6461,23 @@ are limited. -FromUTCTimestamp -`from_utc_timestamp` -Render the input UTC timestamp in the input timezone +FormatNumber +`format_number` +Formats the number x like '#,###,###.##', rounded to d decimal places. None project -timestamp - - - - - - +x +S +S +S +S +S +S -PS
UTC is only supported TZ for TIMESTAMP
+S @@ -6487,17 +6487,17 @@ are limited. -timezone - +d +PS
Literal value only
-PS
Only timezones equivalent to UTC are supported
+NS @@ -6517,8 +6517,8 @@ are limited. -PS
UTC is only supported TZ for TIMESTAMP
+S @@ -6555,6 +6555,74 @@ are limited. UDT +FromUTCTimestamp +`from_utc_timestamp` +Render the input UTC timestamp in the input timezone +None +project +timestamp + + + + + + + + +PS
UTC is only supported TZ for TIMESTAMP
+ + + + + + + + + + + +timezone + + + + + + + + + +PS
Only timezones equivalent to UTC are supported
+ + + + + + + + + + +result + + + + + + + + +PS
UTC is only supported TZ for TIMESTAMP
+ + + + + + + + + + + FromUnixTime `from_unixtime` Get the string from a unix timestamp @@ -6874,6 +6942,32 @@ are limited. NS +Expression +SQL Functions(s) +Description +Notes +Context +Param/Output +BOOLEAN +BYTE +SHORT +INT +LONG +FLOAT +DOUBLE +DATE +TIMESTAMP +STRING +DECIMAL +NULL +BINARY +CALENDAR +ARRAY +MAP +STRUCT +UDT + + GetStructField Gets the named field of the struct @@ -6921,32 +7015,6 @@ are limited. NS -Expression -SQL Functions(s) -Description -Notes -Context -Param/Output -BOOLEAN -BYTE -SHORT -INT -LONG -FLOAT -DOUBLE -DATE -TIMESTAMP -STRING -DECIMAL -NULL -BINARY -CALENDAR -ARRAY -MAP -STRUCT -UDT - - GetTimestamp Gets timestamps from strings using given pattern. diff --git a/integration_tests/src/main/python/string_test.py b/integration_tests/src/main/python/string_test.py index 87fc168928c..316a427db94 100644 --- a/integration_tests/src/main/python/string_test.py +++ b/integration_tests/src/main/python/string_test.py @@ -797,3 +797,57 @@ def test_conv_dec_to_from_hex(from_base, to_base, pattern): lambda spark: unary_op_df(spark, gen).select('a', f.conv(f.col('a'), from_base, to_base)), conf={'spark.rapids.sql.expression.Conv': True} ) + +format_number_gens = integral_gens + [DecimalGen(precision=7, scale=7), DecimalGen(precision=18, scale=0), + DecimalGen(precision=18, scale=3), DecimalGen(precision=36, scale=5), + DecimalGen(precision=36, scale=-5), DecimalGen(precision=38, scale=10), + DecimalGen(precision=38, scale=-10), + DecimalGen(precision=38, scale=30, special_cases=[Decimal('0.000125')]), + DecimalGen(precision=38, scale=32, special_cases=[Decimal('0.000125')])] + +@pytest.mark.parametrize('data_gen', format_number_gens, ids=idfn) +def test_format_number_supported(data_gen): + gen = data_gen + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, gen).selectExpr( + 'format_number(a, -2)', + 'format_number(a, 0)', + 'format_number(a, 1)', + 'format_number(a, 5)', + 'format_number(a, 10)', + 'format_number(a, 100)') + ) + +float_format_number_conf = {'spark.rapids.sql.formatNumberFloat.enabled': 'true'} +format_number_float_gens = [DoubleGen(min_exp=-300, max_exp=15)] + +@pytest.mark.parametrize('data_gen', format_number_float_gens, ids=idfn) +def test_format_number_float_limited(data_gen): + gen = data_gen + assert_gpu_and_cpu_are_equal_collect( + lambda spark: unary_op_df(spark, gen).selectExpr( + 'format_number(a, 5)'), + conf = float_format_number_conf + ) + +# format_number for float/double is disabled by default due to compatibility issue +# GPU will generate result with less precision than CPU +@allow_non_gpu('ProjectExec') +@pytest.mark.parametrize('data_gen', [float_gen, double_gen], ids=idfn) +def test_format_number_float_fallback(data_gen): + assert_gpu_fallback_collect( + lambda spark: unary_op_df(spark, data_gen).selectExpr( + 'format_number(a, 5)'), + 'FormatNumber' + ) + +# fallback due to https://github.com/NVIDIA/spark-rapids/issues/9309 +@allow_non_gpu('ProjectExec') +@pytest.mark.parametrize('data_gen', [float_gen, double_gen], ids=idfn) +def test_format_number_decimal_big_scale_fallback(data_gen): + data_gen = DecimalGen(precision=38, scale=37) + assert_gpu_fallback_collect( + lambda spark: unary_op_df(spark, data_gen).selectExpr( + 'format_number(a, 5)'), + 'FormatNumber' + ) diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala index a5bda29670f..bb8668cbb9b 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/GpuOverrides.scala @@ -3085,6 +3085,35 @@ object GpuOverrides extends Logging { |For instance decimal strings not longer than 18 characters / hexadecimal strings |not longer than 15 characters disregarding the sign cannot cause an overflow. """.stripMargin.replaceAll("\n", " ")), + expr[FormatNumber]( + "Formats the number x like '#,###,###.##', rounded to d decimal places.", + ExprChecks.binaryProject(TypeSig.STRING, TypeSig.STRING, + ("x", TypeSig.gpuNumeric, TypeSig.cpuNumeric), + ("d", TypeSig.lit(TypeEnum.INT), TypeSig.INT+TypeSig.STRING)), + (in, conf, p, r) => new BinaryExprMeta[FormatNumber](in, conf, p, r) { + override def tagExprForGpu(): Unit = { + in.children.head.dataType match { + case _: FloatType | DoubleType => { + if (!conf.isFloatFormatNumberEnabled) { + willNotWorkOnGpu("format_number with floating point types on the GPU returns " + + "results that have a different precision than the default results of Spark. " + + "To enable this operation on the GPU, set" + + s" ${RapidsConf.ENABLE_FLOAT_FORMAT_NUMBER} to true.") + } + } + case dt: DecimalType => { + if (dt.scale > 32) { + willNotWorkOnGpu("format_number will generate results mismatched from Spark " + + "when the scale is larger than 32.") + } + } + case _ => + } + } + override def convertToGpu(lhs: Expression, rhs: Expression): GpuExpression = + GpuFormatNumber(lhs, rhs) + } + ), expr[MapConcat]( "Returns the union of all the given maps", ExprChecks.projectOnly(TypeSig.MAP.nested(TypeSig.commonCudfTypes + TypeSig.DECIMAL_128 + diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index adddeecdc61..f3edc99a53f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -716,6 +716,12 @@ object RapidsConf { .booleanConf .createWithDefault(true) + val ENABLE_FLOAT_FORMAT_NUMBER = conf("spark.rapids.sql.formatNumberFloat.enabled") + .doc("format_number with floating point types on the GPU returns results that have " + + "a different precision than the default results of Spark.") + .booleanConf + .createWithDefault(false) + val ENABLE_CAST_FLOAT_TO_INTEGRAL_TYPES = conf("spark.rapids.sql.castFloatToIntegralTypes.enabled") .doc("Casting from floating point types to integral types on the GPU supports a " + @@ -2332,6 +2338,8 @@ class RapidsConf(conf: Map[String, String]) extends Logging { lazy val isCastFloatToStringEnabled: Boolean = get(ENABLE_CAST_FLOAT_TO_STRING) + lazy val isFloatFormatNumberEnabled: Boolean = get(ENABLE_FLOAT_FORMAT_NUMBER) + lazy val isCastStringToTimestampEnabled: Boolean = get(ENABLE_CAST_STRING_TO_TIMESTAMP) lazy val hasExtendedYearValues: Boolean = get(HAS_EXTENDED_YEAR_VALUES) diff --git a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala index e2c55b32647..febbf75ba58 100644 --- a/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala +++ b/sql-plugin/src/main/scala/org/apache/spark/sql/rapids/stringFunctions.scala @@ -17,11 +17,12 @@ package org.apache.spark.sql.rapids import java.nio.charset.Charset -import java.util.Optional +import java.text.DecimalFormatSymbols +import java.util.{Locale, Optional} import scala.collection.mutable.ArrayBuffer -import ai.rapids.cudf.{BinaryOp, BinaryOperable, CaptureGroups, ColumnVector, ColumnView, DType, PadSide, RegexProgram, Scalar, Table} +import ai.rapids.cudf.{BinaryOp, BinaryOperable, CaptureGroups, ColumnVector, ColumnView, DType, PadSide, RegexProgram, RoundMode, Scalar, Table} import com.nvidia.spark.rapids._ import com.nvidia.spark.rapids.Arm._ import com.nvidia.spark.rapids.RapidsPluginImplicits._ @@ -1989,8 +1990,6 @@ case class GpuStringInstr(str: Expression, substr: Expression) } } - - class GpuConvMeta( expr: Conv, conf: RapidsConf, @@ -2079,4 +2078,660 @@ case class GpuConv(num: Expression, fromBase: Expression, toBase: Expression) override def third: Expression = toBase override def dataType: DataType = StringType -} \ No newline at end of file +} + +case class GpuFormatNumber(x: Expression, d: Expression) + extends GpuBinaryExpression with ExpectsInputTypes with NullIntolerant { + + override def left: Expression = x + override def right: Expression = d + override def dataType: DataType = StringType + override def nullable: Boolean = true + override def inputTypes: Seq[AbstractDataType] = Seq(NumericType, IntegerType) + + private def removeNegSign(cv: ColumnVector): ColumnVector = { + withResource(Scalar.fromString("-")) { negativeSign => + cv.lstrip(negativeSign) + } + } + + private def getZeroCv(size: Int): ColumnVector = { + withResource(Scalar.fromString("0")) { zero => + ColumnVector.fromScalar(zero, size) + } + } + + private def handleDoublePosExp(cv: ColumnVector, intPart: ColumnVector, decPart: ColumnVector, + exp: ColumnVector, d: Int): (ColumnVector, ColumnVector) = { + // handle cases when exp is positive + // append "0" * zerosNum after end of strings, zerosNum = exp - decLen + d + val expSubDecLen = withResource(decPart.getCharLengths) { decLen => + exp.sub(decLen) + } + val zerosNum = withResource(expSubDecLen) { _ => + withResource(Scalar.fromInt(d)) { dScalar => + expSubDecLen.add(dScalar) + } + } + val zeroCv = withResource(Scalar.fromString("0")) { zero => + ColumnVector.fromScalar(zero, cv.getRowCount.toInt) + } + val zeros = withResource(zerosNum) { _ => + withResource(zeroCv) { _ => + zeroCv.repeatStrings(zerosNum) + } + } + + val intAndDecParts = withResource(zeros) { _ => + ColumnVector.stringConcatenate(Array(intPart, decPart, zeros)) + } + // split intAndDecParts to intPart and decPart with substrings, start = len(intAndDecParts) - d + closeOnExcept(ArrayBuffer.empty[ColumnVector]) { resourceArray => + val (intPartPosExp, decPartPosExpTemp) = withResource(intAndDecParts) { _ => + val (start, end) = withResource(intAndDecParts.getCharLengths) { partsLength => + (withResource(Scalar.fromInt(d)) { d => + partsLength.sub(d) + }, partsLength.incRefCount()) + } + withResource(start) { _ => + withResource(end) { _ => + val zeroIntCv = withResource(Scalar.fromInt(0)) { zero => + ColumnVector.fromScalar(zero, cv.getRowCount.toInt) + } + val intPart = withResource(zeroIntCv) { _ => + intAndDecParts.substring(zeroIntCv, start) + } + val decPart = closeOnExcept(intPart) { _ => + intAndDecParts.substring(start, end) + } + (intPart, decPart) + } + } + } + resourceArray += intPartPosExp + // if decLen - exp > d, convert to float/double, round, convert back to string + // decLen's max value is 9, abs(expPart)'s min value is 7, so it is possible only when d < 2 + // because d is small, we can use double to do the rounding + val decPartPosExp = if (0 < d && d < 2) { + val pointCv = closeOnExcept(decPartPosExpTemp) { _ => + withResource(Scalar.fromString(".")) { point => + ColumnVector.fromScalar(point, cv.getRowCount.toInt) + } + } + val withPoint = withResource(decPartPosExpTemp) { _ => + withResource(pointCv) { pointCv => + ColumnVector.stringConcatenate(Array(pointCv, decPartPosExpTemp)) + } + } + val decimalTypeRounding = DType.create(DType.DTypeEnum.DECIMAL128, -9) + val withPointDecimal = withResource(withPoint) { _ => + withResource(withPoint.castTo(decimalTypeRounding)) { decimal => + decimal.round(d, RoundMode.HALF_EVEN) + } + } + val roundedString = withResource(withPointDecimal) { _ => + withPointDecimal.castTo(DType.STRING) + } + withResource(roundedString) { _ => + withResource(roundedString.stringSplit(".", 2)) { splited => + splited.getColumn(1).incRefCount() + } + } + } else { + decPartPosExpTemp + } + (intPartPosExp, decPartPosExp) + } + } + + private def handleDoubleNegExp(cv: ColumnVector, intPart: ColumnVector, decPart: ColumnVector, + exp: ColumnVector, d: Int): (ColumnVector, ColumnVector) = { + // handle cases when exp is negative + // "0." + (- exp - 1) * "0" + intPart + decPart + // if -1 - d <= exp and decLen - exp > d, need to rounding + val cond1 = withResource(Scalar.fromInt(-1 - d)) { negOneSubD => + exp.greaterOrEqualTo(negOneSubD) + } + val cond2 = closeOnExcept(cond1) { _ => + val decLenSubExp = withResource(decPart.getCharLengths) { decLen => + decLen.sub(exp) + } + withResource(decLenSubExp) { _ => + withResource(Scalar.fromInt(d)) { d => + decLenSubExp.greaterThan(d) + } + } + } + val needRounding = withResource(cond1) { _ => + withResource(cond2) { _ => + cond1.and(cond2) + } + } + val anyNeedRounding = withResource(needRounding) { _ => + withResource(needRounding.any()) { any => + any.isValid && any.getBoolean + } + } + anyNeedRounding match { + case false => + // a shortcut when no need to rounding + // "0." + (- exp - 1) * "0" + intPart + decPart + withResource(getZeroCv(cv.getRowCount.toInt)) { zeroCv => + val expSubOne = withResource(Scalar.fromInt(-1)) { negOne => + negOne.sub(exp) + } + val addingZeros = withResource(expSubOne) { _ => + zeroCv.repeatStrings(expSubOne) + } + val decPartNegExp = withResource(addingZeros) { _ => + ColumnVector.stringConcatenate(Array(addingZeros, intPart, decPart)) + } + val decPartNegSubstr = withResource(decPartNegExp) { _ => + decPartNegExp.substring(0, d) + } + (zeroCv.incRefCount(), decPartNegSubstr) + } + case true => + // if -exp <= d + 1 && -exp + decLen + 1 > d, need to rounding + // dec will be round to (d + exp + 1) digits + val dExpOne = withResource(Scalar.fromInt(d + 1)) { dExpOne => + exp.add(dExpOne) + } + // To do a dataframe operation, add some zeros before + // (intPat + decPart) and round them to 10 + // zerosNumRounding = (10 - (d + exp + 1)) . max(0) + val tenSubDExpOne = withResource(dExpOne) { _ => + withResource(Scalar.fromInt(10)) { ten => + ten.sub(dExpOne) + } + } + val zerosNumRounding = withResource(tenSubDExpOne) { _ => + withResource(Scalar.fromInt(0)) { zero => + withResource(tenSubDExpOne.lessThan(zero)) { lessThanZero => + lessThanZero.ifElse(zero, tenSubDExpOne) + } + } + } + val leadingZeros = withResource(zerosNumRounding) { _ => + withResource(getZeroCv(cv.getRowCount.toInt)) { zeroCv => + zeroCv.repeatStrings(zerosNumRounding) + } + } + val numberToRoundStr = withResource(leadingZeros) { _ => + val zeroPointCv = withResource(Scalar.fromString("0.")) { point => + ColumnVector.fromScalar(point, cv.getRowCount.toInt) + } + withResource(zeroPointCv) { _ => + ColumnVector.stringConcatenate(Array(zeroPointCv, leadingZeros, intPart, decPart)) + } + } + // use a decimal type to round, set scale to -20 to keep all digits + val decimalTypeRounding = DType.create(DType.DTypeEnum.DECIMAL128, -20) + val numberToRound = withResource(numberToRoundStr) { _ => + numberToRoundStr.castTo(decimalTypeRounding) + } + // rounding 10 digits + val rounded = withResource(numberToRound) { _ => + numberToRound.round(10, RoundMode.HALF_EVEN) + } + val roundedStr = withResource(rounded) { _ => + rounded.castTo(DType.STRING) + } + // substr 2 to remove "0." + val roundedDecPart = withResource(roundedStr) { _ => + roundedStr.substring(2) + } + val decPartStriped = withResource(roundedDecPart) { _ => + withResource(Scalar.fromString("0")) { zero => + roundedDecPart.lstrip(zero) + } + } + val decPartNegExp = withResource(decPartStriped) { _ => + decPartStriped.pad(d, PadSide.LEFT, "0") + } + closeOnExcept(decPartNegExp) { _ => + (getZeroCv(cv.getRowCount.toInt), decPartNegExp) + } + } + } + + private def normalDoubleSplit(cv: ColumnVector, d: Int): (ColumnVector, ColumnVector) = { + val roundingScale = d.min(10) // cuDF will keep at most 9 digits after decimal point + val roundedStr = withResource(cv.round(roundingScale, RoundMode.HALF_EVEN)) { rounded => + rounded.castTo(DType.STRING) + } + val (intPart, decPart) = withResource(roundedStr) { _ => + withResource(roundedStr.stringSplit(".", 2)) { intAndDec => + (intAndDec.getColumn(0).incRefCount(), intAndDec.getColumn(1).incRefCount()) + } + } + val intPartNoNeg = closeOnExcept(decPart) { _ => + withResource(intPart) { _ => + removeNegSign(intPart) + } + } + val decPartPad = closeOnExcept(intPartNoNeg) { _ => + withResource(decPart) { _ => + decPart.pad(d, PadSide.RIGHT, "0") + } + } + // a workaround for cuDF float to string, e.g. 12.3 => "12.30000019" instead of "12.3" + val decPartSubstr = closeOnExcept(intPartNoNeg) { _ => + withResource(decPartPad) { _ => + decPartPad.substring(0, d) + } + } + (intPartNoNeg, decPartSubstr) + } + + private def expDoubleSplit(cv: ColumnVector, d: Int): (ColumnVector, ColumnVector) = { + // handle special case: 1.234567e+7 or 1.234567e-6 + // get three parts first: + val replaceDelimToE = withResource(Scalar.fromString("e")) { e => + withResource(Scalar.fromString(".")) { p => + cv.stringReplace(e, p) + } + } + // get three parts: 1.234567e+7 -> 1, 234567, +7 + val (intPartSign, decPart, expPart) = withResource(replaceDelimToE) { _ => + withResource(replaceDelimToE.stringSplit(".", 3)) { intDecExp => + (intDecExp.getColumn(0).incRefCount(), + intDecExp.getColumn(1).incRefCount(), + intDecExp.getColumn(2).incRefCount()) + } + } + // sign will be handled later, use string-based solution instead abs to avoid overfolw + val intPart = closeOnExcept(decPart) { _ => + closeOnExcept(expPart) { _ => + withResource(intPartSign) { _ => + removeNegSign(intPartSign) + } + } + } + val exp = closeOnExcept(decPart) { _ => + closeOnExcept(intPart) { _ => + withResource(expPart) { _ => + expPart.castTo(DType.INT32) + } + } + } + // handle positive and negative exp separately + val (intPartPosExp, decPartPosExp) = closeOnExcept(intPart) { _ => + closeOnExcept(decPart) { _ => + closeOnExcept(exp) { _ => + handleDoublePosExp(cv, intPart, decPart, exp, d) + } + } + } + withResource(ArrayBuffer.empty[ColumnVector]) { resourceArray => + val (intPartNegExp, decPartNegExp) = withResource(intPart) { _ => + withResource(decPart) { _ => + closeOnExcept(exp) { _ => + handleDoubleNegExp(cv, intPart, decPart, exp, d) + } + } + } + resourceArray += intPartNegExp + resourceArray += decPartNegExp + val expPos = withResource(exp) { _ => + withResource(Scalar.fromInt(0)) { zero => + exp.greaterOrEqualTo(zero) + } + } + // combine results + withResource(expPos) { _ => + val intPartExp = withResource(intPartPosExp) { _ => + expPos.ifElse(intPartPosExp, intPartNegExp) + } + val decPartExp = closeOnExcept(intPartExp) { _ => + withResource(decPartPosExp) { _ => + expPos.ifElse(decPartPosExp, decPartNegExp) + } + } + (intPartExp, decPartExp) + } + } + } + + private def getPartsFromDouble(cv: ColumnVector, d: Int): (ColumnVector, ColumnVector) = { + // handle normal case: 1234.567 + closeOnExcept(ArrayBuffer.empty[ColumnVector]) { resourceArray => + val (normalInt, normalDec) = normalDoubleSplit(cv, d) + resourceArray += normalInt + resourceArray += normalDec + // first check special case + val cvStr = withResource(cv.castTo(DType.STRING)) { cvStr => + cvStr.incRefCount() + } + val containsE = closeOnExcept(cvStr) { _ => + withResource(Scalar.fromString("e")) { e => + cvStr.stringContains(e) + } + } + withResource(containsE) { _ => + // if no special case, return normal case directly + val anyExp = closeOnExcept(cvStr) { _ => + withResource(containsE.any()) { any => + any.isValid && any.getBoolean + } + } + anyExp match { + case false => { + cvStr.safeClose() + (normalInt, normalDec) + } + case true => { + val noEReplaced = withResource(cvStr) { _ => + // replace normal case with 0e0 to avoid error + withResource(Scalar.fromString("0.0e0")) { default => + containsE.ifElse(cvStr, default) + } + } + // handle scientific notation case: + val (expInt, expDec) = withResource(noEReplaced) { _ => + expDoubleSplit(noEReplaced, d) + } + // combine results + // remove normalInt from resourceArray + resourceArray.remove(0) + val intPart = closeOnExcept(expDec) { _ => + withResource(expInt) { _ => + withResource(normalInt) { _ => + containsE.ifElse(expInt, normalInt) + } + } + } + resourceArray.clear() + resourceArray += intPart + val decPart = withResource(expDec) { _ => + withResource(normalDec) { _ => + containsE.ifElse(expDec, normalDec) + } + } + (intPart, decPart) + } + } + } + } + } + + private def getPartsFromDecimal(cv: ColumnVector, d: Int, scale: Int): + (ColumnVector, ColumnVector) = { + // prevent d too large to fit in decimalType + val roundingScale = scale.min(d) + // append zeros to the end of decPart, zerosNum = d - scale + // if d <= scale, no need to append zeros, if scale < 0, append d zeros + val appendZeroNum = (d - scale).max(0).min(d) + val (intPart, decTemp) = if (roundingScale <= 0) { + withResource(ArrayBuffer.empty[ColumnVector]) { resourceArray => + val intPart = withResource(cv.round(roundingScale, RoundMode.HALF_EVEN)) { rounded => + rounded.castTo(DType.STRING) + } + resourceArray += intPart + // if intString starts with 0, it must be "00000...", replace it with "0" + val (isZero, zeroCv) = withResource(Scalar.fromString("0")) { zero => + withResource(intPart.startsWith(zero)) { isZero => + (isZero.incRefCount(), ColumnVector.fromScalar(zero, cv.getRowCount.toInt)) + } + } + val intPartZeroHandled = withResource(isZero) { isZero => + withResource(zeroCv) { zeroCv => + isZero.ifElse(zeroCv, intPart) + } + } + resourceArray += intPartZeroHandled + // a temp decPart is empty before appending zeros + val decPart = withResource(Scalar.fromString("")) { emptyString => + ColumnVector.fromScalar(emptyString, cv.getRowCount.toInt) + } + resourceArray += decPart + (intPartZeroHandled.incRefCount(), decPart.incRefCount()) + } + } else { + withResource(cv.round(roundingScale, RoundMode.HALF_EVEN)) { rounded => + withResource(rounded.castTo(DType.STRING)) { roundedStr => + withResource(roundedStr.stringSplit(".", 2)) { intAndDec => + (intAndDec.getColumn(0).incRefCount(), intAndDec.getColumn(1).incRefCount()) + } + } + } + } + closeOnExcept(ArrayBuffer.empty[ColumnVector]) { resourceArray => + // remove negative sign from intPart, sign will be handled later + val intPartPos = closeOnExcept(decTemp) { _ => + withResource(intPart) { _ => + removeNegSign(intPart) + } + } + resourceArray += intPartPos + // append zeros + val appendZeros = "0" * appendZeroNum + val appendZerosCv = closeOnExcept(decTemp) { _ => + withResource(Scalar.fromString(appendZeros)) { zeroString => + ColumnVector.fromScalar(zeroString, cv.getRowCount.toInt) + } + } + val decPart = withResource(decTemp) { _ => + withResource(appendZerosCv) { _ => + ColumnVector.stringConcatenate(Array(decTemp, appendZerosCv)) + } + } + (intPartPos, decPart) + } + } + + private def getParts(cv: ColumnVector, d: Int): (ColumnVector, ColumnVector) = { + // get int part and dec part from a column vector, int part will be set to positive + x.dataType match { + case FloatType | DoubleType => { + getPartsFromDouble(cv, d) + } + case DecimalType.Fixed(_, scale) => { + getPartsFromDecimal(cv, d, scale) + } + case IntegerType | LongType | ShortType | ByteType => { + val intPartPos = withResource(cv.castTo(DType.STRING)) { intPart => + removeNegSign(intPart) + } + // dec part is all zeros + val dzeros = "0" * d + val decPart = closeOnExcept(intPartPos) { _ => + withResource(Scalar.fromString(dzeros)) { zeroString => + ColumnVector.fromScalar(zeroString, cv.getRowCount.toInt) + } + } + (intPartPos, decPart) + } + case _ => { + throw new UnsupportedOperationException(s"format_number doesn't support type ${x.dataType}") + } + } + } + + private def negativeCheck(cv: ColumnVector): ColumnVector = { + withResource(cv.castTo(DType.STRING)) { cvStr => + withResource(Scalar.fromString("-")) { negativeSign => + cvStr.startsWith(negativeSign) + } + } + } + + private def removeExtraCommas(str: ColumnVector): ColumnVector = { + withResource(Scalar.fromString(",")) { comma => + str.rstrip(comma) + } + } + + private def addCommas(str: ColumnVector): ColumnVector = { + val maxstrlen = withResource(str.getCharLengths()) { strlen => + withResource(strlen.max()) { maxlen => + maxlen.isValid match { + case true => maxlen.getInt + case false => 0 + } + } + } + val sepCol = withResource(Scalar.fromString(",")) { sep => + ColumnVector.fromScalar(sep, str.getRowCount.toInt) + } + val substrs = closeOnExcept(sepCol) { _ => + (0 until maxstrlen by 3).safeMap { i => + str.substring(i, i + 3).asInstanceOf[ColumnView] + }.toArray + } + withResource(substrs) { _ => + withResource(sepCol) { _ => + withResource(ColumnVector.stringConcatenate(substrs, sepCol)) { res => + removeExtraCommas(res) + } + } + } + } + + private def handleInfAndNan(cv: ColumnVector, res: ColumnVector): ColumnVector = { + // replace inf and nan with infSymbol and nanSymbol in res according to cv + val symbols = DecimalFormatSymbols.getInstance(Locale.US) + val nanSymbol = symbols.getNaN + val infSymbol = symbols.getInfinity + val negInfSymbol = "-" + infSymbol + val handleNan = withResource(cv.isNan()) { isNan => + withResource(Scalar.fromString(nanSymbol)) { nan => + isNan.ifElse(nan, res) + } + } + val isInf = closeOnExcept(handleNan) { _ => + x.dataType match { + case DoubleType => { + withResource(Scalar.fromDouble(Double.PositiveInfinity)) { inf => + cv.equalTo(inf) + } + } + case FloatType => { + withResource(Scalar.fromFloat(Float.PositiveInfinity)) { inf => + cv.equalTo(inf) + } + } + } + } + val handleInf = withResource(isInf) { _ => + withResource(handleNan) { _ => + withResource(Scalar.fromString(infSymbol)) { inf => + isInf.ifElse(inf, handleNan) + } + } + } + val isNegInf = closeOnExcept(handleInf) { _ => + x.dataType match { + case DoubleType => { + withResource(Scalar.fromDouble(Double.NegativeInfinity)) { negInf => + cv.equalTo(negInf) + } + } + case FloatType => { + withResource(Scalar.fromFloat(Float.NegativeInfinity)) { negInf => + cv.equalTo(negInf) + } + } + } + } + val handleNegInf = withResource(isNegInf) { _ => + withResource(Scalar.fromString(negInfSymbol)) { negInf => + withResource(handleInf) { _ => + isNegInf.ifElse(negInf, handleInf) + } + } + } + handleNegInf + } + + override def doColumnar(lhs: GpuColumnVector, rhs: GpuScalar): ColumnVector = { + // get int d from rhs + if (!rhs.isValid || rhs.getValue.asInstanceOf[Int] < 0) { + return GpuColumnVector.columnVectorFromNull(lhs.getRowCount.toInt, StringType) + } + val d = rhs.getValue.asInstanceOf[Int] + val (integerPart, decimalPart) = getParts(lhs.getBase, d) + // reverse integer part for adding commas + val resWithDecimalPart = withResource(decimalPart) { _ => + val reversedIntegerPart = withResource(integerPart) { intPart => + intPart.reverseStringsOrLists() + } + val reversedIntegerPartWithCommas = withResource(reversedIntegerPart) { _ => + addCommas(reversedIntegerPart) + } + // reverse result back + val reverseBack = withResource(reversedIntegerPartWithCommas) { r => + r.reverseStringsOrLists() + } + d match { + case 0 => { + // d == 0, only return integer part + reverseBack + } + case _ => { + // d > 0, append decimal part to result + withResource(reverseBack) { _ => + withResource(Scalar.fromString(".")) { point => + withResource(Scalar.fromString("")) { empty => + ColumnVector.stringConcatenate(point, empty, Array(reverseBack, decimalPart)) + } + } + } + } + } + } + // add negative sign back + val negCv = withResource(Scalar.fromString("-")) { negativeSign => + ColumnVector.fromScalar(negativeSign, lhs.getRowCount.toInt) + } + val formated = withResource(resWithDecimalPart) { _ => + val resWithNeg = withResource(negCv) { _ => + ColumnVector.stringConcatenate(Array(negCv, resWithDecimalPart)) + } + withResource(negativeCheck(lhs.getBase)) { isNegative => + withResource(resWithNeg) { _ => + isNegative.ifElse(resWithNeg, resWithDecimalPart) + } + } + } + // handle null case + val anyNull = closeOnExcept(formated) { _ => + lhs.getBase.getNullCount > 0 + } + val formatedWithNull = anyNull match { + case true => { + withResource(formated) { _ => + withResource(lhs.getBase.isNull) { isNull => + withResource(Scalar.fromNull(DType.STRING)) { nullScalar => + isNull.ifElse(nullScalar, formated) + } + } + } + } + case false => formated + } + // handle inf and nan + x.dataType match { + case FloatType | DoubleType => { + withResource(formatedWithNull) { _ => + handleInfAndNan(lhs.getBase, formatedWithNull) + } + } + case _ => formatedWithNull + } + } + + override def doColumnar(lhs: GpuScalar, rhs: GpuColumnVector): ColumnVector = { + throw new UnsupportedOperationException() + } + + override def doColumnar(lhs: GpuColumnVector, rhs: GpuColumnVector): ColumnVector = { + throw new UnsupportedOperationException() + } + + override def doColumnar(numRows: Int, lhs: GpuScalar, rhs: GpuScalar): ColumnVector = { + withResource(GpuColumnVector.from(lhs, numRows, dataType)) { col => + doColumnar(col, rhs) + } + } +} diff --git a/tests/src/test/scala/com/nvidia/spark/rapids/StringFunctionSuite.scala b/tests/src/test/scala/com/nvidia/spark/rapids/StringFunctionSuite.scala index 3c3933946c5..4f64839acfd 100644 --- a/tests/src/test/scala/com/nvidia/spark/rapids/StringFunctionSuite.scala +++ b/tests/src/test/scala/com/nvidia/spark/rapids/StringFunctionSuite.scala @@ -220,6 +220,35 @@ class RegExpUtilsSuite extends AnyFunSuite { } } +class FormatNumberSuite extends SparkQueryCompareTestSuite { + def testFormatNumberDf(session: SparkSession): DataFrame = { + import session.sqlContext.implicits._ + Seq[java.lang.Float]( + -0.0f, + 0.0f, + Float.PositiveInfinity, + Float.NegativeInfinity, + Float.NaN, + 1.0f, + 1.2345f, + 123456789.0f, + 123456789.123456789f, + 0.00123456789f, + 0.0000000123456789f, + 1.0000000123456789f + ).toDF("doubles") + } + + testSparkResultsAreEqual("Test format_number float", + testFormatNumberDf, + conf = new SparkConf().set("spark.rapids.sql.formatNumberFloat.enabled", "true")) { + frame => frame.selectExpr("format_number(doubles, -1)", + "format_number(doubles, 0)", + "format_number(doubles, 1)", + "format_number(doubles, 5)") + } +} + /* * This isn't actually a test. It's just useful to help visualize what's going on when there are * differences present. diff --git a/tools/generated_files/operatorsScore.csv b/tools/generated_files/operatorsScore.csv index 9582404b8e0..0c583853490 100644 --- a/tools/generated_files/operatorsScore.csv +++ b/tools/generated_files/operatorsScore.csv @@ -110,6 +110,7 @@ Expm1,4 First,4 Flatten,4 Floor,4 +FormatNumber,4 FromUTCTimestamp,4 FromUnixTime,4 GetArrayItem,4 diff --git a/tools/generated_files/supportedExprs.csv b/tools/generated_files/supportedExprs.csv index f5a99f7197e..5b16293d07a 100644 --- a/tools/generated_files/supportedExprs.csv +++ b/tools/generated_files/supportedExprs.csv @@ -208,6 +208,9 @@ Flatten,S,`flatten`,None,project,input,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA Flatten,S,`flatten`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA Floor,S,`floor`,None,project,input,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA Floor,S,`floor`,None,project,result,NA,NA,NA,NA,S,NA,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA +FormatNumber,S,`format_number`,None,project,x,NA,S,S,S,S,S,S,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA +FormatNumber,S,`format_number`,None,project,d,NA,NA,NA,PS,NA,NA,NA,NA,NA,NS,NA,NA,NA,NA,NA,NA,NA,NA +FormatNumber,S,`format_number`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,NA,S,NA,NA,NA,NA,NA,NA,NA,NA FromUTCTimestamp,S,`from_utc_timestamp`,None,project,timestamp,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA FromUTCTimestamp,S,`from_utc_timestamp`,None,project,timezone,NA,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA FromUTCTimestamp,S,`from_utc_timestamp`,None,project,result,NA,NA,NA,NA,NA,NA,NA,NA,PS,NA,NA,NA,NA,NA,NA,NA,NA,NA From d22848a846c5a9e73e2550052a8b29d430fe2629 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Fri, 29 Sep 2023 10:05:03 -0500 Subject: [PATCH 41/46] Add kuhushukla to blossom ci yml (#9329) Signed-off-by: Kuhu Shukla --- .github/workflows/blossom-ci.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/blossom-ci.yml b/.github/workflows/blossom-ci.yml index 0320c686046..7c22332eb9d 100644 --- a/.github/workflows/blossom-ci.yml +++ b/.github/workflows/blossom-ci.yml @@ -42,6 +42,7 @@ jobs: jbrennan333, \ jlowe,\ krajendrannv,\ + kuhushukla,\ mythrocks,\ nartal1,\ nvdbaranec,\ From 199b35ce81ffc3f44bb4e672394cb11dba64c359 Mon Sep 17 00:00:00 2001 From: Kuhu Shukla Date: Fri, 29 Sep 2023 10:12:25 -0500 Subject: [PATCH 42/46] Building different Cuda versions section profile does not take effect [skip ci] (#9328) * Building different Cuda versions section profile does not take effect Signed-off-by: Kuhu Shukla * Address review comments Signed-off-by: Kuhu Shukla --------- Signed-off-by: Kuhu Shukla --- CONTRIBUTING.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 5d6c0933f1f..b4d7cc040cf 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -144,8 +144,9 @@ specifying the environment variable `BUILD_PARALLEL=`. ### Building against different CUDA Toolkit versions -You can build against different versions of the CUDA Toolkit by using one of the following profiles: -* `-Pcuda11` (CUDA 11.0/11.1/11.2, default) +You can build against different versions of the CUDA Toolkit by modifying the variable `cuda.version`: +* `-Dcuda.version=cuda11` (CUDA 11.x, default) +* `-Dcuda.version=cuda12` (CUDA 12.x) ### Building a Distribution for a Single Spark Release From 6ff22b365b47a8a0917ce58af67c8589015abf1a Mon Sep 17 00:00:00 2001 From: Hao Zhu <9665750+viadea@users.noreply.github.com> Date: Fri, 29 Sep 2023 20:58:16 -0700 Subject: [PATCH 43/46] Add Hao to blossom-ci whitelist (#9330) Signed-off-by: Hao Zhu <9665750+viadea@users.noreply.github.com> --- .github/workflows/blossom-ci.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/blossom-ci.yml b/.github/workflows/blossom-ci.yml index 7c22332eb9d..c80a4e7c3aa 100644 --- a/.github/workflows/blossom-ci.yml +++ b/.github/workflows/blossom-ci.yml @@ -70,6 +70,7 @@ jobs: cindyyuanjiang,\ thirtiseven,\ winningsix,\ + viadea,\ ', format('{0},', github.actor)) && github.event.comment.body == 'build' steps: - name: Check if comment is issued by authorized person From 15511b6ea90e2dacd438389d4bbf6658f6f3607b Mon Sep 17 00:00:00 2001 From: Alessandro Bellina Date: Mon, 2 Oct 2023 08:51:01 -0500 Subject: [PATCH 44/46] Update MULTITHREADED shuffle maxBytesInFlight default to 128MB (#9344) * Update MULTITHREADED shuffle maxBytesInFlight default to 128MB Signed-off-by: Alessandro Bellina * Add context around maxBytesInFlight in our doc --------- Signed-off-by: Alessandro Bellina --- docs/additional-functionality/advanced_configs.md | 2 +- docs/additional-functionality/rapids-shuffle.md | 7 +++++++ .../com/nvidia/spark/rapids/RapidsConf.scala | 15 +++++++++------ 3 files changed, 17 insertions(+), 7 deletions(-) diff --git a/docs/additional-functionality/advanced_configs.md b/docs/additional-functionality/advanced_configs.md index 1b3ac623bbd..f8e31040309 100644 --- a/docs/additional-functionality/advanced_configs.md +++ b/docs/additional-functionality/advanced_configs.md @@ -48,7 +48,7 @@ Name | Description | Default Value | Applicable at spark.rapids.python.memory.gpu.pooling.enabled|Should RMM in Python workers act as a pooling allocator for GPU memory, or should it just pass through to CUDA memory allocation directly. When not specified, It will honor the value of config 'spark.rapids.memory.gpu.pooling.enabled'|None|Runtime spark.rapids.shuffle.enabled|Enable or disable the RAPIDS Shuffle Manager at runtime. The [RAPIDS Shuffle Manager](rapids-shuffle.md) must already be configured. When set to `false`, the built-in Spark shuffle will be used. |true|Runtime spark.rapids.shuffle.mode|RAPIDS Shuffle Manager mode. "MULTITHREADED": shuffle file writes and reads are parallelized using a thread pool. "UCX": (requires UCX installation) uses accelerated transports for transferring shuffle blocks. "CACHE_ONLY": use when running a single executor, for short-circuit cached shuffle (for testing purposes).|MULTITHREADED|Startup -spark.rapids.shuffle.multiThreaded.maxBytesInFlight|The size limit, in bytes, that the RAPIDS shuffle manager configured in "MULTITHREADED" mode will allow to be deserialized concurrently per task. This is also the maximum amount of memory that will be used per task. This should ideally be at least the same size as the batch size so we don't have to wait to process a single batch.|2147483647|Startup +spark.rapids.shuffle.multiThreaded.maxBytesInFlight|The size limit, in bytes, that the RAPIDS shuffle manager configured in "MULTITHREADED" mode will allow to be deserialized concurrently per task. This is also the maximum amount of memory that will be used per task. This should be set larger than Spark's default maxBytesInFlight (48MB). The larger this setting is, the more compressed shuffle chunks are processed concurrently. In practice, care needs to be taken to not go over the amount of off-heap memory that Netty has available. See https://github.com/NVIDIA/spark-rapids/issues/9153.|134217728|Startup spark.rapids.shuffle.multiThreaded.reader.threads|The number of threads to use for reading shuffle blocks per executor in the RAPIDS shuffle manager configured in "MULTITHREADED" mode. There are two special values: 0 = feature is disabled, falls back to Spark built-in shuffle reader; 1 = our implementation of Spark's built-in shuffle reader with extra metrics.|20|Startup spark.rapids.shuffle.multiThreaded.writer.threads|The number of threads to use for writing shuffle blocks per executor in the RAPIDS shuffle manager configured in "MULTITHREADED" mode. There are two special values: 0 = feature is disabled, falls back to Spark built-in shuffle writer; 1 = our implementation of Spark's built-in shuffle writer with extra metrics.|20|Startup spark.rapids.shuffle.transport.earlyStart|Enable early connection establishment for RAPIDS Shuffle|true|Startup diff --git a/docs/additional-functionality/rapids-shuffle.md b/docs/additional-functionality/rapids-shuffle.md index 7c2ef5a3020..8e1e8731ce0 100644 --- a/docs/additional-functionality/rapids-shuffle.md +++ b/docs/additional-functionality/rapids-shuffle.md @@ -57,6 +57,13 @@ configuration can be independently changed for writers and readers using: `spark.rapids.shuffle.multiThreaded.[writer|reader].threads`. An appropriate value for these pools is the number of cores in the system divided by the number of executors per machine. +On the reader side, when blocks are received from the network, they are queued onto these threads +for decompression and decode. The amount of bytes we allow in flight per Spark task is +controlled by: `spark.rapids.shuffle.multiThreaded.maxBytesInFlight`, and it is set to +128MB-per-task as a default. Note that this memory comes from the Netty off-heap pool, and this +is sized at startup automatically by Netty, but this limit can be controlled by setting +`-Dio.netty.maxDirectMemory=[amount in Bytes]` under `spark.executor.extraJavaOptions`. + ## UCX Mode --- diff --git a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala index f3edc99a53f..c502a5ae36f 100644 --- a/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala +++ b/sql-plugin/src/main/scala/com/nvidia/spark/rapids/RapidsConf.scala @@ -1611,14 +1611,17 @@ object RapidsConf { val SHUFFLE_MULTITHREADED_MAX_BYTES_IN_FLIGHT = conf("spark.rapids.shuffle.multiThreaded.maxBytesInFlight") - .doc("The size limit, in bytes, that the RAPIDS shuffle manager configured in " + - "\"MULTITHREADED\" mode will allow to be deserialized concurrently per task. This is " + - "also the maximum amount of memory that will be used per task. This should ideally be " + - "at least the same size as the batch size so we don't have to wait to process a " + - "single batch.") + .doc( + "The size limit, in bytes, that the RAPIDS shuffle manager configured in " + + "\"MULTITHREADED\" mode will allow to be deserialized concurrently per task. This is " + + "also the maximum amount of memory that will be used per task. This should be set larger " + + "than Spark's default maxBytesInFlight (48MB). The larger this setting is, the " + + "more compressed shuffle chunks are processed concurrently. In practice, " + + "care needs to be taken to not go over the amount of off-heap memory that Netty has " + + "available. See https://github.com/NVIDIA/spark-rapids/issues/9153.") .startupOnly() .bytesConf(ByteUnit.BYTE) - .createWithDefault(Integer.MAX_VALUE) + .createWithDefault(128 * 1024 * 1024) val SHUFFLE_MULTITHREADED_WRITER_THREADS = conf("spark.rapids.shuffle.multiThreaded.writer.threads") From 3ce1e9e49450f390e1214e4ad889c281d6252cf1 Mon Sep 17 00:00:00 2001 From: Raza Jafri Date: Mon, 2 Oct 2023 07:39:26 -0700 Subject: [PATCH 45/46] Add NaNs to Data Generators In Floating-Point Testing [databricks] (#9334) * part_and_order_gens with nans * running_part_and_oder_gens * lead_lag_data_gens * removed no_nans gens * renamed generator from with_nans_with_decimal* to with_nan_and_decimal* * added missing gens to init_list * addressed review comments * Signing off Signed-off-by: Raza Jafri --------- Signed-off-by: Raza Jafri --- .../src/main/python/array_test.py | 1 + integration_tests/src/main/python/data_gen.py | 4 - .../src/main/python/hash_aggregate_test.py | 85 +++++-------------- .../src/main/python/hashing_test.py | 2 +- .../src/main/python/window_function_test.py | 6 +- 5 files changed, 26 insertions(+), 72 deletions(-) diff --git a/integration_tests/src/main/python/array_test.py b/integration_tests/src/main/python/array_test.py index 6cbe4382338..99b68ccfba1 100644 --- a/integration_tests/src/main/python/array_test.py +++ b/integration_tests/src/main/python/array_test.py @@ -82,6 +82,7 @@ DecimalGen(precision=12, scale=2, nullable=False), DecimalGen(precision=20, scale=2, nullable=False)] +# This non-nans version is only used for Spark version < 3.1.3 no_neg_zero_all_basic_gens_no_nans = [byte_gen, short_gen, int_gen, long_gen, # -0.0 cannot work because of -0.0 == 0.0 in cudf for distinct FloatGen(special_cases=[], no_nans=True), diff --git a/integration_tests/src/main/python/data_gen.py b/integration_tests/src/main/python/data_gen.py index 9f549adfa46..2a9d7e5e6f0 100644 --- a/integration_tests/src/main/python/data_gen.py +++ b/integration_tests/src/main/python/data_gen.py @@ -1067,12 +1067,8 @@ def gen_scalars_for_sql(data_gen, count, seed=0, force_no_nulls=False): # all of the basic types in a single struct all_basic_struct_gen = StructGen([['child'+str(ind), sub_gen] for ind, sub_gen in enumerate(all_basic_gens)]) -all_basic_struct_gen_no_nan = StructGen([['child'+str(ind), sub_gen] for ind, sub_gen in enumerate(all_basic_gens_no_nan)]) - struct_array_gen = StructGen([['child'+str(ind), sub_gen] for ind, sub_gen in enumerate(single_level_array_gens)]) -struct_array_gen_no_nans = StructGen([['child'+str(ind), sub_gen] for ind, sub_gen in enumerate(single_level_array_gens_no_nan)]) - # Some struct gens, but not all because of nesting nonempty_struct_gens_sample = [all_basic_struct_gen, StructGen([['child0', byte_gen], ['child1', all_basic_struct_gen]]), diff --git a/integration_tests/src/main/python/hash_aggregate_test.py b/integration_tests/src/main/python/hash_aggregate_test.py index 288cf3ebc07..4f58278360c 100644 --- a/integration_tests/src/main/python/hash_aggregate_test.py +++ b/integration_tests/src/main/python/hash_aggregate_test.py @@ -167,25 +167,16 @@ _grpkey_floats_with_nulls_and_nans ] -# List of schemas with no NaNs -_init_list_no_nans = [ - _longs_with_nulls, - _longs_with_no_nulls, - _grpkey_longs_with_nulls, - _grpkey_dbls_with_nulls, - _grpkey_floats_with_nulls, - _grpkey_strings_with_nulls, - _grpkey_nulls, - _grpkey_strings_with_extra_nulls] - # List of schemas with NaNs included -_init_list_with_nans_and_no_nans = [ +_init_list = [ _longs_with_nulls, _longs_with_no_nulls, _grpkey_longs_with_nulls, _grpkey_dbls_with_nulls, _grpkey_floats_with_nulls, _grpkey_strings_with_nulls, + _grpkey_strings_with_extra_nulls, + _grpkey_nulls, _grpkey_floats_with_nulls_and_nans] # grouping decimals with nulls @@ -197,7 +188,7 @@ ('b', DecimalGen(nullable=False)), ('c', DecimalGen(nullable=False))] -_init_list_with_nans_and_no_nans_with_decimals = _init_list_with_nans_and_no_nans + [ +_init_list_with_decimals = _init_list + [ _decimals_with_nulls, _decimals_with_no_nulls] # Used to test ANSI-mode fallback @@ -303,15 +294,7 @@ def get_params(init_list, marked_params=[]): ('c', _decimal_gen_sum_38_neg10)] -_init_list_no_nans_with_decimal = _init_list_no_nans + [ - _grpkey_small_decimals] - -_init_list_no_nans_with_decimalbig = _init_list_no_nans + [ - _grpkey_small_decimals, _grpkey_big_decimals, _grpkey_short_mid_decimals, - _grpkey_short_big_decimals, _grpkey_short_very_big_decimals, - _grpkey_short_very_big_neg_scale_decimals] - -_init_list_with_nans_and_no_nans_with_decimalbig = _init_list_with_nans_and_no_nans + [ +_init_list_with_decimalbig = _init_list + [ _grpkey_small_decimals, _grpkey_big_decimals, _grpkey_short_mid_decimals, _grpkey_short_big_decimals, _grpkey_short_very_big_decimals, _grpkey_short_very_big_neg_scale_decimals] @@ -378,7 +361,7 @@ def test_computation_in_grpby_columns(): @approximate_float @ignore_order @incompat -@pytest.mark.parametrize('data_gen', _init_list_no_nans_with_decimalbig, ids=idfn) +@pytest.mark.parametrize('data_gen', _init_list_with_decimalbig, ids=idfn) @pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_hash_grpby_sum(data_gen, conf): assert_gpu_and_cpu_are_equal_collect( @@ -420,7 +403,7 @@ def test_hash_reduction_sum_full_decimal(data_gen, conf): @approximate_float @ignore_order @incompat -@pytest.mark.parametrize('data_gen', _init_list_with_nans_and_no_nans + [_grpkey_short_mid_decimals, +@pytest.mark.parametrize('data_gen', _init_list + [_grpkey_short_mid_decimals, _grpkey_short_big_decimals, _grpkey_short_very_big_decimals, _grpkey_short_sum_full_decimals], ids=idfn) @pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_hash_grpby_avg(data_gen, conf): @@ -451,7 +434,7 @@ def test_hash_avg_nulls_partial_only(data_gen): @approximate_float @ignore_order @incompat -@pytest.mark.parametrize('data_gen', _init_list_no_nans_with_decimalbig, ids=idfn) +@pytest.mark.parametrize('data_gen', _init_list_with_decimalbig, ids=idfn) def test_intersectAll(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : gen_df(spark, data_gen, length=100).intersectAll(gen_df(spark, data_gen, length=100))) @@ -459,7 +442,7 @@ def test_intersectAll(data_gen): @approximate_float @ignore_order @incompat -@pytest.mark.parametrize('data_gen', _init_list_no_nans_with_decimalbig, ids=idfn) +@pytest.mark.parametrize('data_gen', _init_list_with_decimalbig, ids=idfn) def test_exceptAll(data_gen): assert_gpu_and_cpu_are_equal_collect( lambda spark : gen_df(spark, data_gen, length=100).exceptAll(gen_df(spark, data_gen, length=100).filter('a != b'))) @@ -477,7 +460,7 @@ def test_exceptAll(data_gen): ('b', _pivot_gen_128bit), ('c', decimal_gen_128bit)] -_pivot_gens_with_decimals = _init_list_with_nans_and_no_nans + [ +_pivot_gens_with_decimals = _init_list + [ _grpkey_small_decimals, _pivot_big_decimals, _grpkey_short_mid_decimals, _pivot_short_big_decimals, _grpkey_short_very_big_decimals, _grpkey_short_very_big_neg_scale_decimals] @@ -497,20 +480,7 @@ def test_hash_grpby_pivot(data_gen, conf): @approximate_float @ignore_order(local=True) @incompat -@pytest.mark.parametrize('data_gen', _init_list_no_nans, ids=idfn) -@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) -def test_hash_grpby_pivot_without_nans(data_gen, conf): - assert_gpu_and_cpu_are_equal_collect( - lambda spark: gen_df(spark, data_gen, length=100) - .groupby('a') - .pivot('b') - .agg(f.sum('c')), - conf=conf) - -@approximate_float -@ignore_order(local=True) -@incompat -@pytest.mark.parametrize('data_gen', _init_list_with_nans_and_no_nans, ids=idfn) +@pytest.mark.parametrize('data_gen', _init_list, ids=idfn) @pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_hash_multiple_grpby_pivot(data_gen, conf): assert_gpu_and_cpu_are_equal_collect( @@ -523,22 +493,9 @@ def test_hash_multiple_grpby_pivot(data_gen, conf): @approximate_float @ignore_order(local=True) @incompat -@pytest.mark.parametrize('data_gen', _init_list_no_nans, ids=idfn) -@pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) -def test_hash_reduction_pivot_without_nans(data_gen, conf): - assert_gpu_and_cpu_are_equal_collect( - lambda spark: gen_df(spark, data_gen, length=100) - .groupby() - .pivot('b') - .agg(f.sum('c')), - conf=conf) - -@approximate_float -@ignore_order(local=True) -@incompat -@pytest.mark.parametrize('data_gen', _init_list_with_nans_and_no_nans, ids=idfn) +@pytest.mark.parametrize('data_gen', _init_list, ids=idfn) @pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) -def test_hash_reduction_pivot_with_nans(data_gen, conf): +def test_hash_reduction_pivot(data_gen, conf): assert_gpu_and_cpu_are_equal_collect( lambda spark: gen_df(spark, data_gen, length=100) .groupby() @@ -943,7 +900,7 @@ def test_hash_groupby_typed_imperative_agg_without_gpu_implementation_fallback() @approximate_float @ignore_order @incompat -@pytest.mark.parametrize('data_gen', _init_list_no_nans, ids=idfn) +@pytest.mark.parametrize('data_gen', _init_list, ids=idfn) @pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_hash_multiple_mode_query(data_gen, conf): print_params(data_gen) @@ -965,7 +922,7 @@ def test_hash_multiple_mode_query(data_gen, conf): @approximate_float @ignore_order @incompat -@pytest.mark.parametrize('data_gen', _init_list_no_nans, ids=idfn) +@pytest.mark.parametrize('data_gen', _init_list, ids=idfn) @pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_hash_multiple_mode_query_avg_distincts(data_gen, conf): @@ -978,7 +935,7 @@ def test_hash_multiple_mode_query_avg_distincts(data_gen, conf): @approximate_float @ignore_order @incompat -@pytest.mark.parametrize('data_gen', _init_list_no_nans, ids=idfn) +@pytest.mark.parametrize('data_gen', _init_list, ids=idfn) @pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_hash_query_multiple_distincts_with_non_distinct(data_gen, conf): local_conf = copy_and_update(conf, {'spark.sql.legacy.allowParameterlessCount': 'true'}) @@ -1001,7 +958,7 @@ def test_hash_query_multiple_distincts_with_non_distinct(data_gen, conf): @approximate_float @ignore_order @incompat -@pytest.mark.parametrize('data_gen', _init_list_no_nans, ids=idfn) +@pytest.mark.parametrize('data_gen', _init_list, ids=idfn) @pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_hash_query_max_with_multiple_distincts(data_gen, conf): local_conf = copy_and_update(conf, {'spark.sql.legacy.allowParameterlessCount': 'true'}) @@ -1015,7 +972,7 @@ def test_hash_query_max_with_multiple_distincts(data_gen, conf): conf=local_conf) @ignore_order -@pytest.mark.parametrize('data_gen', _init_list_no_nans, ids=idfn) +@pytest.mark.parametrize('data_gen', _init_list, ids=idfn) @pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_hash_count_with_filter(data_gen, conf): assert_gpu_and_cpu_are_equal_collect( @@ -1027,7 +984,7 @@ def test_hash_count_with_filter(data_gen, conf): @approximate_float @ignore_order @incompat -@pytest.mark.parametrize('data_gen', _init_list_no_nans + [_grpkey_short_mid_decimals, _grpkey_short_big_decimals], ids=idfn) +@pytest.mark.parametrize('data_gen', _init_list + [_grpkey_short_mid_decimals, _grpkey_short_big_decimals], ids=idfn) @pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_hash_multiple_filters(data_gen, conf): assert_gpu_and_cpu_are_equal_sql( @@ -1784,7 +1741,7 @@ def do_it(spark): @ignore_order(local=True) @approximate_float @incompat -@pytest.mark.parametrize('data_gen', _init_list_with_nans_and_no_nans_with_decimals, ids=idfn) +@pytest.mark.parametrize('data_gen', _init_list_with_decimals, ids=idfn) @pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) def test_std_variance(data_gen, conf): local_conf = copy_and_update(conf, { @@ -1850,7 +1807,7 @@ def test_std_variance_nulls(data_gen, conf, ansi_enabled): 'StddevPop', 'StddevSamp', 'VariancePop', 'VarianceSamp', 'SortArray', 'Alias', 'Literal', 'Count', 'AggregateExpression', 'ProjectExec') -@pytest.mark.parametrize('data_gen', _init_list_with_nans_and_no_nans, ids=idfn) +@pytest.mark.parametrize('data_gen', _init_list, ids=idfn) @pytest.mark.parametrize('conf', get_params(_confs, params_markers_for_confs), ids=idfn) @pytest.mark.parametrize('replace_mode', _replace_modes_non_distinct, ids=idfn) @pytest.mark.parametrize('aqe_enabled', ['false', 'true'], ids=idfn) diff --git a/integration_tests/src/main/python/hashing_test.py b/integration_tests/src/main/python/hashing_test.py index 107c3a4576e..6bd56da933d 100644 --- a/integration_tests/src/main/python/hashing_test.py +++ b/integration_tests/src/main/python/hashing_test.py @@ -39,7 +39,7 @@ _xxhash_fallback_gens = single_level_array_gens + nested_array_gens_sample + [ all_basic_struct_gen, - struct_array_gen_no_nans, + struct_array_gen, _struct_of_xxhash_gens] if is_before_spark_320(): _xxhash_fallback_gens += [float_gen, double_gen] diff --git a/integration_tests/src/main/python/window_function_test.py b/integration_tests/src/main/python/window_function_test.py index 5a5347c70bf..b4708b89668 100644 --- a/integration_tests/src/main/python/window_function_test.py +++ b/integration_tests/src/main/python/window_function_test.py @@ -134,15 +134,15 @@ ('a', IntegerGen()), ('b', LongGen(nullable=True))] -part_and_order_gens = [long_gen, DoubleGen(no_nans=True, special_cases=[]), +part_and_order_gens = [long_gen, DoubleGen(special_cases=[]), string_gen, boolean_gen, timestamp_gen, DecimalGen(precision=18, scale=1), DecimalGen(precision=38, scale=1)] -running_part_and_order_gens = [long_gen, DoubleGen(no_nans=True, special_cases=[]), +running_part_and_order_gens = [long_gen, DoubleGen(special_cases=[]), string_gen, byte_gen, timestamp_gen, DecimalGen(precision=18, scale=1), DecimalGen(precision=38, scale=1)] -lead_lag_data_gens = [long_gen, DoubleGen(no_nans=True, special_cases=[]), +lead_lag_data_gens = [long_gen, DoubleGen(special_cases=[]), boolean_gen, timestamp_gen, string_gen, DecimalGen(precision=18, scale=3), DecimalGen(precision=38, scale=4), StructGen(children=[ From dbc4479d52b62d164d68f1b5019c537fe996ff39 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Mon, 2 Oct 2023 09:34:37 -0600 Subject: [PATCH 46/46] Refine rules for skipping `test_csv_infer_schema_timestamp_ntz_*` tests (#9352) * Refine skipif for test_csv_infer_schema_timestamp_ntz_* tests Signed-off-by: Andy Grove * xfail instead of skip --------- Signed-off-by: Andy Grove --- integration_tests/src/main/python/csv_test.py | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/integration_tests/src/main/python/csv_test.py b/integration_tests/src/main/python/csv_test.py index f07f1213cd1..0a4d3123634 100644 --- a/integration_tests/src/main/python/csv_test.py +++ b/integration_tests/src/main/python/csv_test.py @@ -21,7 +21,7 @@ from data_gen import * from marks import * from pyspark.sql.types import * -from spark_session import with_cpu_session, is_before_spark_330, is_spark_340_or_later, is_before_spark_340 +from spark_session import with_cpu_session, is_before_spark_330, is_spark_350_or_later, is_before_spark_340 _acq_schema = StructType([ StructField('loan_id', LongType()), @@ -561,11 +561,12 @@ def test_csv_read_count(spark_tmp_path): conf = {'spark.rapids.sql.explain': 'ALL'}) @allow_non_gpu('FileSourceScanExec', 'ProjectExec', 'CollectLimitExec', 'DeserializeToObjectExec') -@pytest.mark.skip(reason="https://github.com/NVIDIA/spark-rapids/issues/9325") @pytest.mark.skipif(is_before_spark_340(), reason='`TIMESTAMP_NTZ` is only supported in Spark 340+') @pytest.mark.parametrize('date_format', csv_supported_date_formats) @pytest.mark.parametrize('ts_part', csv_supported_ts_parts) -@pytest.mark.parametrize("timestamp_type", ["TIMESTAMP_LTZ", "TIMESTAMP_NTZ"]) +@pytest.mark.parametrize("timestamp_type", [ + pytest.param('TIMESTAMP_LTZ', marks=pytest.mark.xfail(is_spark_350_or_later(), reason="https://github.com/NVIDIA/spark-rapids/issues/9325")), + "TIMESTAMP_NTZ"]) def test_csv_infer_schema_timestamp_ntz_v1(spark_tmp_path, date_format, ts_part, timestamp_type): csv_infer_schema_timestamp_ntz(spark_tmp_path, date_format, ts_part, timestamp_type, 'csv', 'FileSourceScanExec') @@ -574,7 +575,9 @@ def test_csv_infer_schema_timestamp_ntz_v1(spark_tmp_path, date_format, ts_part, @pytest.mark.skipif(is_before_spark_340(), reason='`TIMESTAMP_NTZ` is only supported in Spark 340+') @pytest.mark.parametrize('date_format', csv_supported_date_formats) @pytest.mark.parametrize('ts_part', csv_supported_ts_parts) -@pytest.mark.parametrize("timestamp_type", ["TIMESTAMP_LTZ", "TIMESTAMP_NTZ"]) +@pytest.mark.parametrize("timestamp_type", [ + pytest.param('TIMESTAMP_LTZ', marks=pytest.mark.xfail(is_spark_350_or_later(), reason="https://github.com/NVIDIA/spark-rapids/issues/9325")), + "TIMESTAMP_NTZ"]) def test_csv_infer_schema_timestamp_ntz_v2(spark_tmp_path, date_format, ts_part, timestamp_type): csv_infer_schema_timestamp_ntz(spark_tmp_path, date_format, ts_part, timestamp_type, '', 'BatchScanExec')