From ef0ffb3f43f9f6e96af49629aed2a6ce61a6a2ab Mon Sep 17 00:00:00 2001 From: weizhongjia Date: Tue, 23 Aug 2022 16:56:11 +0800 Subject: [PATCH] initial commit --- .github/ISSUE_TEMPLATE/bug-report.yaml | 41 + .github/ISSUE_TEMPLATE/config.yml | 1 + .github/ISSUE_TEMPLATE/enhancement.yaml | 16 + .github/PULL_REQUEST_TEMPLATE.md | 10 + .github/workflows/ut.yml | 30 + .gitignore | 93 ++ ACKNOWLEDGEMENTS.md | 10 + CODE_OF_CONDUCT.md | 127 ++ CONTRIBUTING.md | 19 + LICENSE | 253 ++++ README.md | 183 +++ README_zh.md | 185 +++ api/pom.xml | 69 ++ .../bytedance/css/api/CssShuffleContext.java | 112 ++ build.sh | 89 ++ client/pom.xml | 135 +++ .../css/client/MapperAttemptStatus.java | 56 + .../bytedance/css/client/MetricsCallback.java | 27 + .../bytedance/css/client/ShuffleClient.java | 189 +++ .../css/client/compress/Compressor.java | 29 + .../client/compress/CompressorFactory.java | 27 + .../client/compress/CssCompressorFactory.java | 82 ++ .../client/compress/CssCompressorTrait.java | 40 + .../css/client/compress/Decompressor.java | 27 + .../css/client/compress/Lz4Compressor.java | 104 ++ .../css/client/compress/Lz4Decompressor.java | 81 ++ .../css/client/compress/ZstdCompressor.java | 145 +++ .../css/client/compress/ZstdDecompressor.java | 105 ++ .../client/compress/ZstdDictCompressor.java | 125 ++ .../client/compress/ZstdDictDecompressor.java | 108 ++ .../css/client/compress/ZstdDictTrainer.java | 75 ++ .../css/client/impl/ShuffleClientImpl.java | 1056 +++++++++++++++++ .../css/client/metrics/BaseSource.java | 47 + .../css/client/metrics/ClientSource.java | 86 ++ .../css/client/stream/CssInputStream.java | 173 +++ .../css/client/stream/CssInputStreamImpl.java | 245 ++++ .../stream/EpochInputStreamBuilder.java | 32 + .../bytedance/css/client/stream/Frame.java | 109 ++ .../css/client/stream/FrameIterator.java | 103 ++ .../stream/disk/CssLocalDiskEpochReader.java | 69 ++ .../stream/disk/CssRemoteDiskEpochReader.java | 248 ++++ .../disk/EpochDiskInputStreamBuilder.java | 140 +++ .../hdfs/EpochHdfsInputStreamBuilder.java | 59 + .../css/client/compress/CompressorSuite.java | 185 +++ .../stream/CssDiskInputStreamSuite.java | 343 ++++++ .../CssEpochReaderRetryCorrectSuite.java | 327 +++++ .../stream/CssEpochReaderRetrySuite.java | 322 +++++ .../stream/CssHdfsInputStreamSuite.java | 243 ++++ client/src/test/resources/log4j.properties | 29 + client/src/test/resources/xml | Bin 0 -> 5345280 bytes common/pom.xml | 128 ++ .../css/common/ChildFirstURLClassLoader.java | 76 ++ .../css/common/ParentClassLoader.java | 41 + .../metrics/ByteDanceMetricsEmitter.java | 192 +++ .../metrics/ByteDanceMetricsReporter.java | 137 +++ .../protocol/CommittedPartitionInfo.java | 98 ++ .../css/common/protocol/CssStatusCode.java | 51 + .../protocol/FailedPartitionInfoBatch.java | 93 ++ .../css/common/protocol/PartitionGroup.java | 67 ++ .../protocol/PartitionGroupManager.java | 61 + .../css/common/protocol/PartitionInfo.java | 83 ++ .../css/common/protocol/ReplicaBase.java | 68 ++ .../css/common/protocol/RpcNameConstants.java | 41 + .../css/common/protocol/ShuffleMode.java | 37 + .../protocol/TransportModuleConstants.java | 34 + .../css/common/protocol/WorkerAddress.java | 59 + .../css/common/protocol/WorkerStatus.java | 107 ++ .../bytedance/css/common/unsafe/Platform.java | 259 ++++ .../bytedance/css/common/util/JsonUtils.java | 62 + .../com/bytedance/css/common/CssConf.scala | 829 +++++++++++++ .../exception/AlreadyClosedException.scala | 29 + .../css/common/exception/CssException.scala | 44 + .../EpochShouldRotateException.scala | 28 + .../css/common/internal/Logging.scala | 220 ++++ .../internal/config/ConfigBuilder.scala | 242 ++++ .../common/internal/config/ConfigEntry.scala | 183 +++ .../internal/config/ConfigProvider.scala | 66 ++ .../common/internal/config/ConfigReader.scala | 123 ++ .../css/common/internal/config/package.scala | 23 + .../css/common/metrics/MetricsConfig.scala | 155 +++ .../css/common/metrics/MetricsSystem.scala | 196 +++ .../common/metrics/sink/BytedanceSink.scala | 69 ++ .../css/common/metrics/sink/ConsoleSink.scala | 65 + .../css/common/metrics/sink/CsvSink.scala | 74 ++ .../common/metrics/sink/GraphiteSink.scala | 103 ++ .../css/common/metrics/sink/JmxSink.scala | 41 + .../css/common/metrics/sink/Sink.scala | 26 + .../css/common/metrics/sink/Slf4jSink.scala | 64 + .../common/metrics/sink/StatsdReporter.scala | 164 +++ .../css/common/metrics/sink/StatsdSink.scala | 73 ++ .../css/common/metrics/source/Source.scala | 27 + .../css/common/protocol/CssRpcMessage.scala | 119 ++ .../bytedance/css/common/rpc/RpcAddress.scala | 52 + .../css/common/rpc/RpcCallContext.scala | 43 + .../css/common/rpc/RpcEndpoint.scala | 150 +++ .../css/common/rpc/RpcEndpointAddress.scala | 76 ++ .../rpc/RpcEndpointNotFoundException.scala | 25 + .../css/common/rpc/RpcEndpointRef.scala | 96 ++ .../com/bytedance/css/common/rpc/RpcEnv.scala | 204 ++++ .../common/rpc/RpcEnvStoppedException.scala | 23 + .../bytedance/css/common/rpc/RpcTimeout.scala | 136 +++ .../css/common/rpc/netty/Dispatcher.scala | 244 ++++ .../css/common/rpc/netty/Inbox.scala | 219 ++++ .../rpc/netty/NettyRpcCallContext.scala | 75 ++ .../css/common/rpc/netty/NettyRpcEnv.scala | 724 +++++++++++ .../common/rpc/netty/NettyStreamManager.scala | 81 ++ .../css/common/rpc/netty/Outbox.scala | 277 +++++ .../rpc/netty/RpcEndpointVerifier.scala | 42 + .../common/serializer/JavaSerializer.scala | 159 +++ .../serializer/SerializationDebugger.scala | 425 +++++++ .../css/common/serializer/Serializer.scala | 180 +++ .../bytedance/css/common/util/BitSet.scala | 247 ++++ .../common/util/ByteBufferInputStream.scala | 76 ++ .../common/util/ByteBufferOutputStream.scala | 62 + .../css/common/util/Collections.scala | 42 + .../bytedance/css/common/util/IntParam.scala | 33 + .../css/common/util/MemoryParam.scala | 34 + .../css/common/util/NextIterator.scala | 92 ++ .../bytedance/css/common/util/RpcUtils.scala | 58 + .../css/common/util/ThreadUtils.scala | 296 +++++ .../com/bytedance/css/common/util/Utils.scala | 536 +++++++++ .../css/ShuffleReadMetricsAdapter.scala | 26 + .../css/ShuffleWriteMetricsAdapter.scala | 26 + common/src/test/resources/log4j.properties | 29 + .../css/common.rpc/RpcAddressSuite.scala | 58 + .../css/common.rpc/RpcEnvSuite.scala | 852 +++++++++++++ .../css/common.rpc/TestRpcEndpoint.scala | 125 ++ .../css/common.rpc/netty/InboxSuite.scala | 152 +++ .../netty/NettyRpcAddressSuite.scala | 37 + .../common.rpc/netty/NettyRpcEnvSuite.scala | 181 +++ .../netty/NettyRpcHandlerSuite.scala | 69 ++ conf/css-defaults.conf | 31 + conf/css-metrics.properties.template | 66 ++ conf/log4j-default.properties | 50 + conf/workers.template | 22 + css-assembly_2/pom.xml | 123 ++ css-assembly_3/pom.xml | 123 ++ dev/checkstyle-suppressions.xml | 51 + dev/checkstyle.xml | 193 +++ docs/img/css-arch.png | Bin 0 -> 188640 bytes network-common/pom.xml | 109 ++ .../css/network/TransportContext.java | 231 ++++ .../buffer/FileSegmentManagedBuffer.java | 156 +++ .../css/network/buffer/ManagedBuffer.java | 80 ++ .../network/buffer/NettyManagedBuffer.java | 83 ++ .../css/network/buffer/NioManagedBuffer.java | 78 ++ .../client/ChunkFetchFailureException.java | 33 + .../network/client/ChunkReceivedCallback.java | 49 + .../network/client/RpcResponseCallback.java | 39 + .../css/network/client/StreamCallback.java | 42 + .../network/client/StreamCallbackWithID.java | 24 + .../css/network/client/StreamInterceptor.java | 97 ++ .../css/network/client/TransportClient.java | 390 ++++++ .../client/TransportClientBootstrap.java | 36 + .../client/TransportClientFactory.java | 354 ++++++ .../client/TransportResponseHandler.java | 273 +++++ .../network/crypto/AuthClientBootstrap.java | 121 ++ .../css/network/crypto/AuthEngine.java | 317 +++++ .../css/network/crypto/AuthRpcHandler.java | 143 +++ .../network/crypto/AuthServerBootstrap.java | 57 + .../css/network/crypto/ClientChallenge.java | 103 ++ .../bytedance/css/network/crypto/README.md | 158 +++ .../css/network/crypto/ServerResponse.java | 87 ++ .../css/network/crypto/TransportCipher.java | 377 ++++++ .../css/network/protocol/AbstractMessage.java | 56 + .../protocol/AbstractResponseMessage.java | 34 + .../protocol/BatchPushDataRequest.java | 143 +++ .../network/protocol/ChunkFetchFailure.java | 80 ++ .../network/protocol/ChunkFetchRequest.java | 74 ++ .../network/protocol/ChunkFetchSuccess.java | 93 ++ .../css/network/protocol/Encodable.java | 43 + .../css/network/protocol/Encoders.java | 118 ++ .../css/network/protocol/Message.java | 78 ++ .../css/network/protocol/MessageDecoder.java | 95 ++ .../css/network/protocol/MessageEncoder.java | 99 ++ .../network/protocol/MessageWithHeader.java | 199 ++++ .../css/network/protocol/OneWayMessage.java | 83 ++ .../css/network/protocol/RequestMessage.java | 25 + .../css/network/protocol/ResponseMessage.java | 25 + .../css/network/protocol/RpcFailure.java | 78 ++ .../css/network/protocol/RpcRequest.java | 90 ++ .../css/network/protocol/RpcResponse.java | 90 ++ .../css/network/protocol/StreamChunkId.java | 77 ++ .../css/network/protocol/StreamFailure.java | 80 ++ .../css/network/protocol/StreamRequest.java | 78 ++ .../css/network/protocol/StreamResponse.java | 94 ++ .../css/network/protocol/UploadStream.java | 110 ++ .../shuffle/BlockTransferMessage.java | 69 ++ .../network/protocol/shuffle/OpenStream.java | 89 ++ .../protocol/shuffle/StreamHandle.java | 82 ++ .../css/network/sasl/SaslClientBootstrap.java | 102 ++ .../css/network/sasl/SaslEncryption.java | 331 ++++++ .../network/sasl/SaslEncryptionBackend.java | 35 + .../css/network/sasl/SaslMessage.java | 80 ++ .../css/network/sasl/SaslRpcHandler.java | 147 +++ .../css/network/sasl/SaslServerBootstrap.java | 51 + .../css/network/sasl/SecretKeyHolder.java | 37 + .../css/network/sasl/SparkSaslClient.java | 163 +++ .../css/network/sasl/SparkSaslServer.java | 219 ++++ .../server/AbstractAuthRpcHandler.java | 109 ++ .../css/network/server/ChunkFetchMetrics.java | 42 + .../server/ChunkFetchRequestHandler.java | 158 +++ .../css/network/server/CssFileInfo.java | 37 + .../server/CssManagedBufferIterator.java | 69 ++ .../css/network/server/MessageHandler.java | 41 + .../css/network/server/NoOpRpcHandler.java | 42 + .../server/OneForOneStreamManager.java | 233 ++++ .../css/network/server/RpcHandler.java | 142 +++ .../css/network/server/StreamManager.java | 109 ++ .../server/TransportChannelHandler.java | 199 ++++ .../server/TransportRequestHandler.java | 300 +++++ .../css/network/server/TransportServer.java | 194 +++ .../server/TransportServerBootstrap.java | 38 + .../css/network/util/AbstractFileRegion.java | 55 + .../util/ByteArrayReadableChannel.java | 64 + .../util/ByteArrayWritableChannel.java | 71 ++ .../bytedance/css/network/util/ByteUnit.java | 70 ++ .../css/network/util/ConfigProvider.java | 59 + .../css/network/util/CryptoUtils.java | 51 + .../bytedance/css/network/util/IOMode.java | 29 + .../bytedance/css/network/util/JavaUtils.java | 380 ++++++ .../css/network/util/LevelDBProvider.java | 154 +++ .../css/network/util/LimitedInputStream.java | 130 ++ .../css/network/util/MapConfigProvider.java | 58 + .../css/network/util/NettyUtils.java | 173 +++ .../css/network/util/TransportConf.java | 355 ++++++ .../network/util/TransportFrameDecoder.java | 289 +++++ .../io/netty/buffer/NettyMemoryMetrics.java | 171 +++ .../network/ChunkFetchIntegrationSuite.java | 238 ++++ .../bytedance/css/network/ProtocolSuite.java | 115 ++ .../RequestTimeoutIntegrationSuite.java | 279 +++++ .../css/network/RpcIntegrationSuite.java | 487 ++++++++ .../bytedance/css/network/StreamSuite.java | 309 +++++ .../css/network/StreamTestHelper.java | 105 ++ .../css/network/TestManagedBuffer.java | 110 ++ .../com/bytedance/css/network/TestUtils.java | 34 + .../network/TransportRequestHandlerSuite.java | 154 +++ .../TransportResponseHandlerSuite.java | 166 +++ .../client/TransportClientFactorySuite.java | 219 ++++ .../protocol/MessageWithHeaderSuite.java | 188 +++ .../server/OneForOneStreamManagerSuite.java | 142 +++ .../network/util/NettyMemoryMetricsSuite.java | 173 +++ .../util/TransportFrameDecoderSuite.java | 313 +++++ .../src/test/resources/log4j.properties | 29 + patch/spark-3.0-aqe-skewjoin.patch | 48 + pom.xml | 704 +++++++++++ sbin/css-config.sh | 45 + sbin/css-daemon.sh | 221 ++++ sbin/start-all.sh | 37 + sbin/start-master.sh | 66 ++ sbin/start-worker.sh | 108 ++ sbin/start-workers.sh | 45 + sbin/stop-all.sh | 49 + sbin/stop-master.sh | 30 + sbin/stop-worker.sh | 45 + sbin/stop-workers.sh | 26 + sbin/workers.sh | 98 ++ scalastyle-config.xml | 397 +++++++ service/pom.xml | 128 ++ .../deploy/worker/DiskFileFlushTask.java | 67 ++ .../service/deploy/worker/DiskFileWriter.java | 311 +++++ .../service/deploy/worker/FileFlusher.java | 31 + .../deploy/worker/FileFlusherImpl.java | 103 ++ .../service/deploy/worker/FileNotifier.java | 49 + .../css/service/deploy/worker/FileWriter.java | 68 ++ .../deploy/worker/FileWriterMetrics.java | 94 ++ .../deploy/worker/FileWriterStatus.java | 39 + .../css/service/deploy/worker/FlushTask.java | 48 + .../deploy/worker/HdfsFileFlushTask.java | 61 + .../service/deploy/worker/HdfsFileWriter.java | 300 +++++ .../deploy/worker/WritablePartitionInfo.java | 38 + .../worker/handler/FetchDataHandler.java | 26 + .../worker/handler/FetchDataRpcHandler.java | 162 +++ .../worker/handler/PushDataHandler.java | 27 + .../worker/handler/PushDataRpcHandler.java | 155 +++ .../worker/handler/RecycleShuffleHandler.java | 27 + .../src/main/resources/css-metrics.properties | 66 ++ .../main/resources/log4j-defaults.properties | 50 + .../css/impl/CssShuffleContextImpl.scala | 95 ++ .../service/deploy/common/BaseSource.scala | 96 ++ .../deploy/common/HeartbeatReceiver.scala | 100 ++ .../deploy/common/ScheduledManager.scala | 86 ++ .../deploy/master/AssignStrategy.scala | 100 ++ .../css/service/deploy/master/Master.scala | 332 ++++++ .../deploy/master/MasterArguments.scala | 92 ++ .../service/deploy/master/MasterSource.scala | 64 + .../deploy/master/RandomAssignStrategy.scala | 79 ++ .../deploy/master/ShuffleAppManager.scala | 100 ++ .../deploy/master/ShuffleStageManager.scala | 75 ++ .../master/ShuffleStageManagerImpl.scala | 290 +++++ .../deploy/master/ShuffleTaskManager.scala | 183 +++ .../deploy/master/ShuffleWorkerManager.scala | 138 +++ .../deploy/metadata/ExternalShuffleMeta.scala | 64 + .../deploy/metadata/WorkerProvider.scala | 101 ++ .../deploy/metadata/WorkerRegistry.scala | 55 + .../standalone/StandaloneWorkerProvider.scala | 37 + .../standalone/StandaloneWorkerRegistry.scala | 82 ++ .../zookeeper/ZooKeeperWorkerRegistry.scala | 220 ++++ .../metadata/zookeeper/ZookeeperClient.scala | 182 +++ .../ZookeeperExternalShuffleMeta.scala | 144 +++ .../zookeeper/ZookeeperWorkerProvider.scala | 112 ++ .../css/service/deploy/worker/Storage.scala | 288 +++++ .../deploy/worker/StorageManager.scala | 147 +++ .../css/service/deploy/worker/Worker.scala | 561 +++++++++ .../deploy/worker/WorkerArguments.scala | 115 ++ .../service/deploy/worker/WorkerInfo.scala | 155 +++ .../service/deploy/worker/WorkerSource.scala | 84 ++ .../deploy/worker/DiskFileFlushTaskSuite.java | 68 ++ .../deploy/worker/DiskFileFlusherSuite.java | 93 ++ .../deploy/worker/DiskFileWriterSuite.java | 309 +++++ .../deploy/worker/FileNotifierSuite.java | 45 + .../deploy/worker/HdfsFileFlushTaskSuite.java | 77 ++ .../deploy/worker/HdfsFileFlusherSuite.java | 101 ++ .../deploy/worker/HdfsFileWriterSuite.java | 104 ++ .../deploy/master/AssignStrategySuite.scala | 139 +++ .../deploy/master/BatchPushDataSuite.scala | 193 +++ .../service/deploy/master/CleanupSuite.scala | 98 ++ .../deploy/master/DeDuplicateSuite.scala | 190 +++ .../deploy/master/EpochRotateSuite.scala | 136 +++ .../deploy/master/FetchDataSuite.scala | 139 +++ .../service/deploy/master/LocalCluster.scala | 288 +++++ .../deploy/master/RegisterShuffleSuite.scala | 292 +++++ .../deploy/master/ShuffleClientSuite.scala | 328 +++++ .../service/deploy/master/StageEndSuite.scala | 102 ++ spark-shuffle-manager-2/pom.xml | 116 ++ .../shuffle/css/AsyncPushDataTaskManager.java | 1 + .../spark/shuffle/css/CssShuffleWriter.java | 290 +++++ .../java/org/apache/spark/shuffle/css/sort | 1 + .../spark/shuffle/css/CssShuffleHandle.scala | 37 + .../spark/shuffle/css/CssShuffleManager.scala | 188 +++ .../spark/shuffle/css/CssShuffleReader.scala | 130 ++ spark-shuffle-manager-3/pom.xml | 104 ++ .../shuffle/css/AsyncPushDataTaskManager.java | 255 ++++ .../spark/shuffle/css/CssShuffleWriter.java | 293 +++++ .../css/sort/CssShuffleExternalSorter.java | 450 +++++++ .../shuffle/css/sort/PackedRecordPointer.java | 104 ++ .../css/sort/ShuffleInMemorySorter.java | 205 ++++ .../css/sort/ShuffleSortDataFormat.java | 80 ++ .../spark/shuffle/css/sort/SpillInfo.java | 36 + .../spark/shuffle/css/CssShuffleHandle.scala | 37 + .../spark/shuffle/css/CssShuffleManager.scala | 236 ++++ .../spark/shuffle/css/CssShuffleReader.scala | 131 ++ version_release.sh | 36 + 343 files changed, 45032 insertions(+) create mode 100644 .github/ISSUE_TEMPLATE/bug-report.yaml create mode 100644 .github/ISSUE_TEMPLATE/config.yml create mode 100644 .github/ISSUE_TEMPLATE/enhancement.yaml create mode 100644 .github/PULL_REQUEST_TEMPLATE.md create mode 100644 .github/workflows/ut.yml create mode 100644 .gitignore create mode 100644 ACKNOWLEDGEMENTS.md create mode 100644 CODE_OF_CONDUCT.md create mode 100644 CONTRIBUTING.md create mode 100644 LICENSE create mode 100644 README.md create mode 100644 README_zh.md create mode 100644 api/pom.xml create mode 100644 api/src/main/java/com/bytedance/css/api/CssShuffleContext.java create mode 100755 build.sh create mode 100644 client/pom.xml create mode 100644 client/src/main/java/com/bytedance/css/client/MapperAttemptStatus.java create mode 100644 client/src/main/java/com/bytedance/css/client/MetricsCallback.java create mode 100644 client/src/main/java/com/bytedance/css/client/ShuffleClient.java create mode 100644 client/src/main/java/com/bytedance/css/client/compress/Compressor.java create mode 100644 client/src/main/java/com/bytedance/css/client/compress/CompressorFactory.java create mode 100644 client/src/main/java/com/bytedance/css/client/compress/CssCompressorFactory.java create mode 100644 client/src/main/java/com/bytedance/css/client/compress/CssCompressorTrait.java create mode 100644 client/src/main/java/com/bytedance/css/client/compress/Decompressor.java create mode 100644 client/src/main/java/com/bytedance/css/client/compress/Lz4Compressor.java create mode 100644 client/src/main/java/com/bytedance/css/client/compress/Lz4Decompressor.java create mode 100644 client/src/main/java/com/bytedance/css/client/compress/ZstdCompressor.java create mode 100644 client/src/main/java/com/bytedance/css/client/compress/ZstdDecompressor.java create mode 100644 client/src/main/java/com/bytedance/css/client/compress/ZstdDictCompressor.java create mode 100644 client/src/main/java/com/bytedance/css/client/compress/ZstdDictDecompressor.java create mode 100644 client/src/main/java/com/bytedance/css/client/compress/ZstdDictTrainer.java create mode 100644 client/src/main/java/com/bytedance/css/client/impl/ShuffleClientImpl.java create mode 100644 client/src/main/java/com/bytedance/css/client/metrics/BaseSource.java create mode 100644 client/src/main/java/com/bytedance/css/client/metrics/ClientSource.java create mode 100644 client/src/main/java/com/bytedance/css/client/stream/CssInputStream.java create mode 100644 client/src/main/java/com/bytedance/css/client/stream/CssInputStreamImpl.java create mode 100644 client/src/main/java/com/bytedance/css/client/stream/EpochInputStreamBuilder.java create mode 100644 client/src/main/java/com/bytedance/css/client/stream/Frame.java create mode 100644 client/src/main/java/com/bytedance/css/client/stream/FrameIterator.java create mode 100644 client/src/main/java/com/bytedance/css/client/stream/disk/CssLocalDiskEpochReader.java create mode 100644 client/src/main/java/com/bytedance/css/client/stream/disk/CssRemoteDiskEpochReader.java create mode 100644 client/src/main/java/com/bytedance/css/client/stream/disk/EpochDiskInputStreamBuilder.java create mode 100644 client/src/main/java/com/bytedance/css/client/stream/hdfs/EpochHdfsInputStreamBuilder.java create mode 100644 client/src/test/java/com/bytedance/css/client/compress/CompressorSuite.java create mode 100644 client/src/test/java/com/bytedance/css/client/stream/CssDiskInputStreamSuite.java create mode 100644 client/src/test/java/com/bytedance/css/client/stream/CssEpochReaderRetryCorrectSuite.java create mode 100644 client/src/test/java/com/bytedance/css/client/stream/CssEpochReaderRetrySuite.java create mode 100644 client/src/test/java/com/bytedance/css/client/stream/CssHdfsInputStreamSuite.java create mode 100644 client/src/test/resources/log4j.properties create mode 100644 client/src/test/resources/xml create mode 100644 common/pom.xml create mode 100644 common/src/main/java/com/bytedance/css/common/ChildFirstURLClassLoader.java create mode 100644 common/src/main/java/com/bytedance/css/common/ParentClassLoader.java create mode 100644 common/src/main/java/com/bytedance/css/common/metrics/ByteDanceMetricsEmitter.java create mode 100644 common/src/main/java/com/bytedance/css/common/metrics/ByteDanceMetricsReporter.java create mode 100644 common/src/main/java/com/bytedance/css/common/protocol/CommittedPartitionInfo.java create mode 100644 common/src/main/java/com/bytedance/css/common/protocol/CssStatusCode.java create mode 100644 common/src/main/java/com/bytedance/css/common/protocol/FailedPartitionInfoBatch.java create mode 100644 common/src/main/java/com/bytedance/css/common/protocol/PartitionGroup.java create mode 100644 common/src/main/java/com/bytedance/css/common/protocol/PartitionGroupManager.java create mode 100644 common/src/main/java/com/bytedance/css/common/protocol/PartitionInfo.java create mode 100644 common/src/main/java/com/bytedance/css/common/protocol/ReplicaBase.java create mode 100644 common/src/main/java/com/bytedance/css/common/protocol/RpcNameConstants.java create mode 100644 common/src/main/java/com/bytedance/css/common/protocol/ShuffleMode.java create mode 100644 common/src/main/java/com/bytedance/css/common/protocol/TransportModuleConstants.java create mode 100644 common/src/main/java/com/bytedance/css/common/protocol/WorkerAddress.java create mode 100644 common/src/main/java/com/bytedance/css/common/protocol/WorkerStatus.java create mode 100644 common/src/main/java/com/bytedance/css/common/unsafe/Platform.java create mode 100644 common/src/main/java/com/bytedance/css/common/util/JsonUtils.java create mode 100644 common/src/main/scala/com/bytedance/css/common/CssConf.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/exception/AlreadyClosedException.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/exception/CssException.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/exception/EpochShouldRotateException.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/internal/Logging.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/internal/config/ConfigBuilder.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/internal/config/ConfigEntry.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/internal/config/ConfigProvider.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/internal/config/ConfigReader.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/internal/config/package.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/metrics/MetricsConfig.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/metrics/MetricsSystem.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/metrics/sink/BytedanceSink.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/metrics/sink/ConsoleSink.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/metrics/sink/CsvSink.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/metrics/sink/GraphiteSink.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/metrics/sink/JmxSink.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/metrics/sink/Sink.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/metrics/sink/Slf4jSink.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/metrics/sink/StatsdReporter.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/metrics/sink/StatsdSink.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/metrics/source/Source.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/protocol/CssRpcMessage.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/rpc/RpcAddress.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/rpc/RpcCallContext.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/rpc/RpcEndpoint.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/rpc/RpcEndpointAddress.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/rpc/RpcEndpointNotFoundException.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/rpc/RpcEndpointRef.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/rpc/RpcEnv.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/rpc/RpcEnvStoppedException.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/rpc/RpcTimeout.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/rpc/netty/Dispatcher.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/rpc/netty/Inbox.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/rpc/netty/NettyRpcCallContext.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/rpc/netty/NettyRpcEnv.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/rpc/netty/NettyStreamManager.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/rpc/netty/Outbox.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/rpc/netty/RpcEndpointVerifier.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/serializer/JavaSerializer.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/serializer/SerializationDebugger.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/serializer/Serializer.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/util/BitSet.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/util/ByteBufferInputStream.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/util/ByteBufferOutputStream.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/util/Collections.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/util/IntParam.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/util/MemoryParam.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/util/NextIterator.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/util/RpcUtils.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/util/ThreadUtils.scala create mode 100644 common/src/main/scala/com/bytedance/css/common/util/Utils.scala create mode 100644 common/src/main/scala/org/apache/spark/shuffle/css/ShuffleReadMetricsAdapter.scala create mode 100644 common/src/main/scala/org/apache/spark/shuffle/css/ShuffleWriteMetricsAdapter.scala create mode 100644 common/src/test/resources/log4j.properties create mode 100644 common/src/test/scala/com/bytedance/css/common.rpc/RpcAddressSuite.scala create mode 100644 common/src/test/scala/com/bytedance/css/common.rpc/RpcEnvSuite.scala create mode 100644 common/src/test/scala/com/bytedance/css/common.rpc/TestRpcEndpoint.scala create mode 100644 common/src/test/scala/com/bytedance/css/common.rpc/netty/InboxSuite.scala create mode 100644 common/src/test/scala/com/bytedance/css/common.rpc/netty/NettyRpcAddressSuite.scala create mode 100644 common/src/test/scala/com/bytedance/css/common.rpc/netty/NettyRpcEnvSuite.scala create mode 100644 common/src/test/scala/com/bytedance/css/common.rpc/netty/NettyRpcHandlerSuite.scala create mode 100644 conf/css-defaults.conf create mode 100644 conf/css-metrics.properties.template create mode 100644 conf/log4j-default.properties create mode 100644 conf/workers.template create mode 100644 css-assembly_2/pom.xml create mode 100644 css-assembly_3/pom.xml create mode 100644 dev/checkstyle-suppressions.xml create mode 100644 dev/checkstyle.xml create mode 100644 docs/img/css-arch.png create mode 100644 network-common/pom.xml create mode 100644 network-common/src/main/java/com/bytedance/css/network/TransportContext.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/buffer/FileSegmentManagedBuffer.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/buffer/ManagedBuffer.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/buffer/NettyManagedBuffer.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/buffer/NioManagedBuffer.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/client/ChunkFetchFailureException.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/client/ChunkReceivedCallback.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/client/RpcResponseCallback.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/client/StreamCallback.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/client/StreamCallbackWithID.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/client/StreamInterceptor.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/client/TransportClient.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/client/TransportClientBootstrap.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/client/TransportClientFactory.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/client/TransportResponseHandler.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/crypto/AuthClientBootstrap.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/crypto/AuthEngine.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/crypto/AuthRpcHandler.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/crypto/AuthServerBootstrap.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/crypto/ClientChallenge.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/crypto/README.md create mode 100644 network-common/src/main/java/com/bytedance/css/network/crypto/ServerResponse.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/crypto/TransportCipher.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/AbstractMessage.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/AbstractResponseMessage.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/BatchPushDataRequest.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/ChunkFetchFailure.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/ChunkFetchRequest.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/ChunkFetchSuccess.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/Encodable.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/Encoders.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/Message.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/MessageDecoder.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/MessageEncoder.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/MessageWithHeader.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/OneWayMessage.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/RequestMessage.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/ResponseMessage.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/RpcFailure.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/RpcRequest.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/RpcResponse.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/StreamChunkId.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/StreamFailure.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/StreamRequest.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/StreamResponse.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/UploadStream.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/shuffle/BlockTransferMessage.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/shuffle/OpenStream.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/protocol/shuffle/StreamHandle.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/sasl/SaslClientBootstrap.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/sasl/SaslEncryption.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/sasl/SaslEncryptionBackend.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/sasl/SaslMessage.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/sasl/SaslRpcHandler.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/sasl/SaslServerBootstrap.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/sasl/SecretKeyHolder.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/sasl/SparkSaslClient.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/sasl/SparkSaslServer.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/server/AbstractAuthRpcHandler.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/server/ChunkFetchMetrics.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/server/ChunkFetchRequestHandler.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/server/CssFileInfo.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/server/CssManagedBufferIterator.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/server/MessageHandler.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/server/NoOpRpcHandler.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/server/OneForOneStreamManager.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/server/RpcHandler.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/server/StreamManager.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/server/TransportChannelHandler.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/server/TransportRequestHandler.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/server/TransportServer.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/server/TransportServerBootstrap.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/util/AbstractFileRegion.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/util/ByteArrayReadableChannel.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/util/ByteArrayWritableChannel.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/util/ByteUnit.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/util/ConfigProvider.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/util/CryptoUtils.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/util/IOMode.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/util/JavaUtils.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/util/LevelDBProvider.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/util/LimitedInputStream.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/util/MapConfigProvider.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/util/NettyUtils.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/util/TransportConf.java create mode 100644 network-common/src/main/java/com/bytedance/css/network/util/TransportFrameDecoder.java create mode 100644 network-common/src/main/java/io/netty/buffer/NettyMemoryMetrics.java create mode 100644 network-common/src/test/java/com/bytedance/css/network/ChunkFetchIntegrationSuite.java create mode 100644 network-common/src/test/java/com/bytedance/css/network/ProtocolSuite.java create mode 100644 network-common/src/test/java/com/bytedance/css/network/RequestTimeoutIntegrationSuite.java create mode 100644 network-common/src/test/java/com/bytedance/css/network/RpcIntegrationSuite.java create mode 100644 network-common/src/test/java/com/bytedance/css/network/StreamSuite.java create mode 100644 network-common/src/test/java/com/bytedance/css/network/StreamTestHelper.java create mode 100644 network-common/src/test/java/com/bytedance/css/network/TestManagedBuffer.java create mode 100644 network-common/src/test/java/com/bytedance/css/network/TestUtils.java create mode 100644 network-common/src/test/java/com/bytedance/css/network/TransportRequestHandlerSuite.java create mode 100644 network-common/src/test/java/com/bytedance/css/network/TransportResponseHandlerSuite.java create mode 100644 network-common/src/test/java/com/bytedance/css/network/client/TransportClientFactorySuite.java create mode 100644 network-common/src/test/java/com/bytedance/css/network/protocol/MessageWithHeaderSuite.java create mode 100644 network-common/src/test/java/com/bytedance/css/network/server/OneForOneStreamManagerSuite.java create mode 100644 network-common/src/test/java/com/bytedance/css/network/util/NettyMemoryMetricsSuite.java create mode 100644 network-common/src/test/java/com/bytedance/css/network/util/TransportFrameDecoderSuite.java create mode 100644 network-common/src/test/resources/log4j.properties create mode 100644 patch/spark-3.0-aqe-skewjoin.patch create mode 100644 pom.xml create mode 100755 sbin/css-config.sh create mode 100755 sbin/css-daemon.sh create mode 100755 sbin/start-all.sh create mode 100755 sbin/start-master.sh create mode 100755 sbin/start-worker.sh create mode 100755 sbin/start-workers.sh create mode 100755 sbin/stop-all.sh create mode 100755 sbin/stop-master.sh create mode 100755 sbin/stop-worker.sh create mode 100755 sbin/stop-workers.sh create mode 100755 sbin/workers.sh create mode 100644 scalastyle-config.xml create mode 100644 service/pom.xml create mode 100644 service/src/main/java/com/bytedance/css/service/deploy/worker/DiskFileFlushTask.java create mode 100644 service/src/main/java/com/bytedance/css/service/deploy/worker/DiskFileWriter.java create mode 100644 service/src/main/java/com/bytedance/css/service/deploy/worker/FileFlusher.java create mode 100644 service/src/main/java/com/bytedance/css/service/deploy/worker/FileFlusherImpl.java create mode 100644 service/src/main/java/com/bytedance/css/service/deploy/worker/FileNotifier.java create mode 100644 service/src/main/java/com/bytedance/css/service/deploy/worker/FileWriter.java create mode 100644 service/src/main/java/com/bytedance/css/service/deploy/worker/FileWriterMetrics.java create mode 100644 service/src/main/java/com/bytedance/css/service/deploy/worker/FileWriterStatus.java create mode 100644 service/src/main/java/com/bytedance/css/service/deploy/worker/FlushTask.java create mode 100644 service/src/main/java/com/bytedance/css/service/deploy/worker/HdfsFileFlushTask.java create mode 100644 service/src/main/java/com/bytedance/css/service/deploy/worker/HdfsFileWriter.java create mode 100644 service/src/main/java/com/bytedance/css/service/deploy/worker/WritablePartitionInfo.java create mode 100644 service/src/main/java/com/bytedance/css/service/deploy/worker/handler/FetchDataHandler.java create mode 100644 service/src/main/java/com/bytedance/css/service/deploy/worker/handler/FetchDataRpcHandler.java create mode 100644 service/src/main/java/com/bytedance/css/service/deploy/worker/handler/PushDataHandler.java create mode 100644 service/src/main/java/com/bytedance/css/service/deploy/worker/handler/PushDataRpcHandler.java create mode 100644 service/src/main/java/com/bytedance/css/service/deploy/worker/handler/RecycleShuffleHandler.java create mode 100644 service/src/main/resources/css-metrics.properties create mode 100644 service/src/main/resources/log4j-defaults.properties create mode 100644 service/src/main/scala/com/bytedance/css/impl/CssShuffleContextImpl.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/common/BaseSource.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/common/HeartbeatReceiver.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/common/ScheduledManager.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/master/AssignStrategy.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/master/Master.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/master/MasterArguments.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/master/MasterSource.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/master/RandomAssignStrategy.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleAppManager.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleStageManager.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleStageManagerImpl.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleTaskManager.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleWorkerManager.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/metadata/ExternalShuffleMeta.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/metadata/WorkerProvider.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/metadata/WorkerRegistry.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/metadata/standalone/StandaloneWorkerProvider.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/metadata/standalone/StandaloneWorkerRegistry.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/metadata/zookeeper/ZooKeeperWorkerRegistry.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/metadata/zookeeper/ZookeeperClient.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/metadata/zookeeper/ZookeeperExternalShuffleMeta.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/metadata/zookeeper/ZookeeperWorkerProvider.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/worker/Storage.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/worker/StorageManager.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/worker/Worker.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/worker/WorkerArguments.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/worker/WorkerInfo.scala create mode 100644 service/src/main/scala/com/bytedance/css/service/deploy/worker/WorkerSource.scala create mode 100644 service/src/test/java/com/bytedance/css/service/deploy/worker/DiskFileFlushTaskSuite.java create mode 100644 service/src/test/java/com/bytedance/css/service/deploy/worker/DiskFileFlusherSuite.java create mode 100644 service/src/test/java/com/bytedance/css/service/deploy/worker/DiskFileWriterSuite.java create mode 100644 service/src/test/java/com/bytedance/css/service/deploy/worker/FileNotifierSuite.java create mode 100644 service/src/test/java/com/bytedance/css/service/deploy/worker/HdfsFileFlushTaskSuite.java create mode 100644 service/src/test/java/com/bytedance/css/service/deploy/worker/HdfsFileFlusherSuite.java create mode 100644 service/src/test/java/com/bytedance/css/service/deploy/worker/HdfsFileWriterSuite.java create mode 100644 service/src/test/scala/com/bytedance/css/service/deploy/master/AssignStrategySuite.scala create mode 100644 service/src/test/scala/com/bytedance/css/service/deploy/master/BatchPushDataSuite.scala create mode 100644 service/src/test/scala/com/bytedance/css/service/deploy/master/CleanupSuite.scala create mode 100644 service/src/test/scala/com/bytedance/css/service/deploy/master/DeDuplicateSuite.scala create mode 100644 service/src/test/scala/com/bytedance/css/service/deploy/master/EpochRotateSuite.scala create mode 100644 service/src/test/scala/com/bytedance/css/service/deploy/master/FetchDataSuite.scala create mode 100644 service/src/test/scala/com/bytedance/css/service/deploy/master/LocalCluster.scala create mode 100644 service/src/test/scala/com/bytedance/css/service/deploy/master/RegisterShuffleSuite.scala create mode 100644 service/src/test/scala/com/bytedance/css/service/deploy/master/ShuffleClientSuite.scala create mode 100644 service/src/test/scala/com/bytedance/css/service/deploy/master/StageEndSuite.scala create mode 100644 spark-shuffle-manager-2/pom.xml create mode 120000 spark-shuffle-manager-2/src/main/java/org/apache/spark/shuffle/css/AsyncPushDataTaskManager.java create mode 100644 spark-shuffle-manager-2/src/main/java/org/apache/spark/shuffle/css/CssShuffleWriter.java create mode 120000 spark-shuffle-manager-2/src/main/java/org/apache/spark/shuffle/css/sort create mode 100644 spark-shuffle-manager-2/src/main/scala/org/apache/spark/shuffle/css/CssShuffleHandle.scala create mode 100644 spark-shuffle-manager-2/src/main/scala/org/apache/spark/shuffle/css/CssShuffleManager.scala create mode 100644 spark-shuffle-manager-2/src/main/scala/org/apache/spark/shuffle/css/CssShuffleReader.scala create mode 100644 spark-shuffle-manager-3/pom.xml create mode 100644 spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/AsyncPushDataTaskManager.java create mode 100644 spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/CssShuffleWriter.java create mode 100644 spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/CssShuffleExternalSorter.java create mode 100644 spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/PackedRecordPointer.java create mode 100644 spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/ShuffleInMemorySorter.java create mode 100644 spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/ShuffleSortDataFormat.java create mode 100644 spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/SpillInfo.java create mode 100644 spark-shuffle-manager-3/src/main/scala/org/apache/spark/shuffle/css/CssShuffleHandle.scala create mode 100644 spark-shuffle-manager-3/src/main/scala/org/apache/spark/shuffle/css/CssShuffleManager.scala create mode 100644 spark-shuffle-manager-3/src/main/scala/org/apache/spark/shuffle/css/CssShuffleReader.scala create mode 100755 version_release.sh diff --git a/.github/ISSUE_TEMPLATE/bug-report.yaml b/.github/ISSUE_TEMPLATE/bug-report.yaml new file mode 100644 index 0000000..d28a0ac --- /dev/null +++ b/.github/ISSUE_TEMPLATE/bug-report.yaml @@ -0,0 +1,41 @@ +name: Bug Report +description: Report a bug encountered +body: + - type: textarea + id: problem + attributes: + label: What happened? + description: | + Please provide as much info as possible. Not doing so may result in your bug not being addressed in a timely manner. + If this matter is security related, please disclose it privately via https://security.bytedance.com + validations: + required: true + + - type: textarea + id: expected + attributes: + label: What did you expect to happen? + validations: + required: true + + - type: textarea + id: repro + attributes: + label: How can we reproduce it (as minimally and precisely as possible)? + validations: + required: true + + - type: textarea + id: version + attributes: + label: Software version + value: | +
+ + ```console + $ version + # paste output here + ``` +
+ validations: + required: false diff --git a/.github/ISSUE_TEMPLATE/config.yml b/.github/ISSUE_TEMPLATE/config.yml new file mode 100644 index 0000000..a49eab2 --- /dev/null +++ b/.github/ISSUE_TEMPLATE/config.yml @@ -0,0 +1 @@ +blank_issues_enabled: true \ No newline at end of file diff --git a/.github/ISSUE_TEMPLATE/enhancement.yaml b/.github/ISSUE_TEMPLATE/enhancement.yaml new file mode 100644 index 0000000..5076eab --- /dev/null +++ b/.github/ISSUE_TEMPLATE/enhancement.yaml @@ -0,0 +1,16 @@ +name: Feature Tracking Issue +description: Provide supporting details for a feature in development +body: + - type: textarea + id: feature + attributes: + label: What would you like to be added? + validations: + required: true + + - type: textarea + id: rationale + attributes: + label: Why is this needed? + validations: + required: true diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md new file mode 100644 index 0000000..a85dcc9 --- /dev/null +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -0,0 +1,10 @@ +#### What type of PR is this? + + +#### What this PR does / why we need it: + +#### Which issue(s) this PR fixes: + +#### Special notes for your reviewer: diff --git a/.github/workflows/ut.yml b/.github/workflows/ut.yml new file mode 100644 index 0000000..e78e7ec --- /dev/null +++ b/.github/workflows/ut.yml @@ -0,0 +1,30 @@ +name: Junit + +on: + push: + branches: [ main ] + pull_request: + branches: [ main ] + workflow_dispatch: + +jobs: + build: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v2 + - uses: actions/cache@v1 + with: + path: ~/.m2/repository + key: ${{ runner.os }}-maven-${{ hashFiles('**/pom.xml') }} + restore-keys: | + ${{ runner.os }}-maven- + - name: Set up JDK 8 + uses: actions/setup-java@v1 + with: + java-version: 8 + - name: Build with Maven + run: mvn -B test --file pom.xml + - uses: ashley-taylor/junit-report-annotations-action@master + with: + access-token: ${{ secrets.GITHUB_TOKEN }} + if: always() \ No newline at end of file diff --git a/.gitignore b/.gitignore new file mode 100644 index 0000000..bdf9ff4 --- /dev/null +++ b/.gitignore @@ -0,0 +1,93 @@ +*#*# +*.#* +*.iml +*.ipr +*.iws +*.pyc +*.pyo +*.swp +*~ +.DS_Store +.cache +.classpath +.ensime +.ensime_cache/ +.ensime_lucene +.generated-mima* +.idea/ +.idea_modules/ +.project +.pydevproject +.scala_dependencies +.settings +/lib/ +R-unit-tests.log +R/unit-tests.out +R/cran-check.out +R/pkg/vignettes/sparkr-vignettes.html +R/pkg/tests/fulltests/Rplots.pdf +build/*.jar +build/apache-maven* +build/scala* +build/zinc* +cache +checkpoint +dependency-reduced-pom.xml +derby.log +dev/create-release/*final +dev/create-release/*txt +dev/pr-deps/ +dist/ +docs/_site +docs/api +sql/docs +sql/site +lib_managed/ +lint-r-report.log +log/ +logs/ +out/ +output/ +project/boot/ +project/build/target/ +project/plugins/lib_managed/ +project/plugins/project/build.properties +project/plugins/src_managed/ +project/plugins/target/ +python/lib/pyspark.zip +python/.eggs/ +python/deps +python/test_coverage/coverage_data +python/test_coverage/htmlcov +python/pyspark/python +reports/ +scalastyle-on-compile.generated.xml +scalastyle-output.xml +scalastyle.txt +spark-*-bin-*.tgz +spark-tests.log +src_managed/ +streaming-tests.log +target/ +unit-tests.log +work/ +docs/.jekyll-metadata +*.crc +css-*-bin.tgz + +# For Hive +TempStatsStore/ +metastore/ +metastore_db/ +sql/hive-thriftserver/test_warehouses +warehouse/ +spark-warehouse/ + +# For R session data +.RData +.RHistory +.Rhistory +*.Rproj +*.Rproj.* + +.Rproj.user diff --git a/ACKNOWLEDGEMENTS.md b/ACKNOWLEDGEMENTS.md new file mode 100644 index 0000000..a9e90d8 --- /dev/null +++ b/ACKNOWLEDGEMENTS.md @@ -0,0 +1,10 @@ +Cloud Shuffle Service(CSS) would not have been possible without the following contributors: +- Chang Liu +- Chen Wei +- Hang Chen +- Hao Wang +- Lei Liu +- Rui Shi +- Xuewei Lin +- Xin Gao +- Zhongjia Wei diff --git a/CODE_OF_CONDUCT.md b/CODE_OF_CONDUCT.md new file mode 100644 index 0000000..41dd9eb --- /dev/null +++ b/CODE_OF_CONDUCT.md @@ -0,0 +1,127 @@ +# Contributor Covenant Code of Conduct + +## Our Pledge + +We as members, contributors, and leaders pledge to make participation in our +community a harassment-free experience for everyone, regardless of age, body +size, visible or invisible disability, ethnicity, sex characteristics, gender +identity and expression, level of experience, education, socio-economic status, +nationality, personal appearance, race, religion, or sexual identity +and orientation. + +We pledge to act and interact in ways that contribute to an open, welcoming, +diverse, inclusive, and healthy community. + +## Our Standards + +Examples of behavior that contributes to a positive environment for our +community include: + +* Demonstrating empathy and kindness toward other people +* Being respectful of differing opinions, viewpoints, and experiences +* Giving and gracefully accepting constructive feedback +* Accepting responsibility and apologizing to those affected by our mistakes, + and learning from the experience +* Focusing on what is best not just for us as individuals, but for the + overall community + +Examples of unacceptable behavior include: + +* The use of sexualized language or imagery, and sexual attention or + advances of any kind +* Trolling, insulting or derogatory comments, and personal or political attacks +* Public or private harassment +* Publishing others' private information, such as a physical or email + address, without their explicit permission +* Other conduct which could reasonably be considered inappropriate in a + professional setting + +## Enforcement Responsibilities + +Community leaders are responsible for clarifying and enforcing our standards of +acceptable behavior and will take appropriate and fair corrective action in +response to any behavior that they deem inappropriate, threatening, offensive, +or harmful. + +Community leaders have the right and responsibility to remove, edit, or reject +comments, commits, code, wiki edits, issues, and other contributions that are +not aligned to this Code of Conduct, and will communicate reasons for moderation +decisions when appropriate. + +## Scope + +This Code of Conduct applies within all community spaces, and also applies when +an individual is officially representing the community in public spaces. +Examples of representing our community include using an official e-mail address, +posting via an official social media account, or acting as an appointed +representative at an online or offline event. + +## Enforcement + +Instances of abusive, harassing, or otherwise unacceptable behavior may be +reported to the community leaders responsible for enforcement. +All complaints will be reviewed and investigated promptly and fairly. + +All community leaders are obligated to respect the privacy and security of the +reporter of any incident. + +## Enforcement Guidelines + +Community leaders will follow these Community Impact Guidelines in determining +the consequences for any action they deem in violation of this Code of Conduct: + +### 1. Correction + +**Community Impact**: Use of inappropriate language or other behavior deemed +unprofessional or unwelcome in the community. + +**Consequence**: A private, written warning from community leaders, providing +clarity around the nature of the violation and an explanation of why the +behavior was inappropriate. A public apology may be requested. + +### 2. Warning + +**Community Impact**: A violation through a single incident or series +of actions. + +**Consequence**: A warning with consequences for continued behavior. No +interaction with the people involved, including unsolicited interaction with +those enforcing the Code of Conduct, for a specified period of time. This +includes avoiding interactions in community spaces as well as external channels +like social media. Violating these terms may lead to a temporary or +permanent ban. + +### 3. Temporary Ban + +**Community Impact**: A serious violation of community standards, including +sustained inappropriate behavior. + +**Consequence**: A temporary ban from any sort of interaction or public +communication with the community for a specified period of time. No public or +private interaction with the people involved, including unsolicited interaction +with those enforcing the Code of Conduct, is allowed during this period. +Violating these terms may lead to a permanent ban. + +### 4. Permanent Ban + +**Community Impact**: Demonstrating a pattern of violation of community +standards, including sustained inappropriate behavior, harassment of an +individual, or aggression toward or disparagement of classes of individuals. + +**Consequence**: A permanent ban from any sort of public interaction within +the community. + +## Attribution + +This Code of Conduct is adapted from the [Contributor Covenant][homepage], +version 2.0, available at +https://www.contributor-covenant.org/version/2/0/code_of_conduct.html. + +Community Impact Guidelines were inspired by [Mozilla's code of conduct +enforcement ladder](https://github.com/mozilla/diversity). + +[homepage]: https://www.contributor-covenant.org + +For answers to common questions about this code of conduct, see the FAQ at +https://www.contributor-covenant.org/faq. Translations are available at +https://www.contributor-covenant.org/translations. diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md new file mode 100644 index 0000000..5646539 --- /dev/null +++ b/CONTRIBUTING.md @@ -0,0 +1,19 @@ +## Contributing to Cloud Shuffle Service(CSS) + +If you want to report a bug or contribute new feature, you can start a discussion by creating an issue first, +and then submit a pull request for your code changes. It's better to find known issues before creating a new one. + +### Discuss via an Issue +- Search for known issues to check whether there have been an existing topic +- If Yes, you can participate in the discussion +- If Not, create a new issue by yourself according to the doc [Create a New Issue](https://docs.github.com/en/issues/tracking-your-work-with-issues/creating-an-issue) + +### Submit a Pull Request +If it is the first time to submit a pull request, you can read this doc [About Pull Request](https://docs.github.com/en/pull-requests/collaborating-with-pull-requests/proposing-changes-to-your-work-with-pull-requests/about-pull-requests) + +- [Fork CSS Repo](https://docs.github.com/en/get-started/quickstart/fork-a-repo) +- Checkout a new branch in your fork repo +- Patch you code, including neccessary test cases & coding-style check +- Commit changes to the branch and push to the fork repo +- Create a pull request to the ByteDance CSS repo + diff --git a/LICENSE b/LICENSE new file mode 100644 index 0000000..5d09a7a --- /dev/null +++ b/LICENSE @@ -0,0 +1,253 @@ + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + 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. + + +------------------------------------------------------------------------------------ +This product bundles various third-party components under other open source licenses. +This section summarizes those components and their licenses. See licenses/ +for text of these licenses. + + +Apache Software Foundation License 2.0 +-------------------------------------- + +com.netflix.concurrency-limits:concurrency-limits-core:0.3.6 +com.fasterxml.jackson.core:jackson-databind:2.10.0 +com.fasterxml.jackson.core:jackson-annotations:2.10.0 +org.apache.spark:spark-tags_2.12:3.0.1 +com.google.guava:guava:14.0.1 +org.lz4:lz4-java:1.7.1 +log4j:log4j:1.2.17 +org.slf4j:slf4j-api:1.7.30 +org.slf4j:slf4j-log4j12:1.7.30 +org.slf4j:jul-to-slf4j:1.7.30 +org.slf4j:jcl-over-slf4j:1.7.30 +junit:junit:4.12 EPL 1.0 +org.scalatest:scalatest_2.12:3.0.8 +io.netty:netty-all:4.1.47.Final +org.apache.commons:commons-lang3:3.9 +com.google.code.findbugs:jsr305:3.0.0 +org.apache.commons:commons-crypto:1.0.0 +io.dropwizard.metrics:metrics-core:4.1.1 +io.dropwizard.metrics:metrics-graphite:4.1.1 +org.apache.hadoop:hadoop-client:2.7.5 +org.apache.curator:curator-framework:2.7.1 +org.apache.curator:curator-test:2.7.1 + + +BSD License +------------ + +org.fusesource.leveldbjni:leveldbjni-all:1.8 +com.github.luben:zstd-jni:1.4.4-3 + + +MIT License +----------- + +org.mockito:mockito-core:1.10.19 + + +GPL License +----------- + +javax.servlet:javax.servlet-api:3.1.0 diff --git a/README.md b/README.md new file mode 100644 index 0000000..34b9606 --- /dev/null +++ b/README.md @@ -0,0 +1,183 @@ +English | **[简体中文](README_zh.md)** +# Cloud Shuffle Service + +[![GitHub license](https://img.shields.io/badge/license-Apache%202-blue)](https://github.com/bytedance/ByteX/blob/master/LICENSE) + +Cloud Shuffle Service(CSS) is a general purpose remote shuffle solution for compute engines, including Spark/Flink/MapReduce. +It provides reliable, high-performance, and elastic data shuffling capabilities to these compute engines. +Shuffled data is pushed to the CSS cluster, stored in disk or HDFS, and can be fetched from the CSS cluster by compute engines. + +![CSS Architecture](docs/img/css-arch.png) +- CSS Worker + + Stores shuffled data that is pushed from map tasks in memory and persist them to file system asynchronously, allowing reduce tasks to subsequently fetch data from CSS workers. + +- CSS Master + + CSS Master is a coordinator component for an application's shuffle process, and is integrated into the application. It reads CSS worker list from ZooKeeper and assigns them to the application to do shuffling, tracks the progress of running map tasks, and then notifies CSS workers to commit files after all the map tasks have finished. + +- CSS Client + + Map/Reduce task use CSS client to push/fetch shuffled data to the assigned CSS workers. + +## Building CSS + +### mvn build + +CSS is built using [Apache Maven](https://maven.apache.org/). Building CSS using Maven requires Java 8 and either Scala 2.12 or Scala 2.11. + +```bash +mvn -DskipTests clean package +``` +### Building a Runnable Distribution +To create a CSS distribution, use ./build.sh in the project root directory. +```bash +./build.sh +``` +It generates a tgz package, you can copy it to the nodes you want to deploy CSS. +``` +css-1.0.0-bin +├── LICENSE +├── README.md +├── client +├── conf +├── docs +├── lib // CSS cluster lib +└── sbin +``` + +## Deploy CSS Cluster +CSS provides two deployment modes, standalone and zookeeper mode. +The standalone mode is currently only for testing, +while the zookeeper mode is used in the production environment. + +1. Place the above built CSS tgz file on each node of the Cluster. +2. Unpack it to a dir, which can be set to CSS_HOME environment, all default conf, metrics and workers list can be found in the ```$CSS_HOME/conf``` directory. +3. Update ```$CSS_HOME/sbin/css-config.sh```. + ``` + # standalone mode + CSS_MASTER_HOST= + MASTER_JAVA_OPTS="-Xmx8192m" + WORKER_JAVA_OPTS="-Xmx8192m -XX:MaxDirectMemorySize=100000m" + + # zookeeper mode + WORKER_JAVA_OPTS="-Xmx8192m -XX:MaxDirectMemorySize=100000m" + ``` +4. Update ```$CSS_HOME/conf/css-defaults.conf``` + ``` + css.cluster.name = + + # standalone(for testing) or zookeeper(for production) + css.worker.registry.type = zookeeper + # only for zookeeper mode + css.zookeeper.address = :,:,: + + # css worker common conf + css.flush.queue.capacity = 4096 + css.flush.buffer.size = 128k + css.network.timeout = 600s + css.epoch.rotate.threshold = 1g + css.push.io.numConnectionsPerPeer = 8 + css.push.io.threads = 128 + css.replicate.threads = 128 + css.fetch.io.threads = 64 + css.fetch.chunk.size = 4m + css.shuffle.server.chunkFetchHandlerThreadsPercent = 400 + + # hdfs storage + css.hdfsFlusher.base.dir = hdfs://xxx + css.hdfsFlusher.num = -1 + css.hdfsFlusher.replica = 2 + + # local disk storage + css.diskFlusher.base.dirs = /data00/css,/data01/css + css.disk.dir.num.min = 1 + ``` +5. Define your metrics and worker node host in the following files: + ``` + $CSS_HOME/conf/css-metrics.properties + $CSS_HOME/conf/workers + ``` +6. Sync all the updated config files to each node of the Cluster. +7. Start the CSS Cluster Shuffle workers. The script will ssh into each css worker node and start the Workers. + ``` + # standalone mode + cd $CSS_HOME;bash ./sbin/start-all.sh + # zookeeper mode + cd $CSS_HOME;bash ./sbin/start-workers.sh + ``` + +## Running with Spark +1. Copy ```$CSS_HOME/client/spark-${version}/*.jar``` to ```$SPARK_HOME/jars/``` . +2. Run spark with CSS. + ``` + # standalone mode + --conf spark.css.cluster.name= \ + --conf spark.css.master.address=css://:\ + --conf spark.shuffle.manager=org.apache.spark.shuffle.css.CssShuffleManager\ + + # zookeeper mode + --conf spark.css.cluster.name= \ + --conf spark.css.zookeeper.address=":,:,:" \ + --conf spark.shuffle.manager=org.apache.spark.shuffle.css.CssShuffleManager\ + ``` + +## Spark Adaptive Query Execution Support +CSS supports all the features of AQE. To support skew join optimization, it is necessary to patch the file to Spark and re-build Spark. +``` +./patch/spark-3.0-aqe-skewjoin.patch +``` + +## Configuration +### CSS Cluster Server +All detailed configuration can be found in the CssConf class. + +| Property Name | Default | Meaning | +|---------------------------|------------|-----------------------------------------------------------------------------------------------------------------------------------| +| css.cluster.name | - | The cluster name for the CSS cluster. | +| css.worker.registry.type | standalone | The worker registry type (e.g. standalone, zookeeper). This will also specify if CSS will run under Standalone or zookeeper mode. | +| css.zookeeper.address | - | (For zookeeper mode) The CSS zookeeper address. | +| css.push.io.threads | 32 | The CSS Threads for netty push data io. | +| css.fetch.io.threads | 32 | The CSS Threads for netty fetch data io. | +| css.commit.threads | 128 | The CSS Threads for stage end to close partition file. | +| css.diskFlusher.base.dirs | /tmp/css | The CSS Disk Base dirs (e.g. /data00/css,/data01/css). | +| css.hdfsFlusher.base.dir | - | The CSS HDFS Base dir (e.g. hdfs://xxx). | + +### CSS Spark Client + +| Property Name | Default | Meaning | +|-------------------------------------------|---------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| css.max.allocate.worker | 1000 | The Maximum number of workers requested for shuffling. | +| css.worker.allocate.extraRatio | 1.5 | The application can allocate additional workers controlled by this extra ratio, the final number will be calculated with Min(Max(2, targetWorker), MaxAllocateWorker). | +| css.backpressure.enabled | true | The back pressure control, when enabled, it will use Gradient2Limit to control push data rate, otherwise use FixedLimit. | +| css.fixRateLimit.threshold | 64 | Fixed Rate for the back pressure control. | +| css.data.io.threads | 8 | The Maximum client side data sending for netty thread. | +| css.maxPartitionsPerGroup | 100 | The Maximum number of partitions per group, each data push will send one group at a time. | +| css.partitionGroup.push.buffer.size | 4m | The Maximum buffer size sent per each data push, in the same format as JVM memory strings with a size unit suffix ("k", "m", "g" or "t") (e.g. 512m, 2g). | +| css.client.mapper.end.timeout | 600s | The Maximum timeout to wait for all data to be sent before mapTask ends. | +| css.stage.end.timeout | 600s | The Maximum timeout to wait for all partition files to close. | +| css.sortPush.spill.record.threshold | 1000000 | The Maximum records for sending data. | +| css.sortPush.spill.size.threshold | 256m | The Maximum size for sending data, in the same format as JVM memory strings with a size unit suffix ("k", "m", "g" or "t") (e.g. 512m, 2g). | +| css.shuffle.mode | DISK | Choose which storage mode to use (e.g. DISK, HDFS). | +| css.epoch.rotate.threshold | 1g | The file auto rotate switch threshold size for new files, in the same format as JVM memory strings with a size unit suffix ("k", "m", "g" or "t") (e.g. 512m, 2g). | +| css.client.failed.batch.blacklist.enabled | true | When MapTask encounters onFailure, the current reduceId-epochId-mapId-mapAttemptId-batchId will be recorded into the blacklist. In AE skewjoin mode, this switch must be turned on, otherwise there will be correctness problems. | +| css.compression.codec | lz4 | It is recommended to use zstd compression mode. Compared with lz4, it can improve the compression ratio by 30%, and only consume an additional 8% of performance. | + +## Contribution + +Please check [Contributing](CONTRIBUTING.md) for more details. + +## Code of Conduct + +Please check [Code of Conduct](CODE_OF_CONDUCT.md) for more details. + +## Security + +If you discover a potential security issue in this project, or think you may +have discovered a security issue, we ask that you notify Bytedance Security via our [security center](https://security.bytedance.com/src) or [vulnerability reporting email](mailto:sec@bytedance.com). + +Please do **not** create a public GitHub issue. + +## License + +This project is licensed under the [Apache-2.0 License](LICENSE). \ No newline at end of file diff --git a/README_zh.md b/README_zh.md new file mode 100644 index 0000000..024e310 --- /dev/null +++ b/README_zh.md @@ -0,0 +1,185 @@ +**[English](README.md)** | 简体中文 + +# Cloud Shuffle Service + +[![GitHub license](https://img.shields.io/badge/license-Apache%202-blue)](https://github.com/bytedance/ByteX/blob/master/LICENSE) + +Cloud Shuffle Service(CSS) 是一个通用的远程shuffle解决方案。其支持当前主流计算引擎,如Spark/Flink/MapReduce等, +并为这些主流的计算框架提供相比原生方案更可靠的、性能更高的、更弹性的数据shuffle能力。 +计算框架将shuffle数据推送到CSS集群并存放在磁盘或者HDFS中, +之后当需要读取数据时,再通过CSS集群的接口拉取相关的shuffle数据。 + + +![CSS Architecture](docs/img/css-arch.png) +- CSS Worker + + 负责把来自Map Task发送的数据存储到内存中并最终异步的把数据刷写到文件系统中。当Reduce Task开始时可以从Worker拉取之前存储的数据。 + +- CSS Master + + 负责协调application的整个shuffle过程,需要被整合进application的运行过程中,例如跑在Spark的driver中。 + CSS Master会从ZooKeeper中获取worker的列表并且分配合适的worker给application提供shuffle服务,并且跟进所有Map Task完成的进度, + 当Map Task运行完成的时候会通知所有worker把所有缓存中的数据持久化到磁盘并关闭文件。 + +- CSS Client + + 当运行Map或者reduce task时,负责从所有worker推送或者拉取shuffle数据 + +## 编译 CSS + +### mvn build + +CSS 使用 [Apache Maven](https://maven.apache.org/) 作为编译工具。使用Maven编译CSS需要使用Java 8,和Scala 2.12或Scala 2.11 + +```bash +mvn -DskipTests clean package +``` +### 编译可执行文件 +使用项目根目录下的 ```./build.sh``` 来编译得到可执行文件 +```bash +./build.sh +``` +执行之后会生成一个tgz包,可以将其拷贝到想要部署 CSS 服务的节点上 +``` +css-1.0.0-bin +├── LICENSE +├── README.md +├── client +├── conf +├── docs +├── lib // CSS cluster lib +└── sbin +``` + +## 部署 CSS 集群 +CSS 提供两种部署模式,分别是独立部署模式和Zookeeper模式。其中独立部署模式当前仅支持测试场景,在生产场景下建议使用zookeeper模式。 + +1. 把按照之前步骤编译好的tgz包发送到集群中的每个节点上 +2. 解压该tgz包到某个目录,并设置该目录路径作为环境变量CSS_HOME的值,集群默认配置文件、metrics配置文件、worker列表文件都放在```$CSS_HOME/conf```目录下 +3. 修改 ```$CSS_HOME/sbin/css-config.sh``` + ``` + # standalone mode + CSS_MASTER_HOST= + MASTER_JAVA_OPTS="-Xmx8192m" + WORKER_JAVA_OPTS="-Xmx8192m -XX:MaxDirectMemorySize=100000m" + + # zookeeper mode + WORKER_JAVA_OPTS="-Xmx8192m -XX:MaxDirectMemorySize=100000m" + ``` +4. 修改 ```$CSS_HOME/conf/css-defaults.conf``` + ``` + css.cluster.name = + + # standalone(for testing) or zookeeper(for production) + css.worker.registry.type = zookeeper + # only for zookeeper mode + css.zookeeper.address = :,:,: + + # css worker common conf + css.flush.queue.capacity = 4096 + css.flush.buffer.size = 128k + css.network.timeout = 600s + css.epoch.rotate.threshold = 1g + css.push.io.numConnectionsPerPeer = 8 + css.push.io.threads = 128 + css.replicate.threads = 128 + css.fetch.io.threads = 64 + css.fetch.chunk.size = 4m + css.shuffle.server.chunkFetchHandlerThreadsPercent = 400 + + # hdfs storage + css.hdfsFlusher.base.dir = hdfs://xxx + css.hdfsFlusher.num = -1 + css.hdfsFlusher.replica = 2 + + # local disk storage + css.diskFlusher.base.dirs = /data00/css,/data01/css + css.disk.dir.num.min = 1 + ``` +5. 自定义 metrics 配置文件和 worker 列表文件 + ``` + $CSS_HOME/conf/css-metrics.properties + $CSS_HOME/conf/workers + ``` +6. 把上述所有配置文件发送到集群中所有的节点上 +7. 执行如下命令启动所有的Css Workers,所有的worker所在的节点需要该执行该命令的机器能够通过SSH正常连接 + ``` + # standalone mode + cd $CSS_HOME;bash ./sbin/start-all.sh + # zookeeper mode + cd $CSS_HOME;bash ./sbin/start-workers.sh + ``` + +## Spark使用CSS +1. 拷贝 ```$CSS_HOME/client/spark-${version}/*.jar``` 到 ```$SPARK_HOME/jars/``` +2. 启动 spark 时添加如下参数 + ``` + # standalone mode + --conf spark.css.cluster.name= \ + --conf spark.css.master.address=css://:\ + --conf spark.shuffle.manager=org.apache.spark.shuffle.css.CssShuffleManager\ + + # zookeeper mode + --conf spark.css.cluster.name= \ + --conf spark.css.zookeeper.address=":,:,:" \ + --conf spark.shuffle.manager=org.apache.spark.shuffle.css.CssShuffleManager\ + ``` + +## 支持 Spark Adaptive Query Execution +CSS 支持 Spark AQE 所有特性。对于SkewJoin的支持,需要使用以下文件更新Spark的源码并重新编译Spark。 +``` +./patch/spark-3.0-aqe-skewjoin.patch +``` + +## 配置项 +### CSS 服务端配置 +All detailed configuration can be found in CssConf class. + +| Property Name | Default | Meaning | +|---------------------------|------------|-----------------------------------------------------------------------------------------------------------------------------------| +| css.cluster.name | - | The cluster name for the CSS cluster. | +| css.worker.registry.type | standalone | The worker registry type (e.g. standalone, zookeeper). This will also specify if CSS will run under Standalone or zookeeper mode. | +| css.zookeeper.address | - | (For zookeeper mode) The CSS zookeeper address. | +| css.push.io.threads | 32 | The CSS Threads for netty push data io. | +| css.fetch.io.threads | 32 | The CSS Threads for netty fetch data io. | +| css.commit.threads | 128 | The CSS Threads for stage end to close partition file. | +| css.diskFlusher.base.dirs | /tmp/css | The CSS Disk Base dirs (e.g. /data00/css,/data01/css). | +| css.hdfsFlusher.base.dir | - | The CSS HDFS Base dir (e.g. hdfs://xxx). | + +### CSS 客户端配置 + +| Property Name | Default | Meaning | +|-------------------------------------------|---------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| css.max.allocate.worker | 1000 | The Maximum number of workers requested for shuffling. | +| css.worker.allocate.extraRatio | 1.5 | The application can allocate additional workers controlled by this extra ratio, the final number will be calculated with Min(Max(2, targetWorker), MaxAllocateWorker). | +| css.backpressure.enabled | true | The back pressure control, when enabled, it will use Gradient2Limit to control push data rate, otherwise use FixedLimit. | +| css.fixRateLimit.threshold | 64 | Fixed Rate for the back pressure control. | +| css.data.io.threads | 8 | The Maximum client side data sending for netty thread. | +| css.maxPartitionsPerGroup | 100 | The Maximum number of partitions per group, each data push will send one group at a time. | +| css.partitionGroup.push.buffer.size | 4m | The Maximum buffer size sent per each data push, in the same format as JVM memory strings with a size unit suffix ("k", "m", "g" or "t") (e.g. 512m, 2g). | +| css.client.mapper.end.timeout | 600s | The Maximum timeout to wait for all data to be sent before mapTask ends. | +| css.stage.end.timeout | 600s | The Maximum timeout to wait for all partition files to close. | +| css.sortPush.spill.record.threshold | 1000000 | The Maximum records for sending data. | +| css.sortPush.spill.size.threshold | 256m | The Maximum size for sending data, in the same format as JVM memory strings with a size unit suffix ("k", "m", "g" or "t") (e.g. 512m, 2g). | +| css.shuffle.mode | DISK | Choose which storage mode to use (e.g. DISK, HDFS). | +| css.epoch.rotate.threshold | 1g | The file auto rotate switch threshold size for new files, in the same format as JVM memory strings with a size unit suffix ("k", "m", "g" or "t") (e.g. 512m, 2g). | +| css.client.failed.batch.blacklist.enabled | true | When MapTask encounters onFailure, the current reduceId-epochId-mapId-mapAttemptId-batchId will be recorded into the blacklist. In AE skewjoin mode, this switch must be turned on, otherwise there will be correctness problems. | +| css.compression.codec | lz4 | It is recommended to use zstd compression mode. Compared with lz4, it can improve the compression ratio by 30%, and only consume an additional 8% of performance. | + +## 贡献协议 + +请点击[Contributing](CONTRIBUTING.md)查看更多细节. + +## 行为准则 + +请点击[Code of Conduct](CODE_OF_CONDUCT.md)查看更多细节. + +## 安全漏洞 + +如果你在此项目中发现了一个潜在的安全问题,请联系[字节跳动安全中心](https://security.bytedance.com/src) 或发送邮件到[漏洞汇报](mailto:sec@bytedance.com). + +请**不要**创建公开的Github issue. + +## 开源协议 + +本项目采用[Apache-2.0 License](LICENSE)协议. \ No newline at end of file diff --git a/api/pom.xml b/api/pom.xml new file mode 100644 index 0000000..ef425d6 --- /dev/null +++ b/api/pom.xml @@ -0,0 +1,69 @@ + + + + 4.0.0 + + + com.bytedance.inf + css-root + 1.0.0 + ../pom.xml + + + css-api + jar + Cloud Shuffle Service API + + + + log4j + log4j + provided + + + + org.slf4j + slf4j-api + provided + + + + org.slf4j + slf4j-log4j12 + provided + + + + org.slf4j + jul-to-slf4j + provided + + + + org.slf4j + jcl-over-slf4j + provided + + + + + + + + diff --git a/api/src/main/java/com/bytedance/css/api/CssShuffleContext.java b/api/src/main/java/com/bytedance/css/api/CssShuffleContext.java new file mode 100644 index 0000000..c8cc6d9 --- /dev/null +++ b/api/src/main/java/com/bytedance/css/api/CssShuffleContext.java @@ -0,0 +1,112 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.api; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.reflect.Constructor; +import java.util.Map; + +public abstract class CssShuffleContext { + + private static final String CONTEXT_DEFAULT_IMPL = "com.bytedance.css.impl.CssShuffleContextImpl"; + private static final Logger logger = LoggerFactory.getLogger(CssShuffleContext.class); + + private static CssShuffleContext context = null; + + protected CssShuffleContext() { + } + + public static CssShuffleContext get(String implClass) { + if (context == null) { + synchronized (CssShuffleContext.class) { + if (context == null) { + try { + Class clz = Class.forName(implClass); + Constructor constructor = + (Constructor) clz.getDeclaredConstructor(new Class[]{}); + constructor.setAccessible(true); + context = constructor.newInstance(); + } catch (Exception ex) { + logger.error(String.format("CssShuffleContext reflect initialization with %s failed", implClass), ex); + } + } + } + } + return context; + } + + public static CssShuffleContext get() { + return get(CONTEXT_DEFAULT_IMPL); + } + + /** + * try to start css master + * + * @param host + * @param port + * @param confMap + */ + public abstract void startMaster(String host, int port, Map confMap) throws Exception; + + /** + * try to stop css master + */ + public abstract void stopMaster() throws Exception; + + /** + * return css master host, must be called after startMaster + * + * @return + */ + public abstract String getMasterHost() throws Exception; + + /** + * return css master port, must be called after startMaster + * + * @return + */ + public abstract int getMasterPort() throws Exception; + + /** + * initialize with proper worker nums if needed, must be called after startMaster + * + * @param numWorkers + */ + public abstract void allocateWorkerIfNeeded(int numWorkers) throws Exception; + + /** + * sync wait until shuffle stageEnd, and reduce task is ready to be scheduled + * + * @param appId + * @param shuffleId + */ + public abstract void waitUntilShuffleCommitted(String appId, int shuffleId) throws Exception; + + /** + * destroy and clean up entire shuffle meta and data eagerly, mainly used for shuffle re-computation + * + * @param appId + * @param shuffleId + * @throws Exception + */ + public abstract void eagerDestroyShuffle(String appId, int shuffleId) throws Exception; +} diff --git a/build.sh b/build.sh new file mode 100755 index 0000000..dae3ce9 --- /dev/null +++ b/build.sh @@ -0,0 +1,89 @@ +#!/usr/bin/env bash + +# +# Copyright 2022 Bytedance Inc. +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +set -e +set -x + +dst=$PWD/output + +rm -rf ${dst} + +latest_commit=$(git log -1 --pretty=oneline | cut -d ' ' -f 1) + +if [ -z "${MAVEN_HOME}" ]; then + echo "maven_home not set..." +fi + +export MAVEN_OPTS="-Xmx8g -XX:ReservedCodeCacheSize=2g" + +VERSION=$(mvn help:evaluate -Dexpression=project.version -q -DforceStdout $@) +echo "CSS version is $VERSION" + +mkdir ${dst} +# Add css jars +mkdir ${dst}/lib + +# Build and generate output for spark 3.0 +# server also use scala version 2.12 +mvn clean package -DskipTests $@ +cp service/target/css-service_2.12-*-shaded.jar ${dst}/lib + +# Generate bin +mkdir ${dst}/sbin +cp -r sbin/* ${dst}/sbin + +# Generate conf +mkdir ${dst}/conf +cp -r conf/* ${dst}/conf + +# Generate client +mkdir ${dst}/client + +# Build spark shuffle client +mkdir ${dst}/client/spark-3 +pushd ./css-assembly_3/target/libs +cp -r *.jar ${dst}/client/spark-3 +popd + +mvn clean package -Pscala-11 -DskipTests $@ +mkdir ${dst}/client/spark-2 +pushd ./css-assembly_2/target/libs +cp -r *.jar ${dst}/client/spark-2 +popd + +# cp mr assembly to dst + +# build flink shuffle jar + +# cp doc to target output dir +cp -r ./docs ${dst} +cp -r ./README.md ${dst} +cp -r ./LICENSE ${dst} + +# Copy commons and examples folder to output +rm -f $PWD/css-*.tgz + +TARDIR_NAME="css-$VERSION-bin" +rm -rf ./$TARDIR_NAME +cp -r output $TARDIR_NAME +tar czf "$TARDIR_NAME.tgz" $TARDIR_NAME +rm -rf ./$TARDIR_NAME +cd ../ diff --git a/client/pom.xml b/client/pom.xml new file mode 100644 index 0000000..8fab8a1 --- /dev/null +++ b/client/pom.xml @@ -0,0 +1,135 @@ + + + + 4.0.0 + + + com.bytedance.inf + css-root + 1.0.0 + ../pom.xml + + + css-client_${scala.binary.version} + jar + Cloud Shuffle Service Client + + + + com.bytedance.inf + css-common_${scala.binary.version} + ${project.version} + + + + org.apache.hadoop + hadoop-client + provided + + + + com.github.luben + zstd-jni + + + + org.lz4 + lz4-java + + + + junit + junit + test + + + + org.apache.commons + commons-lang3 + test + + + + com.netflix.concurrency-limits + concurrency-limits-core + + + + + + + net.alchim31.maven + scala-maven-plugin + + + org.apache.maven.plugins + maven-shade-plugin + 2.4.3 + + + package + + shade + + + false + true + + + + com.github.luben:zstd-jni + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + **/log4j.properties + + + + + + reference.conf + + + + META-INF/services/org.apache.spark.sql.sources.DataSourceRegister + + + + + META-INF/services/org.apache.hadoop.fs.FileSystem + + + + + + + + + + diff --git a/client/src/main/java/com/bytedance/css/client/MapperAttemptStatus.java b/client/src/main/java/com/bytedance/css/client/MapperAttemptStatus.java new file mode 100644 index 0000000..5eecb8c --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/MapperAttemptStatus.java @@ -0,0 +1,56 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client; + +import com.bytedance.css.common.protocol.FailedPartitionInfoBatch; +import com.bytedance.css.common.protocol.PartitionInfo; +import io.netty.util.internal.ConcurrentSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +public class MapperAttemptStatus { + + private static final Logger logger = LoggerFactory.getLogger(MapperAttemptStatus.class); + + public final AtomicInteger InFlightReqs = new AtomicInteger(0); + private final AtomicInteger batchId = new AtomicInteger(0); + private AtomicReference exception = new AtomicReference<>(); + public final ConcurrentSet writtenEpochSet = new ConcurrentSet<>(); + public final ConcurrentSet failedBatchBlacklist = new ConcurrentSet<>(); + + public MapperAttemptStatus() {} + + public int getNextBatchId() { + return batchId.incrementAndGet(); + } + + public IOException getException() { + return exception.get(); + } + + public void setException(IOException exception) { + // setException should be called once + this.exception.compareAndSet(null, exception); + } +} diff --git a/client/src/main/java/com/bytedance/css/client/MetricsCallback.java b/client/src/main/java/com/bytedance/css/client/MetricsCallback.java new file mode 100644 index 0000000..4fd3771 --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/MetricsCallback.java @@ -0,0 +1,27 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client; + +public interface MetricsCallback { + + void incBytesRead(long bytesRead); + + void incReadTime(long time); +} diff --git a/client/src/main/java/com/bytedance/css/client/ShuffleClient.java b/client/src/main/java/com/bytedance/css/client/ShuffleClient.java new file mode 100644 index 0000000..2edfb93 --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/ShuffleClient.java @@ -0,0 +1,189 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client; + +import com.bytedance.css.client.impl.ShuffleClientImpl; +import com.bytedance.css.client.stream.CssInputStream; +import com.bytedance.css.client.stream.disk.CssRemoteDiskEpochReader; +import com.bytedance.css.common.CssConf; +import com.bytedance.css.common.protocol.CommittedPartitionInfo; +import com.bytedance.css.common.protocol.PartitionGroup; +import com.bytedance.css.common.protocol.PartitionGroupManager; +import com.bytedance.css.network.client.TransportClientFactory; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; + +public abstract class ShuffleClient implements Cloneable { + + protected static volatile ShuffleClient shuffleClient; + + public static ShuffleClient get(CssConf cssConf) { + if (shuffleClient == null) { + synchronized (ShuffleClient.class) { + if (shuffleClient == null) { + shuffleClient = new ShuffleClientImpl(cssConf); + } + } + } + return shuffleClient; + } + + // key: shuffleId + // value: PartitionGroupManager(Map) + public static final ConcurrentHashMap + shufflePartitionGroupMap = new ConcurrentHashMap<>(); + + /** + * Send PushData to CSS Worker. + * + * In this way, Mapper calls and sends the RegisterShuffle event if needed through this interface. + * and the CSS Master does the lock processing + * + * pushData is responsible for registerShuffle (other engines) so need numMappers numPartitions. + * Return how many bytes were sent, this interface should be asynchronous. + */ + public abstract int[] batchPushData( + String applicationId, + int shuffleId, + int mapperId, + int mapperAttemptId, + int[] reducerIdArray, + byte[] data, + int[] offsetArray, + int[] lengthArray, + int numMappers, + int numPartitions, + boolean skipCompress) throws IOException; + + /** + * Send MapperEnd event to CSS Master. + * + * If it is an empty data shuffle, then there may be no RegisterShuffle at all. + * so the bottom line is, our MapperEnd also needs to send numMappers. + * this way we can finally count until StageEnd is triggered. + */ + public abstract void mapperEnd( + String applicationId, + int shuffleId, + int mapperId, + int mapperAttemptId, + int numMappers) throws IOException; + + /** + * Mapper ClientSide State cleanup + */ + public abstract void mapperClose( + String applicationId, + int shuffleId, + int mapperId, + int mapperAttemptId); + + /** + * Send GetReducerFileGroups event to CSS Master. + * + * only query partition info, do not fetch data + */ + public abstract List getPartitionInfos( + String applicationId, + int shuffleId, + int[] reduceIds, + int startMapIndex, + int endMapIndex) throws IOException; + + /** + * use client's client factory to create epoch reader. used for other engines + */ + public abstract CssRemoteDiskEpochReader createEpochReader( + String applicationId, int shuffleId, + List partitions, + CssConf conf) throws IOException; + + /** + * read all partition data as an input stream + * + * if startMapIndex > endMapIndex, means current PartitionSpec is specially handled BY CSS. + * It's mainly used for SkewJoin for AdaptiveExecution. + * In this case, startMapIndex and endMapIndex have different meanings. + * startMapIndex -> Total SkewPartition task num + * endMapIndex -> SkewPartition task index start from 0 + */ + public abstract CssInputStream readPartitions( + String applicationId, + int shuffleId, + int[] reduceIds, + int startMapIndex, + int endMapIndex) throws IOException; + + /** + * After call getPartitionInfos method. the client had mapper attempts in memory. + */ + public abstract int[] getMapperAttempts(int shuffleId); + + /** + * Register shuffle with retry. + * + * Register shuffle to the master, the master will return the worker pair information + * which allocated by the current shuffle. + * this interface is suitable for different engines + */ + public abstract List registerPartitionGroup( + String applicationId, + int shuffleId, + int numMappers, + int numPartitions, + int maxPartitionsPerGroup) throws IOException; + + /** + * after register shuffle successful, put allocated partitions into global reducerPartitionMap + */ + public void applyShufflePartitionGroup(int shuffleId, List partitionGroups) { + if (!shufflePartitionGroupMap.containsKey(shuffleId)) { + shufflePartitionGroupMap.put(shuffleId, new PartitionGroupManager(partitionGroups)); + } + } + + // For Test Only + public static void cleanShuffle(int shuffleId) { + shufflePartitionGroupMap.remove(shuffleId); + } + + /** + * ShuffleManager unregister shuffle API will be called among Driver and Executors. + * Only Driver should respond to send UnregisterShuffle Event to CSS + */ + public abstract void unregisterShuffle( + String applicationId, + int shuffleId, + boolean isDriver); + + /** + * Enable application reports its status to CSS Master + */ + public abstract void registerApplication(String applicationId); + + /** + * Called by ShuffleManager stop to clean up entire ShuffleClient + */ + public abstract void shutDown(); + + public abstract TransportClientFactory getClientFactory(); +} diff --git a/client/src/main/java/com/bytedance/css/client/compress/Compressor.java b/client/src/main/java/com/bytedance/css/client/compress/Compressor.java new file mode 100644 index 0000000..3dafdfd --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/compress/Compressor.java @@ -0,0 +1,29 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.compress; + +public interface Compressor extends CssCompressorTrait { + + void compress(byte[] data, int offset, int length); + + int getCompressedTotalSize(); + + byte[] getCompressedBuffer(); +} diff --git a/client/src/main/java/com/bytedance/css/client/compress/CompressorFactory.java b/client/src/main/java/com/bytedance/css/client/compress/CompressorFactory.java new file mode 100644 index 0000000..252c306 --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/compress/CompressorFactory.java @@ -0,0 +1,27 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.compress; + +public interface CompressorFactory { + + Compressor getCompressor(); + + Decompressor getDecompressor(); +} diff --git a/client/src/main/java/com/bytedance/css/client/compress/CssCompressorFactory.java b/client/src/main/java/com/bytedance/css/client/compress/CssCompressorFactory.java new file mode 100644 index 0000000..73c8be8 --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/compress/CssCompressorFactory.java @@ -0,0 +1,82 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.compress; + +import com.bytedance.css.common.CssConf; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CssCompressorFactory implements CompressorFactory { + + private static final Logger logger = LoggerFactory.getLogger(CompressorFactory.class); + + private final CssConf cssConf; + private final CompressType compressType; + + public CssCompressorFactory(CssConf cssConf) { + this.cssConf = cssConf; + this.compressType = CompressType.valueOf(CssConf.compressionCodecType(cssConf)); + logger.debug("use compressor type {}", this.compressType); + + /** + * Zstd dict mode. but we don't want to support this. + * because dict can not satisfy all data distribution. + * + * {@link com.bytedance.css.client.compress.ZstdDictTrainer} + */ + } + + @Override + public Compressor getCompressor() { + switch (compressType) { + case lz4: + return new Lz4Compressor(cssConf); + case zstd: + return new ZstdCompressor(cssConf); + default: + throw new IllegalArgumentException(String.format("not support compress type %s", compressType)); + } + } + + @Override + public Decompressor getDecompressor() { + switch (compressType) { + case lz4: + return new Lz4Decompressor(); + case zstd: + return new ZstdDecompressor(); + default: + throw new IllegalArgumentException(String.format("not support decompress type %s", compressType)); + } + } + + private enum CompressType { + lz4("lz4"), + zstd("zstd"), + zstd_dict("zstd-dict"); + + CompressType(String type) { + this.type = type; + } + + public String type; + } +} diff --git a/client/src/main/java/com/bytedance/css/client/compress/CssCompressorTrait.java b/client/src/main/java/com/bytedance/css/client/compress/CssCompressorTrait.java new file mode 100644 index 0000000..4856b53 --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/compress/CssCompressorTrait.java @@ -0,0 +1,40 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.compress; + +public interface CssCompressorTrait { + + byte[] MAGIC = new byte[] { 'C', 's', 's', 'B', 'l', 'o', 'c', 'k' }; + int MAGIC_LENGTH = MAGIC.length; + + int HEADER_LENGTH = + MAGIC_LENGTH // magic bytes + + 1 // token + + 4 // compressed length + + 4 // decompressed length + + 4; // checksum + + int COMPRESSION_LEVEL_BASE = 10; + + int COMPRESSION_METHOD_RAW = 0x10; + int COMPRESSION_METHOD_CSS = 0x20; + + int DEFAULT_SEED = 0x9747b28c; +} diff --git a/client/src/main/java/com/bytedance/css/client/compress/Decompressor.java b/client/src/main/java/com/bytedance/css/client/compress/Decompressor.java new file mode 100644 index 0000000..f4b40e8 --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/compress/Decompressor.java @@ -0,0 +1,27 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.compress; + +public interface Decompressor extends CssCompressorTrait { + + int getOriginalLen(byte[] src); + + int decompress(byte[] src, byte[] dst, int dstOff); +} diff --git a/client/src/main/java/com/bytedance/css/client/compress/Lz4Compressor.java b/client/src/main/java/com/bytedance/css/client/compress/Lz4Compressor.java new file mode 100644 index 0000000..656acd8 --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/compress/Lz4Compressor.java @@ -0,0 +1,104 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.compress; + +import com.bytedance.css.common.CssConf; +import net.jpountz.lz4.LZ4Compressor; +import net.jpountz.lz4.LZ4Factory; +import net.jpountz.xxhash.XXHashFactory; + +import java.util.zip.Checksum; + +public class Lz4Compressor implements Compressor { + + private final int compressionLevel; + private final LZ4Compressor compressor; + private final Checksum checksum; + + // only used for test + private boolean testMode; + private byte[] compressedBuffer; + private int compressedTotalSize; + + public Lz4Compressor() { + this(new CssConf()); + } + + public Lz4Compressor(CssConf cssConf) { + int blockSize = (int) CssConf.pushBufferSize(cssConf); + this.testMode = CssConf.compressionTestMode(cssConf); + int level = 32 - Integer.numberOfLeadingZeros(blockSize - 1) - COMPRESSION_LEVEL_BASE; + this.compressionLevel = Math.max(0, level); + this.compressor = LZ4Factory.fastestInstance().fastCompressor(); + checksum = XXHashFactory.fastestInstance().newStreamingHash32(DEFAULT_SEED).asChecksum(); + initCompressBuffer(blockSize); + } + + private void initCompressBuffer(int size) { + int compressedBlockSize = HEADER_LENGTH + compressor.maxCompressedLength(size); + compressedBuffer = new byte[compressedBlockSize]; + System.arraycopy(MAGIC, 0, compressedBuffer, 0, MAGIC_LENGTH); + } + + @Override + public void compress(byte[] data, int offset, int length) { + checksum.reset(); + checksum.update(data, offset, length); + final int check = (int) checksum.getValue(); + // lz4 maxCompressedLength is equal (length + length / 255 + 16) + // so len check must subtract extra header & placeholder len + if (compressedBuffer.length - HEADER_LENGTH - (length / 255 + 16) < length) { + initCompressBuffer(length); + } + int compressedLength = compressor.compress(data, offset, length, compressedBuffer, HEADER_LENGTH); + final int compressMethod; + if (compressedLength >= length || testMode) { + compressMethod = COMPRESSION_METHOD_RAW; + compressedLength = length; + System.arraycopy(data, offset, compressedBuffer, HEADER_LENGTH, length); + } else { + compressMethod = COMPRESSION_METHOD_CSS; + } + + compressedBuffer[MAGIC_LENGTH] = (byte) (compressMethod | compressionLevel); + writeIntLE(compressedLength, compressedBuffer, MAGIC_LENGTH + 1); + writeIntLE(length, compressedBuffer, MAGIC_LENGTH + 5); + writeIntLE(check, compressedBuffer, MAGIC_LENGTH + 9); + + compressedTotalSize = HEADER_LENGTH + compressedLength; + } + + @Override + public int getCompressedTotalSize() { + return compressedTotalSize; + } + + @Override + public byte[] getCompressedBuffer() { + return compressedBuffer; + } + + private static void writeIntLE(int i, byte[] buf, int off) { + buf[off++] = (byte) i; + buf[off++] = (byte) (i >>> 8); + buf[off++] = (byte) (i >>> 16); + buf[off++] = (byte) (i >>> 24); + } +} diff --git a/client/src/main/java/com/bytedance/css/client/compress/Lz4Decompressor.java b/client/src/main/java/com/bytedance/css/client/compress/Lz4Decompressor.java new file mode 100644 index 0000000..ea86925 --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/compress/Lz4Decompressor.java @@ -0,0 +1,81 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.compress; + +import net.jpountz.lz4.LZ4Factory; +import net.jpountz.lz4.LZ4FastDecompressor; +import net.jpountz.xxhash.XXHashFactory; +import org.apache.log4j.Logger; + +import java.util.zip.Checksum; + +public class Lz4Decompressor implements Decompressor { + + private static final Logger logger = Logger.getLogger(Lz4Decompressor.class); + + private final LZ4FastDecompressor decompressor; + private final Checksum checksum; + + public Lz4Decompressor() { + decompressor = LZ4Factory.fastestInstance().fastDecompressor(); + checksum = XXHashFactory.fastestInstance().newStreamingHash32(DEFAULT_SEED).asChecksum(); + } + + @Override + public int getOriginalLen(byte[] src) { + return readIntLE(src, MAGIC_LENGTH + 5); + } + + @Override + public int decompress(byte[] src, byte[] dst, int dstOff) { + int token = src[MAGIC_LENGTH] & 0xFF; + int compressionMethod = token & 0xF0; + int compressionLevel = COMPRESSION_LEVEL_BASE + (token & 0x0F); + int compressedLen = readIntLE(src, MAGIC_LENGTH + 1); + int originalLen = readIntLE(src, MAGIC_LENGTH + 5); + int check = readIntLE(src, MAGIC_LENGTH + 9); + + switch (compressionMethod) { + case COMPRESSION_METHOD_RAW: + System.arraycopy(src, HEADER_LENGTH, dst, dstOff, originalLen); + break; + case COMPRESSION_METHOD_CSS: + int compressedLen2 = decompressor.decompress(src, HEADER_LENGTH, dst, dstOff, originalLen); + if (compressedLen != compressedLen2) { + logger.error("compressed len corrupted!"); + return -1; + } + } + + checksum.reset(); + checksum.update(dst, dstOff, originalLen); + if ((int) checksum.getValue() != check) { + logger.error("checksum not equal!"); + return -1; + } + + return originalLen; + } + + public static int readIntLE(byte[] buf, int i) { + return (buf[i] & 0xFF) | ((buf[i + 1] & 0xFF) << 8) | + ((buf[i + 2] & 0xFF) << 16) | ((buf[i + 3] & 0xFF) << 24); + } +} diff --git a/client/src/main/java/com/bytedance/css/client/compress/ZstdCompressor.java b/client/src/main/java/com/bytedance/css/client/compress/ZstdCompressor.java new file mode 100644 index 0000000..d004adf --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/compress/ZstdCompressor.java @@ -0,0 +1,145 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.compress; + +import com.bytedance.css.common.ChildFirstURLClassLoader; +import com.bytedance.css.common.CssConf; +import net.jpountz.xxhash.XXHashFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.reflect.Method; +import java.net.URL; +import java.util.zip.Checksum; + +/** + * Use class reflect to get zstd method with target version. + * since lower zstd version in spark 2.x can not support compressByteArray method. + */ +public class ZstdCompressor implements Compressor { + + private static final Logger logger = LoggerFactory.getLogger(ZstdCompressor.class); + + private final Checksum checksum; + private final int zstdCompressLevel; + private final ChildFirstURLClassLoader classLoader; + private final Method compressBoundMethod; + private final Method compressByteArrayMethod; + private final Method compressIsErrorMethod; + private final Method compressGetErrorNameMethod; + private final Method compressGetErrorCodeMethod; + + // only used for test + private boolean testMode; + private byte[] compressedBuffer; + private int compressedTotalSize; + + public ZstdCompressor() { + this(new CssConf()); + } + + public ZstdCompressor(CssConf cssConf) { + try { + String file = getClass().getProtectionDomain().getCodeSource().getLocation().getFile(); + URL url = new URL(String.format("file:%s", file)); + URL[] urls = {url}; + classLoader = new ChildFirstURLClassLoader(urls, Thread.currentThread().getContextClassLoader()); + Class zstdClass = classLoader.loadClass("com.github.luben.zstd.Zstd"); + compressBoundMethod = zstdClass.getDeclaredMethod("compressBound", long.class); + compressByteArrayMethod = zstdClass.getDeclaredMethod("compressByteArray", + byte[].class, int.class, int.class, byte[].class, int.class, int.class, int.class); + compressIsErrorMethod = zstdClass.getDeclaredMethod("isError", long.class); + compressGetErrorNameMethod = zstdClass.getDeclaredMethod("getErrorName", long.class); + compressGetErrorCodeMethod = zstdClass.getDeclaredMethod("getErrorCode", long.class); + + int blockSize = (int) CssConf.pushBufferSize(cssConf); + this.zstdCompressLevel = CssConf.zstdCompressionLevel(cssConf); + this.testMode = CssConf.compressionTestMode(cssConf); + checksum = XXHashFactory.fastestInstance().newStreamingHash32(DEFAULT_SEED).asChecksum(); + initCompressBuffer(blockSize); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + private void initCompressBuffer(int blockSize) throws Exception { + long compressedBlockSize = HEADER_LENGTH + (long) compressBoundMethod.invoke(null, blockSize); + compressedBuffer = new byte[(int) compressedBlockSize]; + System.arraycopy(MAGIC, 0, compressedBuffer, 0, MAGIC_LENGTH); + } + + @Override + public void compress(byte[] data, int offset, int length) { + try { + checksum.reset(); + checksum.update(data, offset, length); + final int check = (int) checksum.getValue(); + if (compressedBuffer.length - HEADER_LENGTH < length) { + initCompressBuffer(length); + } + long tmp = (long) compressByteArrayMethod.invoke( + null, compressedBuffer, HEADER_LENGTH, compressedBuffer.length - HEADER_LENGTH, + data, offset, length, this.zstdCompressLevel); + boolean isCompressError = ((boolean) compressIsErrorMethod.invoke(null, tmp)) | testMode; + if (isCompressError) { + String errorName = (String) compressGetErrorNameMethod.invoke(null, tmp); + long errorCode = (long) compressGetErrorCodeMethod.invoke(null, tmp); + logger.error("zstd compress error with data str {} compressLen {} errorName {} errorCode {}. ignore it.", + new String(data, offset, length), tmp, errorName, errorCode); + } + + int compressedLength = (int) tmp; + final int compressMethod; + if (isCompressError || compressedLength >= length) { + compressMethod = COMPRESSION_METHOD_RAW; + compressedLength = length; + System.arraycopy(data, offset, compressedBuffer, HEADER_LENGTH, length); + } else { + compressMethod = COMPRESSION_METHOD_CSS; + } + + compressedBuffer[MAGIC_LENGTH] = (byte) (compressMethod | zstdCompressLevel); + writeIntLE(compressedLength, compressedBuffer, MAGIC_LENGTH + 1); + writeIntLE(length, compressedBuffer, MAGIC_LENGTH + 5); + writeIntLE(check, compressedBuffer, MAGIC_LENGTH + 9); + + compressedTotalSize = HEADER_LENGTH + compressedLength; + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + @Override + public int getCompressedTotalSize() { + return compressedTotalSize; + } + + @Override + public byte[] getCompressedBuffer() { + return compressedBuffer; + } + + private static void writeIntLE(int i, byte[] buf, int off) { + buf[off++] = (byte) i; + buf[off++] = (byte) (i >>> 8); + buf[off++] = (byte) (i >>> 16); + buf[off++] = (byte) (i >>> 24); + } +} diff --git a/client/src/main/java/com/bytedance/css/client/compress/ZstdDecompressor.java b/client/src/main/java/com/bytedance/css/client/compress/ZstdDecompressor.java new file mode 100644 index 0000000..f2fd63d --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/compress/ZstdDecompressor.java @@ -0,0 +1,105 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.compress; + +import com.bytedance.css.common.ChildFirstURLClassLoader; + +import net.jpountz.xxhash.XXHashFactory; +import org.apache.log4j.Logger; + +import java.lang.reflect.Method; +import java.net.URL; +import java.util.zip.Checksum; + +/** + * Use class reflect to get zstd method with target version. + * since lower zstd version in spark 2.x can not support compressByteArray method. + */ +public class ZstdDecompressor implements Decompressor { + + private static final Logger logger = Logger.getLogger(ZstdDecompressor.class); + + private final Checksum checksum; + private final ChildFirstURLClassLoader classLoader; + private final Method decompressByteArrayMethod; + + public ZstdDecompressor() { + try { + String file = getClass().getProtectionDomain().getCodeSource().getLocation().getFile(); + URL url = new URL(String.format("file:%s", file)); + URL[] urls = {url}; + classLoader = new ChildFirstURLClassLoader(urls, Thread.currentThread().getContextClassLoader()); + Class zstdClass = classLoader.loadClass("com.github.luben.zstd.Zstd"); + decompressByteArrayMethod = zstdClass.getDeclaredMethod("decompressByteArray", + byte[].class, int.class, int.class, byte[].class, int.class, int.class); + + checksum = XXHashFactory.fastestInstance().newStreamingHash32(DEFAULT_SEED).asChecksum(); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + @Override + public int getOriginalLen(byte[] src) { + return readIntLE(src, MAGIC_LENGTH + 5); + } + + @Override + public int decompress(byte[] src, byte[] dst, int dstOff) { + try { + int token = src[MAGIC_LENGTH] & 0xFF; + int compressionMethod = token & 0xF0; + int compressionLevel = token & 0x0F; + int compressedLen = readIntLE(src, MAGIC_LENGTH + 1); + int originalLen = readIntLE(src, MAGIC_LENGTH + 5); + int check = readIntLE(src, MAGIC_LENGTH + 9); + + switch (compressionMethod) { + case COMPRESSION_METHOD_RAW: + System.arraycopy(src, HEADER_LENGTH, dst, dstOff, originalLen); + break; + case COMPRESSION_METHOD_CSS: + long tmp = (long) decompressByteArrayMethod.invoke( + null, dst, dstOff, originalLen, src, HEADER_LENGTH, compressedLen); + int compressedLen2 = (int) tmp; + if (compressedLen2 != originalLen) { + logger.error("compressed len corrupted!"); + return -1; + } + } + + checksum.reset(); + checksum.update(dst, dstOff, originalLen); + if ((int) checksum.getValue() != check) { + logger.error("checksum not equal!"); + return -1; + } + + return originalLen; + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + public static int readIntLE(byte[] buf, int i) { + return (buf[i] & 0xFF) | ((buf[i + 1] & 0xFF) << 8) | + ((buf[i + 2] & 0xFF) << 16) | ((buf[i + 3] & 0xFF) << 24); + } +} diff --git a/client/src/main/java/com/bytedance/css/client/compress/ZstdDictCompressor.java b/client/src/main/java/com/bytedance/css/client/compress/ZstdDictCompressor.java new file mode 100644 index 0000000..d9b5ea8 --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/compress/ZstdDictCompressor.java @@ -0,0 +1,125 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.compress; + +import com.bytedance.css.common.ChildFirstURLClassLoader; +import com.bytedance.css.common.CssConf; +import net.jpountz.xxhash.XXHashFactory; + +import java.lang.reflect.Method; +import java.net.URL; +import java.util.zip.Checksum; + +/** + * Use class reflect to get zstd method with target version. + * since lower zstd version in spark 2.x can not support compressByteArray method. + */ +public class ZstdDictCompressor implements Compressor { + + private final Checksum checksum; + private final int zstdCompressLevel; + private final ChildFirstURLClassLoader classLoader; + private final Method compressBoundMethod; + private final Method compressUsingDictMethod; + + // only used for test + private boolean testMode; + private byte[] compressedBuffer; + private int compressedTotalSize; + + private volatile byte[] dict; + + public ZstdDictCompressor(CssConf cssConf, byte[] dict) { + try { + String file = getClass().getProtectionDomain().getCodeSource().getLocation().getFile(); + URL url = new URL(String.format("file:%s", file)); + URL[] urls = {url}; + classLoader = new ChildFirstURLClassLoader(urls, Thread.currentThread().getContextClassLoader()); + Class zstdClass = classLoader.loadClass("com.github.luben.zstd.Zstd"); + compressBoundMethod = zstdClass.getDeclaredMethod("compressBound", long.class); + compressUsingDictMethod = zstdClass.getDeclaredMethod("compressUsingDict", + byte[].class, int.class, byte[].class, int.class, int.class, byte[].class, int.class); + + this.dict = dict; + this.zstdCompressLevel = CssConf.zstdCompressionLevel(cssConf); + this.testMode = CssConf.compressionTestMode(cssConf); + int blockSize = (int) CssConf.pushBufferSize(cssConf); + checksum = XXHashFactory.fastestInstance().newStreamingHash32(DEFAULT_SEED).asChecksum(); + initCompressBuffer(blockSize); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + private void initCompressBuffer(int blockSize) throws Exception { + long compressedBlockSize = HEADER_LENGTH + (long) compressBoundMethod.invoke(null, blockSize); + compressedBuffer = new byte[(int) compressedBlockSize]; + System.arraycopy(MAGIC, 0, compressedBuffer, 0, MAGIC_LENGTH); + } + + @Override + public void compress(byte[] data, int offset, int length) { + try { + checksum.reset(); + checksum.update(data, offset, length); + final int check = (int) checksum.getValue(); + if (compressedBuffer.length - HEADER_LENGTH < length) { + initCompressBuffer(length); + } + long tmp = (long) compressUsingDictMethod.invoke( + null, this.compressedBuffer, HEADER_LENGTH, data, offset, length, dict, this.zstdCompressLevel); + int compressedLength = (int) tmp; + final int compressMethod; + if (compressedLength >= length || testMode) { + compressMethod = COMPRESSION_METHOD_RAW; + compressedLength = length; + System.arraycopy(data, offset, compressedBuffer, HEADER_LENGTH, length); + } else { + compressMethod = COMPRESSION_METHOD_CSS; + } + + compressedBuffer[MAGIC_LENGTH] = (byte) (compressMethod | zstdCompressLevel); + writeIntLE(compressedLength, compressedBuffer, MAGIC_LENGTH + 1); + writeIntLE(length, compressedBuffer, MAGIC_LENGTH + 5); + writeIntLE(check, compressedBuffer, MAGIC_LENGTH + 9); + + compressedTotalSize = HEADER_LENGTH + compressedLength; + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + @Override + public int getCompressedTotalSize() { + return compressedTotalSize; + } + + @Override + public byte[] getCompressedBuffer() { + return compressedBuffer; + } + + private static void writeIntLE(int i, byte[] buf, int off) { + buf[off++] = (byte) i; + buf[off++] = (byte) (i >>> 8); + buf[off++] = (byte) (i >>> 16); + buf[off++] = (byte) (i >>> 24); + } +} diff --git a/client/src/main/java/com/bytedance/css/client/compress/ZstdDictDecompressor.java b/client/src/main/java/com/bytedance/css/client/compress/ZstdDictDecompressor.java new file mode 100644 index 0000000..d8c235b --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/compress/ZstdDictDecompressor.java @@ -0,0 +1,108 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.compress; + +import com.bytedance.css.common.ChildFirstURLClassLoader; + +import net.jpountz.xxhash.XXHashFactory; +import org.apache.log4j.Logger; + +import java.lang.reflect.Method; +import java.net.URL; +import java.util.zip.Checksum; + +/** + * Use class reflect to get zstd method with target version. + * since lower zstd version in spark 2.x can not support compressByteArray method. + */ +public class ZstdDictDecompressor implements Decompressor { + + private static final Logger logger = Logger.getLogger(ZstdDictDecompressor.class); + + private final Checksum checksum; + private final ChildFirstURLClassLoader classLoader; + private final Method decompressUsingDictMethod; + + private volatile byte[] dict; + + public ZstdDictDecompressor(byte[] dict) { + try { + String file = getClass().getProtectionDomain().getCodeSource().getLocation().getFile(); + URL url = new URL(String.format("file:%s", file)); + URL[] urls = {url}; + classLoader = new ChildFirstURLClassLoader(urls, Thread.currentThread().getContextClassLoader()); + Class zstdClass = classLoader.loadClass("com.github.luben.zstd.Zstd"); + decompressUsingDictMethod = zstdClass.getDeclaredMethod("decompressUsingDict", + byte[].class, int.class, byte[].class, int.class, int.class, byte[].class); + + this.dict = dict; + checksum = XXHashFactory.fastestInstance().newStreamingHash32(DEFAULT_SEED).asChecksum(); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + @Override + public int getOriginalLen(byte[] src) { + return readIntLE(src, MAGIC_LENGTH + 5); + } + + @Override + public int decompress(byte[] src, byte[] dst, int dstOff) { + try { + int token = src[MAGIC_LENGTH] & 0xFF; + int compressionMethod = token & 0xF0; + int compressionLevel = token & 0x0F; + int compressedLen = readIntLE(src, MAGIC_LENGTH + 1); + int originalLen = readIntLE(src, MAGIC_LENGTH + 5); + int check = readIntLE(src, MAGIC_LENGTH + 9); + + switch (compressionMethod) { + case COMPRESSION_METHOD_RAW: + System.arraycopy(src, HEADER_LENGTH, dst, dstOff, originalLen); + break; + case COMPRESSION_METHOD_CSS: + long tmp = (long) decompressUsingDictMethod.invoke( + null, dst, dstOff, src, HEADER_LENGTH, compressedLen, dict); + int compressedLen2 = (int) tmp; + if (compressedLen2 != originalLen) { + logger.error("compressed len corrupted!"); + return -1; + } + } + + checksum.reset(); + checksum.update(dst, dstOff, originalLen); + if ((int) checksum.getValue() != check) { + logger.error("checksum not equal!"); + return -1; + } + + return originalLen; + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + public static int readIntLE(byte[] buf, int i) { + return (buf[i] & 0xFF) | ((buf[i + 1] & 0xFF) << 8) | + ((buf[i + 2] & 0xFF) << 16) | ((buf[i + 3] & 0xFF) << 24); + } +} diff --git a/client/src/main/java/com/bytedance/css/client/compress/ZstdDictTrainer.java b/client/src/main/java/com/bytedance/css/client/compress/ZstdDictTrainer.java new file mode 100644 index 0000000..d12c37f --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/compress/ZstdDictTrainer.java @@ -0,0 +1,75 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.compress; + +import com.bytedance.css.common.ChildFirstURLClassLoader; +import com.bytedance.css.common.util.Utils; +import org.apache.commons.io.IOUtils; + +import java.io.File; +import java.io.FileInputStream; +import java.io.InputStream; +import java.lang.reflect.Method; +import java.net.URL; + +public class ZstdDictTrainer { + + private byte[] dict; + + public ZstdDictTrainer() { + initZstdDict("xml"); + } + + private void initZstdDict(String dictFile) { + try { + String jarFile = ZstdDictTrainer.class.getProtectionDomain().getCodeSource().getLocation().getFile(); + URL url = new URL(String.format("file:%s", jarFile)); + URL[] urls = {url}; + ClassLoader classLoader = new ChildFirstURLClassLoader(urls, Thread.currentThread().getContextClassLoader()); + Class zstdDictTrainerClass = classLoader.loadClass("com.github.luben.zstd.ZstdDictTrainer"); + Object trainer = zstdDictTrainerClass.getConstructor(int.class, int.class).newInstance(1024 * 1024, 32 * 1024); + Method addSampleMethod = zstdDictTrainerClass.getDeclaredMethod("addSample", byte[].class); + Method trainSamplesMethod = zstdDictTrainerClass.getDeclaredMethod("trainSamples", boolean.class); + + Class zstdClass = classLoader.loadClass("com.github.luben.zstd.Zstd"); + Method getDictIdFromDictMethod = zstdClass.getDeclaredMethod("getDictIdFromDict", byte[].class); + + String path = Utils.getClassLoader().getResource(dictFile).getPath(); + File file = new File(path); + InputStream in = new FileInputStream(file); + + int count = 0; + while (count < file.length()) { + byte[] buffer = new byte[1024]; + IOUtils.readFully(in, buffer); + addSampleMethod.invoke(trainer, buffer); + count += buffer.length; + } + dict = (byte[]) trainSamplesMethod.invoke(trainer, true); + assert((long) getDictIdFromDictMethod.invoke(null, dict) != 0L); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public byte[] getDict() { + return this.dict; + } +} diff --git a/client/src/main/java/com/bytedance/css/client/impl/ShuffleClientImpl.java b/client/src/main/java/com/bytedance/css/client/impl/ShuffleClientImpl.java new file mode 100644 index 0000000..b9ecd5d --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/impl/ShuffleClientImpl.java @@ -0,0 +1,1056 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.impl; + +import com.bytedance.css.client.ShuffleClient; +import com.bytedance.css.client.MapperAttemptStatus; +import com.bytedance.css.client.compress.Compressor; +import com.bytedance.css.client.compress.CssCompressorFactory; +import com.bytedance.css.client.metrics.ClientSource; +import com.bytedance.css.client.stream.CssInputStream; +import com.bytedance.css.client.stream.disk.CssRemoteDiskEpochReader; +import com.bytedance.css.common.CssConf; +import com.bytedance.css.common.exception.PartitionInfoNotFoundException; +import com.bytedance.css.common.protocol.*; +import com.bytedance.css.common.rpc.RpcAddress; +import com.bytedance.css.common.rpc.RpcEndpointRef; +import com.bytedance.css.common.rpc.RpcEnv; +import com.bytedance.css.common.unsafe.Platform; +import com.bytedance.css.common.util.ThreadUtils; +import com.bytedance.css.common.util.Utils; +import com.bytedance.css.network.TransportContext; +import com.bytedance.css.network.buffer.NettyManagedBuffer; +import com.bytedance.css.network.client.RpcResponseCallback; +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.client.TransportClientFactory; +import com.bytedance.css.network.protocol.BatchPushDataRequest; +import com.bytedance.css.network.server.NoOpRpcHandler; +import com.bytedance.css.network.util.TransportConf; +import com.codahale.metrics.Counter; +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Meter; +import com.codahale.metrics.Timer; +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.Uninterruptibles; +import com.netflix.concurrency.limits.Limiter; +import com.netflix.concurrency.limits.limit.FixedLimit; +import com.netflix.concurrency.limits.limit.Gradient2Limit; +import com.netflix.concurrency.limits.limiter.BlockingLimiter; +import com.netflix.concurrency.limits.limiter.SimpleLimiter; +import io.netty.buffer.CompositeByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.util.internal.ConcurrentSet; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.reflect.ClassTag$; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.*; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.ReentrantLock; +import java.util.stream.Collectors; + +public class ShuffleClientImpl extends ShuffleClient { + + private static final Logger logger = LoggerFactory.getLogger(ShuffleClient.class); + + // Configuration + private final CssConf cssConf; + private final ShuffleMode defaultShuffleMode; + private final long rotateThreshold; + private final int pushIoMaxRetries; + private final long pushIoRetryWaitMs; + private final int partitionGroupPushRetries; + + // For RPC + private RpcEnv rpcEnv; + private RpcEndpointRef masterRpcRef; + private TransportClientFactory clientFactory; + + // shuffleId in registering + // TODO shuffle register control when push data for diff engine, like mr flink. + private Set registering = new ConcurrentSet<>(); + + // multi control for reallocate + private Set reallocating = new ConcurrentSet<>(); + private ReentrantLock reallocatingLock = new ReentrantLock(); + private Condition pushDataAwaitCondition = reallocatingLock.newCondition(); + private Condition reallocatingAwaitCondition = reallocatingLock.newCondition(); + + // multi control for read partition info + private Set reading = new ConcurrentSet<>(); + + // key: Utils.makeMapperKey(shuffleId-mapId-attemptId) + // value: MapperAttemptStatus + private Map statusMap = new ConcurrentHashMap<>(); + + // key: shuffleId + // value: Set[Utils.makeMapperKey(shuffleId-mapId-attemptId)] + private final ConcurrentHashMap> mapperEndMap = new ConcurrentHashMap<>(); + + // key: shuffleId + // value: (reduceId, Array[CommittedPartitionInfo]) + private Map> reducerFileGroups = + new ConcurrentHashMap<>(); + + // key: shuffleId + // value: mapperAttemptIds remain all success map attempts to do + private Map mapperAttempts = new ConcurrentHashMap<>(); + private Map> batchBlacklistMap = new ConcurrentHashMap<>(); + + // Push Data Retry ThreadPool + private ThreadPoolExecutor pushDataRetryThreadPool; + + // app heart beat + private Thread appHeartBeatThread; + private volatile boolean appHeartBeatStarted = false; + private Object appHeartBeatLock = new Object(); + + // metrics for push data + private Timer mapperEndWaitTimer = ClientSource.instance().mapperEndWaitTimer; + private Timer pushDataWaitTimer = ClientSource.instance().pushDataWaitTimer; + private Timer reducerFileGroupsTimer = ClientSource.instance().reducerFileGroupsTimer; + private Counter dataLostCounter = ClientSource.instance().dataLostCounter; + private Histogram pushDataRawSize = ClientSource.instance().pushDataRawSizeHistogram; + private Histogram pushDataSize = ClientSource.instance().pushDataSizeHistogram; + private Meter batchPushThroughput = ClientSource.instance().batchPushThroughput; + + // Compressor + private CssCompressorFactory compressorFactory; + private ThreadLocal compressorThreadLocal; + + // backpressure send rate control + // Global dynamics backpressure limit. + private final boolean backpressureEnable; + private final boolean backpressureLogEnable; + private final int backpressureMaxConcurrency; + private final int backpressureMinLimit; + private final double backpressureSmoothing; + private final int backpressureLongWindow; + private final double backpressureRttTolerance; + private final int backpressureQueueSize; + // fix rate backpressure limit. + private final int fixRateLimit; + + private final ConcurrentHashMap> limiters = + new ConcurrentHashMap<>(); + private final int reallocateFailedMaxTimes; + + private final boolean failedBatchBlacklistEnable; + + public ShuffleClientImpl(CssConf cssConf) { + this.cssConf = cssConf; + this.failedBatchBlacklistEnable = CssConf.failedBatchBlacklistEnable(cssConf); + this.defaultShuffleMode = CssConf.shuffleMode(cssConf); + this.rotateThreshold = CssConf.epochRotateThreshold(cssConf); + this.reallocateFailedMaxTimes = CssConf.clientReallocateFailedMaxTimes(cssConf); + this.pushDataRetryThreadPool = ThreadUtils.newDaemonCachedThreadPool( + "PushDataRequest Retry ThreadPool", CssConf.pushDataRetryThreads(cssConf), 60); + this.compressorFactory = new CssCompressorFactory(this.cssConf); + this.compressorThreadLocal = ThreadLocal.withInitial(() -> this.compressorFactory.getCompressor()); + this.pushIoMaxRetries = CssConf.pushIoMaxRetries(cssConf); + this.pushIoRetryWaitMs = CssConf.pushIoRetryWaitMs(cssConf); + this.partitionGroupPushRetries = CssConf.partitionGroupPushRetries(cssConf); + + // backpressure init + this.backpressureEnable = CssConf.backpressureEnable(this.cssConf); + this.backpressureLogEnable = CssConf.backpressureLogEnable(this.cssConf); + this.backpressureMaxConcurrency = CssConf.backpressureMaxConcurrency(this.cssConf); + this.backpressureMinLimit = CssConf.backpressureMinLimit(this.cssConf); + this.backpressureSmoothing = CssConf.backpressureSmoothing(this.cssConf); + this.backpressureLongWindow = CssConf.backpressureLongWindow(this.cssConf); + this.backpressureRttTolerance = CssConf.backpressureRttTolerance(this.cssConf); + this.backpressureQueueSize = CssConf.backpressureQueueSize(this.cssConf); + + this.fixRateLimit = CssConf.fixRateLimitThreshold(this.cssConf); + + rpcEnvInit(); + } + + private BlockingLimiter getBackpressureLimiter(String hostPortStr) { + BlockingLimiter limit; + if (!backpressureEnable) { + limit = limiters.computeIfAbsent("fixRateLimiter", (s) -> + BlockingLimiter.wrap(SimpleLimiter.newBuilder().limit( + FixedLimit.of(this.fixRateLimit) + ).build())); + } else { + limit = limiters.computeIfAbsent(hostPortStr, (s) -> { + Gradient2Limit gradient2Limit = Gradient2Limit.newBuilder() + .minLimit(backpressureMinLimit) + .smoothing(backpressureSmoothing) + .longWindow(backpressureLongWindow) + .rttTolerance(backpressureRttTolerance) + .queueSize(backpressureQueueSize) + .maxConcurrency(backpressureMaxConcurrency).build(); + if (backpressureLogEnable) { + gradient2Limit.notifyOnChange((newLimit) -> { + logger.info("Update to new limit {}", newLimit); + }); + } + return BlockingLimiter.wrap(SimpleLimiter.newBuilder().limit( + gradient2Limit + ).build()); + }); + } + return limit; + } + + private void rpcEnvInit() { + TransportConf transportConf = + Utils.fromCssConf(cssConf, TransportModuleConstants.DATA_MODULE, CssConf.dataThreads(cssConf)); + TransportContext transportContext = new TransportContext(transportConf, new NoOpRpcHandler(), false); + this.clientFactory = transportContext.createClientFactory(Lists.newArrayList()); + + rpcEnv = RpcEnv.create(RpcNameConstants.SHUFFLE_CLIENT_SYS, Utils.localHostName(), 0, cssConf, true); + masterRpcRef = rpcEnv.setupEndpointRef( + RpcAddress.fromCssURL(CssConf.masterAddress(cssConf)), RpcNameConstants.MASTER_EP); + } + + private void checkBatchData(int[] reducerIdArray, int[] offsetArray, int[] lengthArray) throws IOException { + if (reducerIdArray.length != offsetArray.length || reducerIdArray.length != lengthArray.length) { + throw new IOException("Tuple3 reducerIdArray offsetArray lengthArray not eq!"); + } + + for (int i = 0; i < offsetArray.length - 1; i ++) { + if (offsetArray[i] + lengthArray[i] != offsetArray[i + 1]) { + throw new IOException("offsetArray lengthArray metadata incorrect!"); + } + } + } + + @Override + public int[] batchPushData( + String applicationId, + int shuffleId, + int mapperId, + int mapperAttemptId, + int[] reducerIdArray, + byte[] data, + int[] offsetArray, + int[] lengthArray, + int numMappers, + int numPartitions, + boolean skipCompress) throws IOException { + + checkBatchData(reducerIdArray, offsetArray, lengthArray); + + // TODO When other engines like mr or flink cannot broadcast the register shuffle result, + // we can initialize register it first. when sending data + + String shuffleKey = Utils.getShuffleKey(applicationId, shuffleId); + PartitionGroupManager manager = shufflePartitionGroupMap.get(shuffleId); + + // make sure all reducerId belong to single partitionGroup + long distinctGroups = Arrays.stream(reducerIdArray).map(manager::groupId).distinct().count(); + if (distinctGroups != 1) { + throw new IOException("reducerIdArray not belong to single partitionGroup"); + } + + int partitionGroupId = manager.groupId(reducerIdArray[0]); + waitUntilReallocatePartitionGroupEnded(applicationId, shuffleId, partitionGroupId); + + PartitionGroup partitionGroup = shufflePartitionGroupMap.get(shuffleId).getGroup(partitionGroupId); + + String mapperKey = Utils.getMapperKey(shuffleId, mapperId, mapperAttemptId); + MapperAttemptStatus status = statusMap.computeIfAbsent(mapperKey, (s) -> new MapperAttemptStatus()); + + if (status.getException() != null) { + throw status.getException(); + } + + // before sending data, record the current send request + 1 + final int reqBatchId = status.getNextBatchId(); + final int[] partitionBatchIds = + Arrays.stream(reducerIdArray).map(reducerId -> status.getNextBatchId()).toArray(); + status.InFlightReqs.incrementAndGet(); + + int writeBytes = 0; + int[] partitionWrittenBytes = new int[reducerIdArray.length]; + List offsetLists = new ArrayList<>(); + offsetLists.add(0); // init offset start with 0. it will be used for worker bytebuf.slice(index, length) + // use current batchId to mark all reduce data. + CompositeByteBuf compositeByteBuf = Unpooled.compositeBuffer(reducerIdArray.length); + for (int i = 0; i < reducerIdArray.length; i ++) { + int partitionBatchId = partitionBatchIds[i]; + byte[] batchBytes; + if (skipCompress) { + batchBytes = addHeader(mapperId, mapperAttemptId, partitionBatchId, data, offsetArray[i], lengthArray[i]); + } else { + batchBytes = addHeaderAndCompress( + mapperId, + mapperAttemptId, + partitionBatchId, + data, + offsetArray[i], + lengthArray[i]); + } + compositeByteBuf.addComponent(true, Unpooled.wrappedBuffer(batchBytes)); + offsetLists.add(offsetLists.get(offsetLists.size() - 1) + batchBytes.length); + partitionWrittenBytes[i] = batchBytes.length; + writeBytes += batchBytes.length; + } + int[] offsets = offsetLists.stream().mapToInt(Integer::valueOf).toArray(); + + // TODO use compositeByteBuf directly to send to transportClient + byte[] sendBytes = new byte[writeBytes]; + compositeByteBuf.readBytes(sendBytes); + + int originTotalLength = Arrays.stream(lengthArray).sum(); + int originTotalWriteBytes = writeBytes; + + asyncBatchPushDataWithRetry(partitionGroup, applicationId, shuffleId, mapperId, mapperAttemptId, + reqBatchId, partitionBatchIds, reducerIdArray, sendBytes, offsets, status, 0, + originTotalLength, originTotalWriteBytes); + + return partitionWrittenBytes; + } + + private void asyncBatchPushDataWithRetry( + PartitionGroup partitionGroup, + String applicationId, + int shuffleId, + int mapperId, + int mapperAttemptId, + int reqBatchId, + int[] partitionBatchIds, + int[] reducerIdArray, + byte[] compressData, + int[] offsets, + MapperAttemptStatus status, + final int retryCount, + int originTotalLength, + int originTotalWriteBytes) throws IOException { + + String shuffleKey = Utils.getShuffleKey(applicationId, shuffleId); + String mapperKey = Utils.getMapperKey(shuffleId, mapperId, mapperAttemptId); + + if (status.getException() != null) { + throw status.getException(); + } + + int pgId = partitionGroup.partitionGroupId; + int epochId = partitionGroup.epochId; + String pairAddressStr = partitionGroup.makeReplicaAddressStr(); + + String shuffleMode = defaultShuffleMode.toString(); + String tmpRotateThreshold = String.valueOf(rotateThreshold); + + Optional listener = Optional.empty(); + Timer.Context waitTimer = pushDataWaitTimer.time(); + listener = getBackpressureLimiter(pairAddressStr).acquire(null); + waitTimer.stop(); + + Optional finalListener = listener; + if (!finalListener.isPresent()) { + // listener not defined means after certain timeout, still no permit granted from limiter + throw new IOException("Backpressure limit timeout!"); + } + + if (retryCount > this.partitionGroupPushRetries) { + throw new IOException("reallocatePartitionGroup retry still failed!"); + } + + List originPair = partitionGroup.getReplicaWorkers(); + List pair = new ArrayList<>(partitionGroup.getReplicaWorkers()); + // shuffle this worker pair to random send order. + Collections.shuffle(pair); + final int expectRet = pair.size(); + + List> pairFutures = pair.stream().map(wa -> { + // return 0 means push successful + // return 1 means rotate force write successful + // return > expectRet means current push request failed + CompletableFuture result = new CompletableFuture<>(); + + RpcResponseCallback callback = new RpcResponseCallback() { + @Override + public void onSuccess(ByteBuffer response) { + int remaining = response.remaining(); + if (remaining == 1 && response.duplicate().get() == CssStatusCode.EpochShouldRotate.getValue()) { + logger.warn(String.format("batchPushDataRequest to %s:%s Hit EpochShouldRotate " + + "appId %s shuffleId %s mapperId %s mapperAttemptId %s groupId %s epochId %s batchId %s", + wa.host, wa.port, + applicationId, shuffleId, mapperId, mapperAttemptId, + pgId, epochId, reqBatchId)); + result.complete(1); + } else if (remaining == 1 && response.duplicate().get() == CssStatusCode.StageEnded.getValue()) { + logger.warn(String.format("batchPushDataRequest to %s:%s Hit StageEnded " + + "appId %s shuffleId %s mapperId %s mapperAttemptId %s groupId %s epochId %s batchId %s", + wa.host, wa.port, + applicationId, shuffleId, mapperId, mapperAttemptId, + pgId, epochId, reqBatchId)); + mapperEndMap.computeIfAbsent(shuffleId, (id) -> new ConcurrentSet<>()).add(mapperKey); + result.complete(expectRet + 1); + } else { + result.complete(0); + } + } + + @Override + public void onFailure(Throwable e) { + // break from worker callback, since next pushData will check exception and fail in the first place + // check whether failed with PartitionInfoNotFoundException. + // it means worker may happen an internal server error. + if (e != null && e instanceof PartitionInfoNotFoundException) { + logger.warn(String.format("batchPushDataRequest to %s:%s Hit PartitionInfoNotFoundException " + + "appId %s shuffleId %s mapperId %s mapperAttemptId %s groupId %s epochId %s batchId %s", + wa.host, wa.port, + applicationId, shuffleId, mapperId, mapperAttemptId, + pgId, epochId, reqBatchId)); + status.setException(new IOException(e)); + } + result.complete(expectRet + 2); + } + }; + + try { + TransportClient client = createClientWithRetry(wa.host, wa.port, -1); + long pushDataStartMs = System.currentTimeMillis(); + client.batchPushData( + new BatchPushDataRequest(shuffleKey, reducerIdArray, epochId, offsets, mapperId, + originPair.indexOf(wa), shuffleMode, tmpRotateThreshold, pushDataStartMs, + new NettyManagedBuffer(Unpooled.wrappedBuffer(compressData))), callback); + } catch (Exception exception) { + logger.error( + String.format("batchPushDataRequest to %s:%s Hit ClientSideException " + + "appId %s shuffleId %s mapperId %s mapperAttemptId %s groupId %s epochId %s batchId %s", + wa.host, wa.port, applicationId, shuffleId, mapperId, mapperAttemptId, + pgId, epochId, reqBatchId), exception); + result.complete(expectRet + 3); + } + return result; + }).collect(Collectors.toList()); + + AtomicInteger resultSum = new AtomicInteger(0); + CompletableFuture.allOf(pairFutures.toArray(new CompletableFuture[0])) + .whenCompleteAsync((v, t) -> { + pairFutures.forEach(future -> resultSum.addAndGet(future.getNow(100))); + + boolean isFinished = true; + if (resultSum.get() <= expectRet) { + // let's see here 2 pair workers & expectRet = 2 + // if return 0 0 means worker writes successfully + // if return 0 1 / 1 0 / 1 1 means the strong write is successful and requires rotate + // when write success. update limiter & writtenEpochSet + finalListener.get().onSuccess(); + Arrays.stream(reducerIdArray).forEach(reducerId -> { + status.writtenEpochSet.add(new PartitionInfo(reducerId, epochId)); + }); + if (resultSum.get() > 0) { + // after the worker forcibly writes, notify the client to rotate + reallocatePartitionGroupWithRetry(applicationId, shuffleId, mapperId, mapperAttemptId, partitionGroup); + } + } else { + // means this data write failed + // update current limiter & failedBatchBlacklist + finalListener.get().onIgnore(); + if (failedBatchBlacklistEnable) { + for (int i = 0; i < reducerIdArray.length; i ++) { + status.failedBatchBlacklist.add(new FailedPartitionInfoBatch( + reducerIdArray[i], epochId, mapperId, mapperAttemptId, partitionBatchIds[i]) + ); + } + } + // reallocate partitionGroup and retry + if (mapperEnded(shuffleId, mapperId, mapperAttemptId)) { + // mapper ended skip retry + logger.warn(String.format("Skip retry since Hit StageEnded " + + "appId %s shuffleId %s mapperId %s mapperAttemptId %s groupId %s", + applicationId, shuffleId, mapperId, mapperAttemptId, pgId)); + } else if (retryCount < this.partitionGroupPushRetries) { + if (!reallocatePartitionGroupWithRetry(applicationId, shuffleId, + mapperId, mapperAttemptId, partitionGroup)) { + logger.error(String.format("Encounter %s retry failed for " + + "appId %s shuffleId %s mapperId %s mapperAttemptId %s groupId %s", + retryCount, applicationId, shuffleId, mapperId, mapperAttemptId, pgId)); + status.setException(new IOException("reallocatePartitionGroup failed.")); + } else { + // double check. since if reallocate find mapperEnd, it can fast skip. + if (mapperEnded(shuffleId, mapperId, mapperAttemptId)) { + // mapper ended skip retry + logger.warn(String.format("Skip retry since Hit StageEnded " + + "appId %s shuffleId %s mapperId %s mapperAttemptId %s groupId %s", + applicationId, shuffleId, mapperId, mapperAttemptId, pgId)); + } else { + logger.info(String.format("Enter %s retry for " + + "appId %s shuffleId %s mapperId %s mapperAttemptId %s groupId %s", + retryCount, applicationId, shuffleId, mapperId, mapperAttemptId, pgId)); + isFinished = false; + PartitionGroup newPartitionGroup = shufflePartitionGroupMap.get(shuffleId).getGroup(pgId); + pushDataRetryThreadPool.submit(() -> { + try { + asyncBatchPushDataWithRetry(newPartitionGroup, applicationId, shuffleId, mapperId, mapperAttemptId, + reqBatchId, partitionBatchIds, reducerIdArray, compressData, offsets, status, retryCount + 1, + originTotalLength, originTotalWriteBytes); + } catch (IOException e) { + status.setException(e); + status.InFlightReqs.decrementAndGet(); + } catch (Throwable throwable) { + status.setException(new IOException(throwable)); + status.InFlightReqs.decrementAndGet(); + } + }); + } + } + } else { + status.setException(new IOException("reallocatePartitionGroup retry still failed")); + } + } + if (isFinished) { + pushDataRawSize.update(originTotalLength); + pushDataSize.update(originTotalWriteBytes); + batchPushThroughput.mark(originTotalWriteBytes * 2L); + status.InFlightReqs.decrementAndGet(); + } + }); + } + + private boolean shouldRetry(Throwable e, int retryCount) { + boolean isIOException = e instanceof IOException || + (e.getCause() != null && e.getCause() instanceof IOException); + boolean hasRemainingRetries = retryCount < pushIoMaxRetries; + return isIOException && hasRemainingRetries; + } + + private TransportClient createClientWithRetry(String remoteHost, int remotePort, int seed) throws Exception { + TransportClient client = null; + int retryCount = 0; + while (true) { + try { + client = clientFactory.createClient(remoteHost, remotePort, seed); + return client; + } catch (Exception ex) { + if (shouldRetry(ex, retryCount)) { + retryCount ++; + logger.warn("createClient with {}:{} with seed {} failed, retry for the {} time", + remoteHost, remotePort, seed, retryCount); + Uninterruptibles.sleepUninterruptibly(pushIoRetryWaitMs, TimeUnit.MILLISECONDS); + } else { + logger.error("createClient with {}:{} with seed {} failed, will not retry again", + remoteHost, remotePort, seed); + throw ex; + } + } + } + } + + /** + * When the reallocate process is encountered during the data transmission process. + * + * It is necessary to wait until the new worker node group is reallocated + * before continuing the process of data writing. + */ + private void waitUntilReallocatePartitionGroupEnded( + String applicationId, + int shuffleId, + int groupId) throws IOException { + String groupKey = Utils.getPartitionGroupKey(applicationId, shuffleId, groupId); + if (reallocating.contains(groupKey)) { + reallocatingLock.lock(); + // if current partition group is under reallocation, better wait until reallocation done. + try { + while (reallocating.contains(groupKey)) { + try { + pushDataAwaitCondition.await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException(e); + } + } + } finally { + reallocatingLock.unlock(); + } + } + } + + /** + * When push data failed or worker notify should write rotate. + * + * Reallocate new worker pairs for the remaining data to be written. + * multi-threaded reallocate control, block push data requests during reallocate. + * if a new epoch has been allocated, use the new epoch directly. + */ + private boolean reallocatePartitionGroupWithRetry( + String applicationId, + int shuffleId, + int mapperId, + int mapperAttemptId, + PartitionGroup partitionGroup) { + String groupKey = Utils.getPartitionGroupKey(applicationId, shuffleId, partitionGroup.partitionGroupId); + String mapperKey = Utils.getMapperKey(shuffleId, mapperId, mapperAttemptId); + + reallocatingLock.lock(); + try { + while (reallocating.contains(groupKey)) { + try { + reallocatingAwaitCondition.await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + + PartitionGroup current = shufflePartitionGroupMap.get(shuffleId).getGroup(partitionGroup.partitionGroupId); + if (mapperEnded(shuffleId, mapperId, mapperAttemptId)) { + logger.info( + String.format("MapperEnded for %s-%s-%s-%s--%s during reallocation", + applicationId, shuffleId, mapperId, mapperAttemptId, + partitionGroup.partitionGroupId, partitionGroup.epochId)); + return true; + } else if (current == null) { + throw new RuntimeException("ReallocatePartitionGroup current NPE."); + } else if (current.epochId == partitionGroup.epochId) { + logger.info( + String.format("Enter allocate state for %s-%s-%s-%s--%s", + applicationId, shuffleId, mapperId, mapperAttemptId, + partitionGroup.partitionGroupId, partitionGroup.epochId)); + reallocating.add(groupKey); + } else { + logger.info( + String.format("New group already allocated for %s-%s-%s-%s--%s using new epoch %s", + applicationId, shuffleId, mapperId, mapperAttemptId, + partitionGroup.partitionGroupId, partitionGroup.epochId, current.epochId)); + return true; + } + } finally { + if (!reallocating.contains(groupKey) && !reallocatingLock.hasWaiters(reallocatingAwaitCondition)) { + pushDataAwaitCondition.signalAll(); + } + reallocatingLock.unlock(); + } + + int numFailures = 0; + long waitIntervalMs = CssConf.clientReallocateRetryIntervalMs(cssConf); + try { + while (numFailures < reallocateFailedMaxTimes) { + try { + logger.info( + String.format("Executing reallocate operation for %s-%s-%s-%s--%s, so far %s times", + applicationId, shuffleId, mapperId, mapperAttemptId, + partitionGroup.partitionGroupId, partitionGroup.epochId, numFailures)); + + CssRpcMessage.ReallocatePartitionGroupResponse response = masterRpcRef.askSync(new CssRpcMessage + .ReallocatePartitionGroup(applicationId, shuffleId, mapperId, mapperAttemptId, partitionGroup), + ClassTag$.MODULE$.apply(CssRpcMessage.ReallocatePartitionGroupResponse.class) + ); + + if (response.statusCode().equals(CssStatusCode.Success)) { + shufflePartitionGroupMap.get(shuffleId) + .updateGroup(partitionGroup.partitionGroupId, response.partitionGroup()); + return true; + } else if (response.statusCode().equals(CssStatusCode.MapEnded)) { + mapperEndMap.computeIfAbsent(shuffleId, (id) -> new ConcurrentSet<>()).add(mapperKey); + return true; + } else { + logger.error( + String.format("reallocatePartitionGroup failed for %s-%s-%s-%s--%s, so far %s times", + applicationId, shuffleId, mapperId, mapperAttemptId, + partitionGroup.partitionGroupId, partitionGroup.epochId, numFailures)); + numFailures++; + } + } catch (Exception e) { + logger.warn( + String.format("reallocatePartitionGroup failed with exception, so far %s times", numFailures), e); + numFailures++; + } + TimeUnit.MILLISECONDS.sleep(waitIntervalMs); + } + String err_msg = String.format("reallocatePartitionGroup failed after retry %s times", reallocateFailedMaxTimes); + logger.warn(err_msg); + throw new IOException(err_msg); + } catch (Exception ex) { + logger.error( + String.format("reallocatePartitionGroup failed with exception for %s-%s-%s-%s--%s", + applicationId, shuffleId, mapperId, mapperAttemptId, + partitionGroup.partitionGroupId, partitionGroup.epochId), ex); + return false; + } finally { + reallocatingLock.lock(); + try { + reallocating.remove(groupKey); + if (!reallocating.contains(groupKey) && !reallocatingLock.hasWaiters(reallocatingAwaitCondition)) { + pushDataAwaitCondition.signalAll(); + } + reallocatingAwaitCondition.signalAll(); + } finally { + reallocatingLock.unlock(); + } + } + } + + private byte[] addHeaderAndCompress( + int mapperId, + int mapperAttemptId, + int batchId, + byte[] originalBytes, + int offset, + int length) { + Compressor compressor = compressorThreadLocal.get(); + compressor.compress(originalBytes, offset, length); + int compressedTotalSize = compressor.getCompressedTotalSize(); + return addHeader(mapperId, mapperAttemptId, batchId, compressor.getCompressedBuffer(), 0, compressedTotalSize); + } + + private byte[] addHeader( + int mapperId, + int mapperAttemptId, + int batchId, + byte[] bytes, + int offset, + int length) { + int BATCH_HEADER_SIZE = 4 * 4; + byte[] body = new byte[BATCH_HEADER_SIZE + length]; + Platform.putInt(body, Platform.BYTE_ARRAY_OFFSET, mapperId); + Platform.putInt(body, Platform.BYTE_ARRAY_OFFSET + 4, mapperAttemptId); + Platform.putInt(body, Platform.BYTE_ARRAY_OFFSET + 8, batchId); + Platform.putInt(body, Platform.BYTE_ARRAY_OFFSET + 12, length); + System.arraycopy(bytes, offset, body, BATCH_HEADER_SIZE, length); + return body; + } + + private boolean mapperEnded(int shuffleId, int mapId, int attemptId) { + return mapperEndMap.containsKey(shuffleId) && + mapperEndMap.get(shuffleId).contains(Utils.getMapperKey(shuffleId, mapId, attemptId)); + } + + private void waitUntilZeroInFlightReqs(MapperAttemptStatus status) throws IOException { + if (status.getException() != null){ + throw status.getException(); + } + + long deltaMs = CssConf.clientMapperEndSleepDeltaMs(cssConf); + long timeoutMs = CssConf.clientMapperEndTimeoutMs(cssConf); + + try { + while (status.InFlightReqs.get() > 0 && timeoutMs > 0 && status.getException() == null) { + TimeUnit.MILLISECONDS.sleep(deltaMs); + timeoutMs -= deltaMs; + } + } catch (InterruptedException e) { + status.setException(new IOException(e)); + } + + if (status.getException() != null) { + throw status.getException(); + } + if (status.InFlightReqs.get() > 0) { + throw new IOException("Client MapperEnd timeout, you might need to increase css.client.mapper.end.timeout"); + } + if (status.getException() != null) { + throw status.getException(); + } + } + + @Override + public void mapperEnd( + String applicationId, + int shuffleId, + int mapperId, + int mapperAttemptId, + int numMappers) throws IOException { + String mapperKey = Utils.getMapperKey(shuffleId, mapperId, mapperAttemptId); + MapperAttemptStatus mapperAttemptStatus = statusMap.computeIfAbsent(mapperKey, (s) -> new MapperAttemptStatus()); + + try { + Timer.Context mapperEndTimer = mapperEndWaitTimer.time(); + waitUntilZeroInFlightReqs(mapperAttemptStatus); + mapperEndTimer.stop(); + List epochList = new ArrayList<>(); + epochList.addAll(mapperAttemptStatus.writtenEpochSet); + List batchBlacklist = this.failedBatchBlacklistEnable ? + new ArrayList<>(mapperAttemptStatus.failedBatchBlacklist) : null; + CssRpcMessage.MapperEndResponse response = masterRpcRef.askSync(new CssRpcMessage + .MapperEnd(applicationId, shuffleId, mapperId, mapperAttemptId, numMappers, epochList, batchBlacklist), + ClassTag$.MODULE$.apply(CssRpcMessage.MapperEndResponse.class)); + if (response.statusCode() != CssStatusCode.Success) { + String msg = String.format("MapperEnd for %s failed! StatusCode: %s", mapperKey, response.statusCode()); + logger.error(msg); + throw new IOException(msg); + } + } finally { + statusMap.remove(mapperKey); + } + } + + @Override + public void mapperClose(String applicationId, int shuffleId, int mapperId, int mapperAttemptId) { + String mapperKey = Utils.getMapperKey(shuffleId, mapperId, mapperAttemptId); + MapperAttemptStatus mapperAttemptStatus = statusMap.remove(mapperKey); + if (mapperAttemptStatus != null) { + mapperAttemptStatus.setException(new IOException("MapperClose accidentally")); + } + } + + @Override + public CssRemoteDiskEpochReader createEpochReader( + String applicationId, + int shuffleId, + List partitions, + CssConf conf) throws IOException { + String shuffleKey = Utils.getShuffleKey(applicationId, shuffleId); + return new CssRemoteDiskEpochReader( + cssConf, + clientFactory, + shuffleKey, + partitions.toArray(new CommittedPartitionInfo[0])); + } + + @Override + public List getPartitionInfos( + String applicationId, + int shuffleId, + int[] reduceIds, + int startMapIndex, + int endMapIndex) throws IOException { + + triggerReadPartitionGroupsIfNeeded(applicationId, shuffleId); + + List partitions = new ArrayList(); + for (int reduceId : reduceIds) { + CommittedPartitionInfo[] reducePartitions = reducerFileGroups.get(shuffleId).get(reduceId); + if (reducePartitions != null) { + partitions.addAll(Arrays.asList(reducePartitions)); + } + } + return partitions; + } + + /** + * Multi-threaded read partition control to avoid excessive rpc requests. + */ + private void triggerReadPartitionGroupsIfNeeded(String applicationId, int shuffleId) throws IOException { + if (!reducerFileGroups.containsKey(shuffleId)) { + synchronized (reading) { + while (reading.contains(shuffleId)) { + try { + reading.wait(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + IOException ioe = new IOException(e); + throw ioe; + } + } + if (reducerFileGroups.get(shuffleId) == null) { + reading.add(shuffleId); + } + } + try { + if (!reducerFileGroups.containsKey(shuffleId)) { + getReadPartitionGroupsWithRetry(applicationId, shuffleId); + } + } finally { + synchronized (reading) { + reading.remove(shuffleId); + reading.notifyAll(); + } + } + } + } + + /** + * Because the stage end is triggered asynchronously, when the reduceTask data is read. + * there will be a situation where the mapStage has not completely ended. + * and we need to continue to wait until the stage end is completed. + */ + private void getReadPartitionGroupsWithRetry(String applicationId, int shuffleId) throws IOException { + long waitIntervalMs = CssConf.stageEndRetryIntervalMs(cssConf); + Timer.Context reducerFileGroupResponseTimer = reducerFileGroupsTimer.time(); + while (!(reducerFileGroups.containsKey(shuffleId) && mapperAttempts.containsKey(shuffleId))) { + CssRpcMessage.GetReducerFileGroupsResponse response = masterRpcRef.askSync(new CssRpcMessage + .GetReducerFileGroups(applicationId, shuffleId), + ClassTag$.MODULE$.apply(CssRpcMessage.GetReducerFileGroupsResponse.class)); + + if (response.status() == CssStatusCode.Waiting) { + try { + logger.info("StageEndWaiting wait {} ms for {}-{}", waitIntervalMs, applicationId, shuffleId); + TimeUnit.MILLISECONDS.sleep(waitIntervalMs); + } catch (InterruptedException ex) { + logger.warn("GetReducerFileGroups await failed.", ex); + Thread.currentThread().interrupt(); + } + } else if (response.status() == CssStatusCode.Timeout) { + String errMsg = String.format("StageEndTimeout for %s-%s", applicationId, shuffleId); + throw new IOException(errMsg); + } else if (response.status() == CssStatusCode.StageEndDataLost) { + dataLostCounter.inc(); + logger.error("shuffle {} read partition failed for data lost", shuffleId); + throw new IOException(String.format("shuffle %s read partition failed for data lost", shuffleId)); + } else if (response.status() == CssStatusCode.Failed) { + logger.error("shuffle {} read partition failed for never trigger stage end", shuffleId); + throw new IOException( + String.format("shuffle %s read partition failed for never trigger stage end", shuffleId)); + } else if (response.status() == CssStatusCode.Success) { + if (response.fileGroup() == null || response.fileGroup().length == 0) { + mapperAttempts.put(shuffleId, response.attempts()); + reducerFileGroups.put(shuffleId, new ConcurrentHashMap<>()); + logger.info(String.format("get read empty partition groups for shuffleId %s", shuffleId)); + } else { + CommittedPartitionInfo[][] reducerCommitPartitions = response.fileGroup(); + ConcurrentHashMap map = new ConcurrentHashMap<>(); + for (int i = 0; i < reducerCommitPartitions.length; i++) { + map.put(i, reducerCommitPartitions[i]); + } + if (response.batchBlacklist() != null) { + batchBlacklistMap.put(shuffleId, response.batchBlacklist()); + } + mapperAttempts.put(shuffleId, response.attempts()); + reducerFileGroups.put(shuffleId, map); + logger.info(String.format("get read partition groups success for shuffleId %s", shuffleId)); + } + } + } + reducerFileGroupResponseTimer.stop(); + } + + @Override + public CssInputStream readPartitions( + String applicationId, + int shuffleId, + int[] reduceIds, + int startMapIndex, + int endMapIndex) throws IOException { + String shuffleKey = Utils.getShuffleKey(applicationId, shuffleId); + List partitions = + getPartitionInfos(applicationId, shuffleId, reduceIds, startMapIndex, endMapIndex); + int[] mapperAttemptIds = mapperAttempts.get(shuffleId); + Set failedBatchBlacklist = null; + if (batchBlacklistMap.get(shuffleId) != null) { + failedBatchBlacklist = batchBlacklistMap.get(shuffleId).stream().map(x -> + String.format("%s-%s-%s-%s-%s", + x.getReducerId(), x.getEpochId(), x.getMapId(), x.getAttemptId(), x.getBatchId())) + .collect(Collectors.toSet()); + } + return CssInputStream.create( + cssConf, + clientFactory, + shuffleKey, + partitions.toArray(new CommittedPartitionInfo[partitions.size()]), + mapperAttemptIds, + failedBatchBlacklist, + startMapIndex, + endMapIndex); + } + + @Override + public int[] getMapperAttempts(int shuffleId) { + return mapperAttempts.get(shuffleId); + } + + @Override + public List registerPartitionGroup( + String applicationId, + int shuffleId, + int numMappers, + int numPartitions, + int maxPartitionsPerGroup) throws IOException { + CssRpcMessage.RegisterPartitionGroupResponse response = masterRpcRef.askSync(new CssRpcMessage + .RegisterPartitionGroup(applicationId, shuffleId, numMappers, numPartitions, maxPartitionsPerGroup), + ClassTag$.MODULE$.apply(CssRpcMessage.RegisterPartitionGroupResponse.class)); + if (response.statusCode() == CssStatusCode.Success) { + applyShufflePartitionGroup(shuffleId, response.partitionGroups()); + return response.partitionGroups(); + } else { + throw new IOException("registerPartitionGroup failed"); + } + } + + @Override + public void unregisterShuffle(String applicationId, int shuffleId, boolean isDriver) { + if (isDriver) { + // Only driver need to send out unregisterShuffle + masterRpcRef.askSync(new CssRpcMessage.UnregisterShuffle(applicationId, shuffleId), + ClassTag$.MODULE$.apply(CssRpcMessage.UnregisterShuffleResponse.class)); + logger.info("ShuffleClient UnregisterShuffle for appId: {} shuffleId: {} success.", applicationId, shuffleId); + } + + mapperEndMap.remove(shuffleId); + reducerFileGroups.remove(shuffleId); + mapperAttempts.remove(shuffleId); + batchBlacklistMap.remove(shuffleId); + registering.remove(shuffleId); + reading.remove(shuffleId); + } + + @Override + public void registerApplication(String applicationId) { + synchronized (appHeartBeatLock) { + if (!appHeartBeatStarted) { + RpcEndpointRef heartbeatRef = rpcEnv.setupEndpointRef( + RpcAddress.fromCssURL(CssConf.masterAddress(cssConf)), RpcNameConstants.HEARTBEAT); + this.appHeartBeatThread = new Thread(() -> { + long appReportTimeMs = CssConf.appTimeoutMs(cssConf) / 4; + while (!Thread.currentThread().isInterrupted()) { + try { + heartbeatRef.send(new CssRpcMessage.HeartbeatFromApp(applicationId)); + Thread.sleep(appReportTimeMs); + } catch (InterruptedException ex) { + logger.info("Application might be shutdown, interrupted heartbeat thread."); + } catch (Exception ex) { + logger.error(String.format("Send heartbeat to Master failed with appId: %s", applicationId), ex); + } + } + }); + this.appHeartBeatThread.setDaemon(true); + this.appHeartBeatThread.setName("ShuffleClient application heartbeat thread."); + this.appHeartBeatThread.start(); + this.appHeartBeatStarted = true; + } + } + } + + @Override + public void shutDown() { + if (appHeartBeatThread != null) { + appHeartBeatThread.interrupt(); + } + + if (pushDataRetryThreadPool != null) { + pushDataRetryThreadPool.shutdown(); + } + + if (rpcEnv != null) { + rpcEnv.shutdown(); + } + + if (clientFactory != null) { + clientFactory.close(); + } + + // State cleanup + statusMap.clear(); + mapperEndMap.clear(); + reducerFileGroups.clear(); + mapperAttempts.clear(); + batchBlacklistMap.clear(); + registering.clear(); + reallocating.clear(); + reading.clear(); + limiters.clear(); + shuffleClient = null; + } + + public TransportClientFactory getClientFactory() { + return clientFactory; + } +} diff --git a/client/src/main/java/com/bytedance/css/client/metrics/BaseSource.java b/client/src/main/java/com/bytedance/css/client/metrics/BaseSource.java new file mode 100644 index 0000000..f9faa23 --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/metrics/BaseSource.java @@ -0,0 +1,47 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.metrics; + +import com.bytedance.css.common.metrics.source.Source; +import com.codahale.metrics.MetricRegistry; + +/** + * Client Side Base Metrics Source. + */ +public abstract class BaseSource implements Source { + + protected MetricRegistry registry = new MetricRegistry(); + protected String namespace; + protected String application; + + public BaseSource(String namespace, String application) { + this.namespace = namespace; + this.application = application; + } + + public String cssMetricsPrefix() { + return String.format("namespace=%s|applicationId=%s", namespace, application); + } + + @Override + public MetricRegistry metricRegistry() { + return registry; + } +} diff --git a/client/src/main/java/com/bytedance/css/client/metrics/ClientSource.java b/client/src/main/java/com/bytedance/css/client/metrics/ClientSource.java new file mode 100644 index 0000000..aa747a3 --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/metrics/ClientSource.java @@ -0,0 +1,86 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.metrics; + +import com.codahale.metrics.Counter; +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Meter; +import com.codahale.metrics.Timer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ClientSource extends BaseSource { + + private static final Logger logger = LoggerFactory.getLogger(ClientSource.class); + + private static volatile ClientSource clientSource; + + public static ClientSource instance() { + return instance("dummy", "test"); + } + + public static ClientSource instance(String namespace, String application) { + if (clientSource == null) { + synchronized (ClientSource.class) { + if (clientSource == null) { + clientSource = new ClientSource(namespace, application); + logger.info(String.format("%s use cssMetricsPrefix %s", + clientSource.getClass().getName(), clientSource.cssMetricsPrefix())); + } + } + } + return clientSource; + } + + private ClientSource(String namespace, String application) { + super(namespace, application); + initMetricSet(); + } + + public Meter batchPushThroughput; + public Timer mapperEndWaitTimer; + public Timer reducerFileGroupsTimer; + public Counter dataLostCounter; + public Timer pushDataWaitTimer; + public Histogram pushDataRawSizeHistogram; + public Histogram pushDataSizeHistogram; + + private void initMetricSet() { + batchPushThroughput = registry.meter( + String.format("%s#css.v2.client.batchPush.throughput", cssMetricsPrefix())); + mapperEndWaitTimer = registry.timer( + String.format("%s#css.v2.client.mapper.end.wait.time", cssMetricsPrefix())); + reducerFileGroupsTimer = registry.timer( + String.format("%s#css.v2.client.reducerFileGroups.get.time", cssMetricsPrefix())); + dataLostCounter = registry.counter( + String.format("%s#css.v2.client.dataLost.count", cssMetricsPrefix())); + pushDataWaitTimer = registry.timer( + String.format("%s#css.v2.client.pushData.wait.time", cssMetricsPrefix())); + pushDataRawSizeHistogram = registry.histogram( + String.format("%s#css.v2.client.pushData.rawSize", cssMetricsPrefix())); + pushDataSizeHistogram = registry.histogram( + String.format("%s#css.v2.client.pushData.size", cssMetricsPrefix())); + } + + @Override + public String sourceName() { + return ""; + } +} diff --git a/client/src/main/java/com/bytedance/css/client/stream/CssInputStream.java b/client/src/main/java/com/bytedance/css/client/stream/CssInputStream.java new file mode 100644 index 0000000..f2de2c6 --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/stream/CssInputStream.java @@ -0,0 +1,173 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.stream; + +import com.bytedance.css.client.MetricsCallback; +import com.bytedance.css.client.stream.disk.EpochDiskInputStreamBuilder; +import com.bytedance.css.client.stream.hdfs.EpochHdfsInputStreamBuilder; +import com.bytedance.css.common.CssConf; +import com.bytedance.css.common.protocol.CommittedPartitionInfo; +import com.bytedance.css.common.protocol.PartitionInfo; +import com.bytedance.css.common.protocol.ShuffleMode; +import com.bytedance.css.network.client.TransportClientFactory; +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.util.*; + +public abstract class CssInputStream extends InputStream { + + private static final Logger logger = LoggerFactory.getLogger(CssInputStream.class); + + // for spark metrics recording + public abstract void setCallback(MetricsCallback callback); + + public static CssInputStream create( + CssConf conf, + TransportClientFactory clientFactory, + String shuffleKey, + CommittedPartitionInfo[] partitions, + int[] mapperAttemptIds, + Set failedBatchBlacklist, + int startMapIndex, + int endMapIndex) throws IOException { + + CommittedPartitionInfo[] filterPartitions = partitions; + + if (startMapIndex > endMapIndex) { + // if startMapIndex > endMapIndex + // startMapIndex & endMapIndex can't be used as partition filter + // since it's totally different meaning at this mode + // let's see there are 0-0 0-1 0-2 0-3 0-4 0-5 0-6 0-7 0-8 8 epoch partition + // and skew partition split to 5 task + // task 0 should read 0-0 0-5 + // task 1 should read 0-1 0-6 + // task 2 should read 0-2 0-7 + // task 3 should read 0-3 + // task 4 should read 0-4 + // the task num might be bigger that epoch num for now. + if (!CssConf.testMode(conf) && !CssConf.failedBatchBlacklistEnable(conf)) { + throw new IOException( + "SinglePartition split read mode need to set css.client.failed.batch.blacklist.enabled to true"); + } + if (partitions != null && partitions.length != 0) { + Set sortSet = new TreeSet<>((o1, o2) -> { + if (o1.getReducerId() > o2.getReducerId()) { + return 1; + } else if (o1.getReducerId() < o2.getReducerId()) { + return -1; + } else { + return o1.getEpochId() - o2.getEpochId(); + } + }); + sortSet.addAll(Arrays.asList(partitions)); + CommittedPartitionInfo[] orderedEpochKeys = sortSet.toArray(new CommittedPartitionInfo[0]); + + int startIndex = endMapIndex; + int stepLen = startMapIndex; + Set assignedEpochKeys = new HashSet<>(); + while (startIndex < orderedEpochKeys.length) { + assignedEpochKeys.add(orderedEpochKeys[startIndex]); + startIndex += stepLen; + } + + filterPartitions = assignedEpochKeys.toArray(new CommittedPartitionInfo[0]); + + logger.info("orderedEpochKeys {} s{} e{} taskIndex{} step{} assignedEpochKeys {} filterPartitions {}", + StringUtils.join(Arrays.stream(orderedEpochKeys) + .map(PartitionInfo::getEpochKey).toArray(String[]::new), ","), + startMapIndex, endMapIndex, endMapIndex, stepLen, + StringUtils.join(Arrays.stream(assignedEpochKeys.toArray(new CommittedPartitionInfo[0])) + .map(PartitionInfo::getEpochKey).toArray(String[]::new), ","), + StringUtils.join(Arrays.stream(filterPartitions) + .map(PartitionInfo::getEpochKey).toArray(String[]::new), ",")); + + if (partitions != null && filterPartitions != null && partitions.length != filterPartitions.length) { + logger.info("After indexBasedSplitEnabled filtered, original {}, current {}", + partitions.length, filterPartitions.length); + } + } + } + + if (filterPartitions == null || filterPartitions.length == 0) { + return emptyInputStream; + } else { + EpochInputStreamBuilder builder; + if (filterPartitions[0].getShuffleMode() == ShuffleMode.DISK) { + builder = new EpochDiskInputStreamBuilder(clientFactory, shuffleKey, conf); + } else { + FileSystem fs = new Path(filterPartitions[0].getFilePath()).getFileSystem(new Configuration()); + builder = new EpochHdfsInputStreamBuilder(fs); + } + return + new CssInputStreamImpl( + conf, + shuffleKey, + filterPartitions, + builder, + mapperAttemptIds, + failedBatchBlacklist, + startMapIndex, + endMapIndex + ); + } + } + + public static Iterator shuffleEpochOrder( + LinkedHashMap> epochMap) { + // random shuffle for replica epoch + epochMap.entrySet().stream().forEach(entry -> { + Collections.shuffle(entry.getValue()); + }); + + // random shuffle for epoch map + List shuffledEpochKeys = new ArrayList<>(epochMap.keySet()); + Collections.shuffle(shuffledEpochKeys); + logger.info("After epochKey shuffle: {}", StringUtils.join(shuffledEpochKeys, ",")); + + return shuffledEpochKeys.iterator(); + } + + public static CssInputStream empty() { + return emptyInputStream; + } + + private static final CssInputStream emptyInputStream = new CssInputStream() { + @Override + public int read() throws IOException { + return -1; + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + return -1; + } + + @Override + public void setCallback(MetricsCallback callback) { + } + }; +} diff --git a/client/src/main/java/com/bytedance/css/client/stream/CssInputStreamImpl.java b/client/src/main/java/com/bytedance/css/client/stream/CssInputStreamImpl.java new file mode 100644 index 0000000..b5422b9 --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/stream/CssInputStreamImpl.java @@ -0,0 +1,245 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.stream; + +import com.bytedance.css.client.MetricsCallback; +import com.bytedance.css.client.compress.*; +import com.bytedance.css.common.CssConf; +import com.bytedance.css.common.protocol.CommittedPartitionInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.DataInputStream; +import java.io.IOException; +import java.util.*; +import java.util.concurrent.TimeUnit; + +public class CssInputStreamImpl extends CssInputStream { + + private static final Logger logger = LoggerFactory.getLogger(CssInputStreamImpl.class); + + private final EpochInputStreamBuilder inputStreamBuilder; + + private final String shuffleKey; + private final int[] mapperAttemptIds; + private final Set failedBatchBlacklist; + private final boolean failedBatchBlacklistEnable; + private final int startMapIndex; + private final int endMapIndex; + private final boolean indexBasedSplitEnabled; + + private final FrameIterator frameIterator; + private int position; + private int limit; + + private Iterator iterator = null; + private String currentEpochKey; + + private byte[] decompressedBuf; + private final CompressorFactory compressorFactory; + private final Decompressor decompressor; + + private LinkedHashMap> epochMap = new LinkedHashMap<>(); + private final Map> batchesRead = new HashMap<>(); + private MetricsCallback callback; + private byte[] compressedBuf; + + public CssInputStreamImpl( + CssConf conf, + String shuffleKey, + CommittedPartitionInfo[] partitions, + EpochInputStreamBuilder inputStreamBuilder, + int[] mapperAttemptIds, + Set failedBatchBlacklist, + int startMapIndex, + int endMapIndex) throws IOException { + this.shuffleKey = shuffleKey; + this.mapperAttemptIds = mapperAttemptIds; + this.failedBatchBlacklist = failedBatchBlacklist; + this.failedBatchBlacklistEnable = this.failedBatchBlacklist != null && this.failedBatchBlacklist.size() > 0; + this.startMapIndex = startMapIndex; + this.endMapIndex = endMapIndex; + this.indexBasedSplitEnabled = startMapIndex > endMapIndex; + + this.inputStreamBuilder = inputStreamBuilder; + for (int i = 0; i < partitions.length; i ++) { + String key = partitions[i].getEpochKey(); + epochMap.putIfAbsent(key, new ArrayList<>()); + epochMap.get(key).add(partitions[i]); + } + + iterator = shuffleEpochOrder(epochMap); + + // initialize length according to the push buffer size + // if giant record met, recreate compressedBuf and decompressedBuf are necessary + int blockInitLength = ((int)(CssConf.pushBufferSize(conf))) + CssCompressorTrait.HEADER_LENGTH; + frameIterator = new FrameIterator(); + compressedBuf = new byte[blockInitLength]; + decompressedBuf = new byte[blockInitLength]; + this.compressorFactory = new CssCompressorFactory(conf); + this.decompressor = this.compressorFactory.getDecompressor(); + + // init first chunk for read() + nextEpoch(); + } + + @Override + public void setCallback(MetricsCallback callback) { + this.callback = callback; + } + + @Override + public int read() throws IOException { + if (position < limit) { + int b = decompressedBuf[position]; + position++; + return b & 0xFF; + } else { + if (!processBatch()) { + return -1; + } else { + // if processBatch return true, position & limit will be set to valid index, call read() again. + return read(); + } + } + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + if (b == null) { + throw new NullPointerException(); + } else if (off < 0 || len < 0 || len > b.length - off) { + throw new IndexOutOfBoundsException(); + } else if (len == 0) { + return 0; + } + + int readBytes = 0; + while (readBytes < len) { + while (position >= limit) { + if (!processBatch()) { + return readBytes > 0 ? readBytes : -1; + } + } + + int bytesToRead = Math.min(limit - position, len - readBytes); + System.arraycopy(decompressedBuf, position, b, off + readBytes, bytesToRead); + position += bytesToRead; + readBytes += bytesToRead; + } + + return readBytes; + } + + @Override + public void close() throws IOException { + frameIterator.close(); + } + + // using bytes data in currentChunk + // fetch HEADER: mapId + attemptId + batchId + size + // and doing decompress to get origin bytes data which ShuffleWriter has been written. + private boolean processBatch() throws IOException { + + long startTime = System.nanoTime(); + + boolean hasData = false; + while (hasNextFrame()) { + Frame frame = frameIterator.next(); + + int mapId = frame.getMapperId(); + int attemptId = frame.getAttemptId(); + int batchId = frame.getBatchId(); + int size = frame.getDataLength(); + + if (compressedBuf.length < size) { + compressedBuf = new byte[size]; + } + frame.getData().readFully(compressedBuf, 0, size); + + // de-duplicate + if (attemptId == mapperAttemptIds[mapId] && + (indexBasedSplitEnabled || (mapId >= startMapIndex && mapId < endMapIndex))) { + if (failedBatchBlacklistEnable) { + String blacklistKey = String.format("%s-%s-%s-%s", currentEpochKey, mapId, attemptId, batchId); + if (failedBatchBlacklist.contains(blacklistKey)) { + logger.warn("duplicated batch for failedBatchBlacklist: " + blacklistKey); + continue; + } + } + + if (!batchesRead.containsKey(mapId)) { + Set batchSet = new HashSet<>(); + batchesRead.put(mapId, batchSet); + } + Set batchSet = batchesRead.get(mapId); + if (!batchSet.contains(batchId)) { + batchSet.add(batchId); + if (callback != null) { + callback.incBytesRead(frame.getFrameLength()); + } + // decompress data + int originalLength = decompressor.getOriginalLen(compressedBuf); + if (decompressedBuf.length < originalLength) { + decompressedBuf = new byte[originalLength]; + } + // TODO decompress data while read compress data could save byte array copy one time here + limit = decompressor.decompress(compressedBuf, decompressedBuf, 0); + position = 0; + hasData = true; + break; + } else { + logger.warn("duplicated batch: mapId " + mapId + ", attemptId " + + attemptId + ", batchId " + batchId); + } + } + } + + if (callback != null) { + callback.incReadTime(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTime)); + } + return hasData; + } + + private void nextEpoch() throws IOException { + currentEpochKey = iterator.next(); + logger.info(String.format("Move to next Epoch %s for shuffle %s.", currentEpochKey, shuffleKey)); + ArrayList partitions = epochMap.get(currentEpochKey); + + DataInputStream inputStream = inputStreamBuilder.createInputStream(partitions); + + if (inputStream == null) { + String errMsg = String.format( + "Failed to open all replica Epoch %s for shuffle %s.",currentEpochKey, shuffleKey); + throw new IOException(errMsg); + } + frameIterator.resetInputStream(inputStream); + } + + private boolean hasNextFrame() throws IOException { + if (frameIterator.hasNext()) { + return true; + } + if (iterator.hasNext()) { + nextEpoch(); + } + return frameIterator.hasNext(); + } +} diff --git a/client/src/main/java/com/bytedance/css/client/stream/EpochInputStreamBuilder.java b/client/src/main/java/com/bytedance/css/client/stream/EpochInputStreamBuilder.java new file mode 100644 index 0000000..b8317f7 --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/stream/EpochInputStreamBuilder.java @@ -0,0 +1,32 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.stream; + +import com.bytedance.css.common.protocol.CommittedPartitionInfo; + +import java.io.DataInputStream; +import java.io.IOException; +import java.util.List; + +public interface EpochInputStreamBuilder { + + DataInputStream createInputStream(List partitions) throws IOException; + +} diff --git a/client/src/main/java/com/bytedance/css/client/stream/Frame.java b/client/src/main/java/com/bytedance/css/client/stream/Frame.java new file mode 100644 index 0000000..8be2c51 --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/stream/Frame.java @@ -0,0 +1,109 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.stream; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.commons.lang.builder.ToStringStyle; + +import java.io.DataInputStream; + +public class Frame { + + private int mapperId; + private int attemptId; + private int batchId; + private int dataLength; + private int frameLength; + private DataInputStream data; + + public Frame( + int mapperId, + int attemptId, + int batchId, + int dataLength, + int frameLength, + DataInputStream data) { + this.mapperId = mapperId; + this.attemptId = attemptId; + this.batchId = batchId; + this.dataLength = dataLength; + this.frameLength = frameLength; + this.data = data; + } + + public int getMapperId() { + return mapperId; + } + + public void setMapperId(int mapperId) { + this.mapperId = mapperId; + } + + public int getAttemptId() { + return attemptId; + } + + public void setAttemptId(int attemptId) { + this.attemptId = attemptId; + } + + public int getBatchId() { + return batchId; + } + + public void setBatchId(int batchId) { + this.batchId = batchId; + } + + public int getDataLength() { + return dataLength; + } + + public void setDataLength(int dataLength) { + this.dataLength = dataLength; + } + + public DataInputStream getData() { + return data; + } + + public void setData(DataInputStream data) { + this.data = data; + } + + public int getFrameLength() { + return frameLength; + } + + public void setFrameLength(int frameLength) { + this.frameLength = frameLength; + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("mapperId", mapperId) + .append("attemptId", attemptId) + .append("batchId", batchId) + .append("dataLength", dataLength) + .append("frameLength", frameLength) + .toString(); + } +} diff --git a/client/src/main/java/com/bytedance/css/client/stream/FrameIterator.java b/client/src/main/java/com/bytedance/css/client/stream/FrameIterator.java new file mode 100644 index 0000000..dc96419 --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/stream/FrameIterator.java @@ -0,0 +1,103 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.stream; + +import com.bytedance.css.common.exception.CssRuntimeException; +import com.bytedance.css.common.unsafe.Platform; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.DataInputStream; +import java.io.EOFException; +import java.io.IOException; +import java.util.Iterator; + +public class FrameIterator implements Iterator { + + private static final Logger logger = LoggerFactory.getLogger(FrameIterator.class); + + private DataInputStream currentInputStream; + // mapId, attemptId, batchId, size + private final int BATCH_HEADER_SIZE; + private final byte[] headerBuf; + private Frame currentFrame; + + public FrameIterator() { + BATCH_HEADER_SIZE = 4 * 4; + headerBuf = new byte[BATCH_HEADER_SIZE]; + } + + @Override + public boolean hasNext() { + if (currentInputStream == null) { + return false; + } + + int size; + try { + currentInputStream.readFully(headerBuf); + size = Platform.getInt(headerBuf, Platform.BYTE_ARRAY_OFFSET + 12); + } catch (IOException e) { + try { + closeInputStream(); + } catch (IOException ex) { + logger.error("Error occur when close inputStream", ex); + } + if (e instanceof EOFException) { + return false; + } + throw new CssRuntimeException("Read frame data failed.", e); + } + + int mapId = Platform.getInt(headerBuf, Platform.BYTE_ARRAY_OFFSET); + int attemptId = Platform.getInt(headerBuf, Platform.BYTE_ARRAY_OFFSET + 4); + int batchId = Platform.getInt(headerBuf, Platform.BYTE_ARRAY_OFFSET + 8); + + currentFrame = new Frame(mapId, attemptId, batchId, size, BATCH_HEADER_SIZE + size, currentInputStream); + return true; + } + + private void closeInputStream() throws IOException { + if (currentInputStream != null) { + currentInputStream.close(); + } + currentInputStream = null; + } + + @Override + public Frame next() { + return currentFrame; + } + + public void resetInputStream(DataInputStream inputStream) throws IOException { + if (this.currentInputStream != null) { + this.currentInputStream.close(); + } + this.currentInputStream = inputStream; + } + + public void close() throws IOException { + if (this.currentInputStream != null) { + this.currentInputStream.close(); + } + this.currentInputStream = null; + } + +} diff --git a/client/src/main/java/com/bytedance/css/client/stream/disk/CssLocalDiskEpochReader.java b/client/src/main/java/com/bytedance/css/client/stream/disk/CssLocalDiskEpochReader.java new file mode 100644 index 0000000..1c1f465 --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/stream/disk/CssLocalDiskEpochReader.java @@ -0,0 +1,69 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.stream.disk; + +import com.bytedance.css.common.protocol.CommittedPartitionInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedInputStream; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; + +public class CssLocalDiskEpochReader extends InputStream { + + private static final Logger logger = LoggerFactory.getLogger(CssLocalDiskEpochReader.class); + + private final String shuffleKey; + private final String epochKey; + private InputStream inputStream; + + public CssLocalDiskEpochReader(String shuffleKey, CommittedPartitionInfo partition) throws IOException { + this.shuffleKey = shuffleKey; + this.epochKey = partition.getEpochKey(); + + try { + inputStream = new BufferedInputStream(new FileInputStream(partition.getFilePath())); + } catch (IOException e) { + throw new IOException(e); + } + logger.info(String.format("read next epoch %s for shuffle %s from local", epochKey, shuffleKey)); + } + + @Override + public int read() throws IOException { + return inputStream.read(); + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + return inputStream.read(b, off, len); + } + + @Override + public void close() throws IOException { + if (inputStream != null) { + logger.info(String.format("Closing local reader for shuffle %s epoch %s", shuffleKey, epochKey)); + inputStream.close(); + inputStream = null; + } + } +} diff --git a/client/src/main/java/com/bytedance/css/client/stream/disk/CssRemoteDiskEpochReader.java b/client/src/main/java/com/bytedance/css/client/stream/disk/CssRemoteDiskEpochReader.java new file mode 100644 index 0000000..376ee1f --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/stream/disk/CssRemoteDiskEpochReader.java @@ -0,0 +1,248 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.stream.disk; + +import com.bytedance.css.common.CssConf; +import com.bytedance.css.common.protocol.CommittedPartitionInfo; +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.buffer.NettyManagedBuffer; +import com.bytedance.css.network.client.ChunkReceivedCallback; +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.client.TransportClientFactory; +import com.bytedance.css.network.protocol.shuffle.BlockTransferMessage; +import com.bytedance.css.network.protocol.shuffle.OpenStream; +import com.bytedance.css.network.protocol.shuffle.StreamHandle; +import io.netty.buffer.ByteBuf; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Iterator; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +public class CssRemoteDiskEpochReader { + + private static final Logger logger = LoggerFactory.getLogger(CssRemoteDiskEpochReader.class); + + private CssConf cssConf; + private TransportClient client; + private TransportClientFactory clientFactory; + + private final boolean chunkFetchRetryEnable; + private final int chunkFetchFailedRetryMaxTimes; + private final long chunkRetryWaitIntervalMs; + private int chunkFetchRetryTimes; + + private final String shuffleKey; + private final long timeoutMs; + private final int maxInFlight; + + private volatile CommittedPartitionInfo active; + private final Iterator iterator; + + private long streamId; + private int numChunks; + + private int returnedChunks; + private int chunkIndex; + + private final LinkedBlockingQueue results; + private ChunkReceivedCallback callback; + + private boolean closed = false; + private final AtomicReference exception = new AtomicReference<>(); + + public CssRemoteDiskEpochReader( + CssConf cssConf, + TransportClientFactory clientFactory, + String shuffleKey, + CommittedPartitionInfo[] partitions) throws IOException { + this.cssConf = cssConf; + this.clientFactory = clientFactory; + this.shuffleKey = shuffleKey; + this.iterator = Arrays.asList(partitions).iterator(); + this.chunkFetchRetryEnable = CssConf.chunkFetchRetryEnable(cssConf); + this.chunkFetchFailedRetryMaxTimes = CssConf.chunkFetchFailedRetryMaxTimes(cssConf); + this.chunkRetryWaitIntervalMs = CssConf.chunkFetchRetryWaitTimes(cssConf); + this.timeoutMs = CssConf.fetchChunkTimeoutMs(cssConf); + this.maxInFlight = CssConf.fetchChunkMaxReqsInFlight(cssConf); + + while (iterator.hasNext()) { + this.active = iterator.next(); + openStream(0); + if (exception.get() == null) { + break; + } + } + if (exception.get() != null) { + String err_msg = String.format("init all active committed partition %s for shuffleKey %s it still failed.", + Arrays.asList(partitions), this.shuffleKey); + throw new IOException(err_msg, exception.get()); + } + + // once fetch success, place ByteBuf into result queue + results = new LinkedBlockingQueue<>(); + callback = new RetryingChunkFetchListener(); + } + + private class RetryingChunkFetchListener implements ChunkReceivedCallback { + @Override + public void onSuccess(int chunkIndex, ManagedBuffer buffer) { + synchronized (CssRemoteDiskEpochReader.this) { + if (callback == this && exception.get() == null) { + ByteBuf buf = ((NettyManagedBuffer) buffer).getBuf(); + if (!closed) { + buf.retain(); + results.add(buf); + } + } else { + logger.warn("ignore chunk fetch result with shuffleKey {} epoch {} partition {} streamId {} chunkIndex {}." + + "because this callback is old.", shuffleKey, active.getEpochKey(), active, streamId, chunkIndex); + } + } + } + + @Override + public void onFailure(int chunkIndex, Throwable e) { + synchronized (CssRemoteDiskEpochReader.this) { + if (callback == this) { + String err_msg = + String.format("fetchChunk failed with shuffleKey %s epoch %s partition %s streamId %s chunkIndex %s." + + "it will retry.", shuffleKey, active.getEpochKey(), active, streamId, chunkIndex); + logger.warn(err_msg, e); + exception.set(new IOException(err_msg, e)); + } else { + logger.warn("ignore chunk fetch failed with shuffleKey {} epoch {} partition {} streamId {} chunkIndex {}." + + "because this callback is old.", shuffleKey, active.getEpochKey(), active, streamId, chunkIndex); + } + } + } + } + + private void openStream(int initChunkIndex) { + try { + this.client = this.clientFactory.createClient(this.active.getHost(), this.active.getPort()); + // request encoding for fetch streamId and numChunks and initChunkIndex + OpenStream openStream = new OpenStream(this.shuffleKey, this.active.getFilePath(), initChunkIndex); + ByteBuffer response = this.client.sendRpcSync(openStream.toByteBuffer(), this.timeoutMs); + StreamHandle streamHandle = (StreamHandle) BlockTransferMessage.Decoder.fromByteBuffer(response); + streamId = streamHandle.streamId; + numChunks = streamHandle.numChunks; + this.exception.set(null); + } catch (Exception e) { + exception.set(new IOException(e)); + logger.warn(String.format("init partition stream error for shuffleKey %s epoch %s partition %s.", + shuffleKey, active.getEpochKey(), active), e); + } + } + + private void tryFetchChunk() throws IOException, InterruptedException { + while (exception.get() != null && chunkFetchRetryEnable) { + boolean wait = false; + synchronized (this) { + if (chunkFetchRetryTimes < chunkFetchFailedRetryMaxTimes) { + this.chunkFetchRetryTimes++; + this.callback = new RetryingChunkFetchListener(); + safeClearResults(); + this.exception.set(null); + this.chunkIndex = returnedChunks; + wait = true; + logger.info("chunk fetch retry with shuffleKey {} epoch {} partition {} so far {} times", + shuffleKey, active.getEpochKey(), active, chunkFetchRetryTimes); + } else if (iterator.hasNext()) { + this.active = iterator.next(); + this.callback = new RetryingChunkFetchListener(); + safeClearResults(); + this.chunkFetchRetryTimes = 0; + this.chunkIndex = 0; + this.returnedChunks = 0; + this.exception.set(null); + logger.info("move to next commit file to chunk fetch retry with shuffleKey {} epoch {} partition {} " + + "so far {} times", shuffleKey, active.getEpochKey(), active, chunkFetchRetryTimes); + } else { + throw new IOException("chunk fetch retry all active committed partition for shuffleKey %s it still failed."); + } + } + if (wait) { + Thread.sleep(chunkRetryWaitIntervalMs); + } + // if chunk retry start from returnedChunks. if file retry start from 0. + openStream(returnedChunks); + } + + // here only check no retry and throw exception. + // for retry times, skip this and let it retry fill max times. + if (exception.get() != null && !chunkFetchRetryEnable) { + throw exception.get(); + } else if (chunkIndex < numChunks) { + fetchChunks(); + } + } + + private void safeClearResults() { + if (results.size() > 0) { + results.forEach(res -> res.release()); + } + results.clear(); + } + + public void close() throws IOException { + synchronized(this) { + closed = true; + } + safeClearResults(); + } + + public boolean hasNext() { + return returnedChunks < numChunks; + } + + public ByteBuf next() throws IOException { + ByteBuf chunk = null; + try { + while (chunk == null) { + tryFetchChunk(); + chunk = results.poll(500, TimeUnit.MILLISECONDS); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + IOException ioe = new IOException(e); + exception.set(ioe); + throw ioe; + } + returnedChunks++; + return chunk; + } + + private void fetchChunks() { + final int inFlight = chunkIndex - returnedChunks; + if (inFlight < maxInFlight) { + final int toFetch = Math.min(maxInFlight - inFlight + 1, numChunks - chunkIndex); + for (int i = 0; i < toFetch; i++) { + client.fetchChunk(streamId, chunkIndex++, callback); + } + } + } + +} diff --git a/client/src/main/java/com/bytedance/css/client/stream/disk/EpochDiskInputStreamBuilder.java b/client/src/main/java/com/bytedance/css/client/stream/disk/EpochDiskInputStreamBuilder.java new file mode 100644 index 0000000..3cdeefe --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/stream/disk/EpochDiskInputStreamBuilder.java @@ -0,0 +1,140 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.stream.disk; + +import com.bytedance.css.client.stream.EpochInputStreamBuilder; +import com.bytedance.css.common.CssConf; +import com.bytedance.css.common.protocol.CommittedPartitionInfo; +import com.bytedance.css.common.util.Utils; +import com.bytedance.css.network.client.TransportClientFactory; +import io.netty.buffer.ByteBuf; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.*; +import java.util.List; + +public class EpochDiskInputStreamBuilder implements EpochInputStreamBuilder { + + private TransportClientFactory clientFactory; + private String shuffleKey; + private CssConf conf; + + public EpochDiskInputStreamBuilder( + TransportClientFactory clientFactory, + String shuffleKey, + CssConf conf) { + this.clientFactory = clientFactory; + this.shuffleKey = shuffleKey; + this.conf = conf; + } + + @Override + public DataInputStream createInputStream(List partitions) throws IOException { + // locality check for replica file. check if it can be read from local disk first + CommittedPartitionInfo localPartitionInfo = partitions.stream() + .filter(partition -> Utils.isLocalBlockFetchable(CssConf.localChunkFetchEnable(conf), partition.getFilePath())) + .findFirst().orElse(null); + if (localPartitionInfo != null) { + return new DataInputStream(new CssLocalDiskEpochReader(shuffleKey, localPartitionInfo)); + } else { + return + new DataInputStream( + new CssRemoteDiskEpochInputStream( + new CssRemoteDiskEpochReader( + conf, + clientFactory, + shuffleKey, + partitions.toArray(new CommittedPartitionInfo[0]) + ) + ) + ); + } + } + + public static class CssRemoteDiskEpochInputStream extends InputStream { + + private static final Logger logger = LoggerFactory.getLogger(CssRemoteDiskEpochInputStream.class); + + private CssRemoteDiskEpochReader currentReader; + private ByteBuf currentChunk; + + public CssRemoteDiskEpochInputStream(CssRemoteDiskEpochReader currentReader) throws IOException { + this.currentReader = currentReader; + this.currentChunk = currentReader.next(); + } + + @Override + public int read() throws IOException { + boolean readable = readable(); + return readable ? currentChunk.readByte() & 255 : -1; + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + boolean readable = this.readable(); + if (!readable) { + return -1; + } else { + currentChunk.readBytes(b, off, len); + return len; + } + } + + private boolean readable() throws IOException { + return currentChunk.isReadable() || nextChunk(); + } + + @Override + public void close() throws IOException { + if (currentChunk != null) { + logger.info("Release chunk!"); + currentChunk.release(); + currentChunk = null; + } + if (currentReader != null) { + logger.info("Closing reader"); + currentReader.close(); + currentReader = null; + } + } + + private boolean nextChunk() throws IOException { + currentChunk.release(); + currentChunk = null; + while (currentChunk == null && currentReader.hasNext()) { + currentChunk = currentReader.next(); + if (!currentChunk.isReadable()) { + currentChunk = null; + } else { + return true; + } + } + + // nextChunk and nextEpochReader EOF + if (currentReader != null) { + currentReader.close(); + currentReader = null; + } + return false; + } + + } +} diff --git a/client/src/main/java/com/bytedance/css/client/stream/hdfs/EpochHdfsInputStreamBuilder.java b/client/src/main/java/com/bytedance/css/client/stream/hdfs/EpochHdfsInputStreamBuilder.java new file mode 100644 index 0000000..fa6bd60 --- /dev/null +++ b/client/src/main/java/com/bytedance/css/client/stream/hdfs/EpochHdfsInputStreamBuilder.java @@ -0,0 +1,59 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.stream.hdfs; + +import com.bytedance.css.client.stream.CssInputStreamImpl; +import com.bytedance.css.client.stream.EpochInputStreamBuilder; +import com.bytedance.css.common.protocol.CommittedPartitionInfo; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.DataInputStream; +import java.io.IOException; +import java.util.List; + +public class EpochHdfsInputStreamBuilder implements EpochInputStreamBuilder { + + private static final Logger logger = LoggerFactory.getLogger(CssInputStreamImpl.class); + + private FileSystem fs; + + public EpochHdfsInputStreamBuilder(FileSystem fs) { + this.fs = fs; + } + + @Override + public DataInputStream createInputStream(List partitions) throws IOException { + DataInputStream inputStream = null; + // could be multi hdfs file for single epoch, just pick the first one that could be opened. + for (CommittedPartitionInfo partition : partitions) { + try { + Path path = new Path(partition.getFilePath()); + inputStream = fs.open(path); + break; + } catch (Exception ex) { + logger.warn(String.format("Create input stream failed for hdfs %s.", partition.getFilePath()), ex); + } + } + return inputStream; + } +} diff --git a/client/src/test/java/com/bytedance/css/client/compress/CompressorSuite.java b/client/src/test/java/com/bytedance/css/client/compress/CompressorSuite.java new file mode 100644 index 0000000..50e1303 --- /dev/null +++ b/client/src/test/java/com/bytedance/css/client/compress/CompressorSuite.java @@ -0,0 +1,185 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.compress; + +import com.bytedance.css.common.CssConf; +import org.apache.commons.lang3.RandomStringUtils; +import org.junit.Before; +import org.junit.Test; +import static org.junit.Assert.*; + +import java.io.*; + +public class CompressorSuite { + + private byte[] dict; + + @Before + public void beforeEach() { + ZstdDictTrainer zstdDictTrainer = new ZstdDictTrainer(); + this.dict = zstdDictTrainer.getDict(); + } + + @Test + public void testLz4() throws IOException { + CssConf cssConf = new CssConf(); + cssConf.set("css.push.buffer.size", "1024"); + Compressor compressor = new Lz4Compressor(cssConf); + Decompressor decompressor = new Lz4Decompressor(); + + for (int i = 800; i < 2000; i++) { + String context = RandomStringUtils.randomAlphanumeric(i, i); + byte[] source = context.getBytes(); + compressor.compress(source, 0, source.length); + byte[] compressdBytes = compressor.getCompressedBuffer(); + + byte[] dist = new byte[decompressor.getOriginalLen(compressdBytes)]; + int limit = decompressor.decompress(compressor.getCompressedBuffer(), dist, 0); + assertEquals(decompressor.getOriginalLen(compressdBytes), limit); + assertEquals(context, new String(dist)); + } + } + + @Test + public void testLz4WithRawMethod() throws IOException { + CssConf cssConf = new CssConf(); + cssConf.set("css.push.buffer.size", "1024"); + cssConf.set("css.compression.test.mode", "true"); + Compressor compressor = new Lz4Compressor(cssConf); + Decompressor decompressor = new Lz4Decompressor(); + + for (int i = 800; i < 2000; i++) { + String context = RandomStringUtils.randomAlphanumeric(i, i); + byte[] source = context.getBytes(); + compressor.compress(source, 0, source.length); + byte[] compressdBytes = compressor.getCompressedBuffer(); + + // raw method. compress data = origin data. + assertEquals(source.length, (compressor.getCompressedTotalSize() - CssCompressorTrait.HEADER_LENGTH)); + byte[] raw_array = new byte[source.length]; + System.arraycopy(compressor.getCompressedBuffer(), CssCompressorTrait.HEADER_LENGTH, + raw_array, 0, raw_array.length); + assertEquals(context, new String(raw_array)); + + byte[] dist = new byte[decompressor.getOriginalLen(compressdBytes)]; + int limit = decompressor.decompress(compressor.getCompressedBuffer(), dist, 0); + assertEquals(decompressor.getOriginalLen(compressdBytes), limit); + assertEquals(context, new String(dist)); + } + } + + @Test + public void testZstd() throws IOException { + CssConf cssConf = new CssConf(); + cssConf.set("css.push.buffer.size", "1024"); + Compressor compressor = new ZstdCompressor(cssConf); + Decompressor decompressor = new ZstdDecompressor(); + + for (int i = 800; i < 2000; i++) { + String context = RandomStringUtils.randomAlphanumeric(i, i); + byte[] source = context.getBytes(); + compressor.compress(source, 0, source.length); + byte[] compressdBytes = compressor.getCompressedBuffer(); + + byte[] dist = new byte[decompressor.getOriginalLen(compressdBytes)]; + int limit = decompressor.decompress(compressor.getCompressedBuffer(), dist, 0); + assertEquals(decompressor.getOriginalLen(compressdBytes), limit); + assertEquals(context, new String(dist)); + } + } + + @Test + public void testZstdWithRawMethod() throws IOException { + CssConf cssConf = new CssConf(); + cssConf.set("css.push.buffer.size", "1024"); + cssConf.set("css.compression.test.mode", "true"); + Compressor compressor = new ZstdCompressor(cssConf); + Decompressor decompressor = new ZstdDecompressor(); + + for (int i = 800; i < 2000; i++) { + String context = RandomStringUtils.randomAlphanumeric(i, i); + byte[] source = context.getBytes(); + compressor.compress(source, 0, source.length); + byte[] compressdBytes = compressor.getCompressedBuffer(); + + // raw method. compress data = origin data. + assertEquals(source.length, (compressor.getCompressedTotalSize() - CssCompressorTrait.HEADER_LENGTH)); + byte[] raw_array = new byte[source.length]; + System.arraycopy(compressor.getCompressedBuffer(), CssCompressorTrait.HEADER_LENGTH, + raw_array, 0, raw_array.length); + assertEquals(context, new String(raw_array)); + + byte[] dist = new byte[decompressor.getOriginalLen(compressdBytes)]; + int limit = decompressor.decompress(compressor.getCompressedBuffer(), dist, 0); + // raw method. compress data = origin data. + assertEquals(source.length, (compressor.getCompressedTotalSize() - CssCompressorTrait.HEADER_LENGTH)); + assertEquals(decompressor.getOriginalLen(compressdBytes), limit); + assertEquals(context, new String(dist)); + } + } + + @Test + public void testZstdDict() throws IOException { + CssConf cssConf = new CssConf(); + cssConf.set("css.push.buffer.size", "1024"); + Compressor compressor = new ZstdDictCompressor(cssConf, dict); + Decompressor decompressor = new ZstdDictDecompressor(dict); + + for (int i = 800; i < 2000; i++) { + String context = RandomStringUtils.randomAlphanumeric(i, i); + byte[] source = context.getBytes(); + compressor.compress(source, 0, source.length); + byte[] compressdBytes = compressor.getCompressedBuffer(); + + byte[] dist = new byte[decompressor.getOriginalLen(compressdBytes)]; + int limit = decompressor.decompress(compressor.getCompressedBuffer(), dist, 0); + assertEquals(decompressor.getOriginalLen(compressdBytes), limit); + assertEquals(context, new String(dist)); + } + } + + @Test + public void testZstdDictWithRawMethod() throws IOException { + CssConf cssConf = new CssConf(); + cssConf.set("css.push.buffer.size", "1024"); + cssConf.set("css.compression.test.mode", "true"); + Compressor compressor = new ZstdDictCompressor(cssConf, dict); + Decompressor decompressor = new ZstdDictDecompressor(dict); + + for (int i = 800; i < 2000; i++) { + String context = RandomStringUtils.randomAlphanumeric(i, i); + byte[] source = context.getBytes(); + compressor.compress(source, 0, source.length); + byte[] compressdBytes = compressor.getCompressedBuffer(); + + // raw method. compress data = origin data. + assertEquals(source.length, (compressor.getCompressedTotalSize() - CssCompressorTrait.HEADER_LENGTH)); + byte[] raw_array = new byte[source.length]; + System.arraycopy(compressor.getCompressedBuffer(), CssCompressorTrait.HEADER_LENGTH, + raw_array, 0, raw_array.length); + assertEquals(context, new String(raw_array)); + + byte[] dist = new byte[decompressor.getOriginalLen(compressdBytes)]; + int limit = decompressor.decompress(compressor.getCompressedBuffer(), dist, 0); + assertEquals(decompressor.getOriginalLen(compressdBytes), limit); + assertEquals(context, new String(dist)); + } + } +} diff --git a/client/src/test/java/com/bytedance/css/client/stream/CssDiskInputStreamSuite.java b/client/src/test/java/com/bytedance/css/client/stream/CssDiskInputStreamSuite.java new file mode 100644 index 0000000..193c4b6 --- /dev/null +++ b/client/src/test/java/com/bytedance/css/client/stream/CssDiskInputStreamSuite.java @@ -0,0 +1,343 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.stream; + +import com.bytedance.css.client.compress.Compressor; +import com.bytedance.css.client.compress.CompressorFactory; +import com.bytedance.css.client.compress.CssCompressorFactory; +import com.bytedance.css.client.stream.disk.CssRemoteDiskEpochReader; +import com.bytedance.css.common.CssConf; +import com.bytedance.css.common.protocol.CommittedPartitionInfo; +import com.bytedance.css.common.protocol.FailedPartitionInfoBatch; +import com.bytedance.css.common.protocol.ShuffleMode; +import com.bytedance.css.common.unsafe.Platform; +import com.bytedance.css.common.util.Utils; +import com.bytedance.css.network.TransportContext; +import com.bytedance.css.network.client.RpcResponseCallback; +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.client.TransportClientFactory; +import com.bytedance.css.network.protocol.shuffle.BlockTransferMessage; +import com.bytedance.css.network.protocol.shuffle.OpenStream; +import com.bytedance.css.network.protocol.shuffle.StreamHandle; +import com.bytedance.css.network.server.*; +import com.bytedance.css.network.util.TransportConf; +import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.RandomStringUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.*; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class CssDiskInputStreamSuite { + + private HashMap map = new HashMap<>(); + private TransportContext transportContext = null; + private TransportServer transportServer = null; + + class MockRpcHandler extends RpcHandler { + + private final TransportConf conf; + private final OneForOneStreamManager streamManager; + + MockRpcHandler(TransportConf conf) { + this.conf = conf; + streamManager = new OneForOneStreamManager(); + } + + @Override + public void receive(TransportClient client, ByteBuffer message, RpcResponseCallback callback) { + BlockTransferMessage msg = BlockTransferMessage.Decoder.fromByteBuffer(message); + OpenStream openStream = (OpenStream) msg; + String shuffleKey = openStream.shuffleKey; + String fileName = openStream.filePath; + int chunkIndex = openStream.initChunkIndex; + CssFileInfo fileInfo = map.get(shuffleKey + "-" + fileName); + + try { + CssManagedBufferIterator iterator = new CssManagedBufferIterator(fileInfo, conf); + iterator.setInitIndex(chunkIndex); + long streamId = streamManager.registerStream( + client.getClientId(), iterator, client.getChannel()); + streamManager.setStreamStateCurIndex(streamId, chunkIndex); + + StreamHandle streamHandle = new StreamHandle(streamId, fileInfo.numChunks); + callback.onSuccess(streamHandle.toByteBuffer()); + } catch (IOException e) { + callback.onFailure(new Exception("Chunk offsets meta exception ", e)); + } + } + + @Override + public StreamManager getStreamManager() { + return streamManager; + } + } + + @Before + public void startServer() throws Exception { + map.clear(); + TransportConf transportConf = Utils.fromCssConf(new CssConf(), "TEST", 0); + MockRpcHandler rpcHandler = new MockRpcHandler(transportConf); + transportContext = new TransportContext(transportConf, rpcHandler, false); + transportServer = transportContext.createServer(12345, new ArrayList<>()); + } + + @After + public void stopServer() throws Exception { + if (transportServer != null) { + transportServer.close(); + } + if (transportContext != null) { + transportContext.close(); + } + } + + private byte[] addHeaderAndCompress( + CssConf cssConf, + int mapperId, + int mapperAttemptId, + int batchId, + byte[] originalBytes) { + CompressorFactory factory = new CssCompressorFactory(cssConf); + final Compressor compressor = factory.getCompressor(); + compressor.compress(originalBytes, 0, originalBytes.length); + + final int compressedTotalSize = compressor.getCompressedTotalSize(); + final int BATCH_HEADER_SIZE = 4 * 4; + final byte[] body = new byte[BATCH_HEADER_SIZE + compressedTotalSize]; + Platform.putInt(body, Platform.BYTE_ARRAY_OFFSET, mapperId); + Platform.putInt(body, Platform.BYTE_ARRAY_OFFSET + 4, mapperAttemptId); + Platform.putInt(body, Platform.BYTE_ARRAY_OFFSET + 8, batchId); + Platform.putInt(body, Platform.BYTE_ARRAY_OFFSET + 12, compressedTotalSize); + System.arraycopy(compressor.getCompressedBuffer(), 0, body, BATCH_HEADER_SIZE, compressedTotalSize); + + return body; + } + + + @Test + public void testCssDiskInputStream() throws Exception { + CssConf cssConf = new CssConf(); + String[] localChunkEnables = new String[]{"true", "false"}; + for (String localChunkEnable : localChunkEnables) { + cssConf.set("css.local.chunk.fetch.enabled", localChunkEnable); + cssConf.set("css.chunk.fetch.retry.wait.times", "5ms"); + + String[] compressTypes = new String[]{"lz4", "zstd"}; + for (String type : compressTypes) { + cssConf.set("css.compression.codec", type); + + String shuffleKey = "DontTouchMe"; + TransportConf transportConf = Utils.fromCssConf(cssConf, "TEST", 0); + TransportContext transportContext = + new TransportContext(transportConf, new NoOpRpcHandler(), false); + TransportClientFactory clientFactory = transportContext.createClientFactory(new ArrayList<>()); + + for (int i = 0; i < 100; i++) { + String master = "SHUFFLE-FILE-" + i + "MASTER"; + String slave = "SHUFFLE-FILE-" + i + "SLAVE"; + + File file = File.createTempFile(type + "-ABC", "OUT.M"); + file.deleteOnExit(); + ArrayList chunkOffsets = new ArrayList<>(); + chunkOffsets.add(0L); + long totalLength = 0L; + long deDupLength = 0L; + FileOutputStream outputStream = new FileOutputStream(file); + + // remain all real data string. + List originContentList = new ArrayList<>(); + // add random data for target mapper task. + for (int j = 0; j < 1000; j++) { + String content = RandomStringUtils.randomAlphanumeric(1024, 2048); + originContentList.add(content); + byte[] bytes = addHeaderAndCompress(cssConf, 0, 0, j, content.getBytes(StandardCharsets.UTF_8)); + outputStream.write(bytes); + totalLength += bytes.length; + deDupLength += content.length(); + chunkOffsets.add(totalLength); + + // add duplicate batch data with same mapper task. + if (j % 80 == 0) { + outputStream.write(bytes); + totalLength += bytes.length; + chunkOffsets.add(totalLength); + } + } + + List failedPartitionInfoBatches = new ArrayList<>(); + // add failed batch data for current partition file. + for (int j = 1500; j < 1520; j++) { + String content = RandomStringUtils.randomAlphanumeric(1024, 2048); + byte[] bytes = addHeaderAndCompress(cssConf, 0, 0, j, content.getBytes(StandardCharsets.UTF_8)); + outputStream.write(bytes); + totalLength += bytes.length; + chunkOffsets.add(totalLength); + failedPartitionInfoBatches.add(new FailedPartitionInfoBatch(0, 0, 0, 0, j)); + } + + // add duplicate data with other attemptId + for (int k = 0; k < 100; k++) { + String content = RandomStringUtils.randomAlphanumeric(1024, 2048); + byte[] bytes = addHeaderAndCompress(cssConf, 0, 1, k, content.getBytes(StandardCharsets.UTF_8)); + outputStream.write(bytes); + totalLength += bytes.length; + chunkOffsets.add(totalLength); + } + outputStream.close(); + + // choose one replica available + CssFileInfo fileInfo = new CssFileInfo(file, chunkOffsets, file.length()); + String fileKey = i % 2 == 0 ? master : slave; + map.put(String.format("%s-%s", shuffleKey, fileKey), fileInfo); + + CommittedPartitionInfo[] partitions = new CommittedPartitionInfo[2]; + if (i % 2 == 0) { + partitions[0] = new CommittedPartitionInfo(0, 0, Utils.localHostName(), 12345, + ShuffleMode.DISK, fileKey, file.length()); + + // invalid one + partitions[1] = new CommittedPartitionInfo(0, 0, Utils.localHostName(), 54321, + ShuffleMode.DISK, "/NeverMind", 100000L); + } else { + // invalid one + partitions[0] = new CommittedPartitionInfo(0, 0, Utils.localHostName(), 54321, + ShuffleMode.DISK, "/NeverMind", 100000L); + + partitions[1] = new CommittedPartitionInfo(0, 0, Utils.localHostName(), 12345, + ShuffleMode.DISK, fileKey, file.length()); + } + + Set failedBatchBlacklist = failedPartitionInfoBatches.stream().map(x -> + String.format("%s-%s-%s-%s-%s", + x.getReducerId(), x.getEpochId(), x.getMapId(), x.getAttemptId(), x.getBatchId())) + .collect(Collectors.toSet()); + + CssInputStream inputStream = CssInputStream.create(cssConf, + clientFactory, shuffleKey, partitions, new int[]{0}, failedBatchBlacklist, 0, 1); + + long resultCount = 0L; + int contentCount = 0; + for (String originContent : originContentList) { + byte[] bytes = new byte[originContent.length()]; + resultCount += inputStream.read(bytes, 0, bytes.length); + contentCount++; + assertEquals(originContent, new String(bytes, StandardCharsets.UTF_8)); + } + assertEquals(inputStream.read(), -1); // inputStream already ended. + assertEquals(originContentList.size(), contentCount); + assertEquals(resultCount, deDupLength); + + inputStream.close(); + } + + clientFactory.close(); + transportContext.close(); + } + } + } + + @Test + public void testCssDiskEpochReader() throws Exception { + CssConf cssConf = new CssConf(); + cssConf.set("css.local.chunk.fetch.enabled", "false"); + cssConf.set("css.chunk.fetch.retry.wait.times", "5ms"); + + String shuffleKey = "DontTouchMe"; + TransportConf transportConf = Utils.fromCssConf(cssConf, "TEST", 0); + TransportContext transportContext = + new TransportContext(transportConf, new NoOpRpcHandler(), false); + TransportClientFactory clientFactory = transportContext.createClientFactory(new ArrayList<>()); + + for (int i = 0; i < 100; i++) { + String master = "SHUFFLE-FILE-" + i + "MASTER"; + String slave = "SHUFFLE-FILE-" + i + "SLAVE"; + + File file = File.createTempFile("ABC", "OUT.M"); + file.deleteOnExit(); + ArrayList chunkOffsets = new ArrayList<>(); + chunkOffsets.add(0L); + long totalLength = 0L; + FileOutputStream outputStream = new FileOutputStream(file); + HashSet set = new HashSet<>(); + for (int j = 0; j < 1000; j++) { + String content = RandomStringUtils.randomAlphanumeric(1024, 2048); + set.add(content); + byte[] bytes = new byte[4 + content.length()]; + Platform.putInt(bytes, Platform.BYTE_ARRAY_OFFSET, content.length()); + System.arraycopy(content.getBytes(StandardCharsets.UTF_8), 0, bytes, 4, content.length()); + outputStream.write(bytes); + totalLength += bytes.length; + chunkOffsets.add(totalLength); + } + outputStream.close(); + + CssFileInfo fileInfo = new CssFileInfo(file, chunkOffsets, file.length()); + String key = i % 2 == 0 ? master : slave; + map.put(String.format("%s-%s", shuffleKey, key), fileInfo); + + CommittedPartitionInfo[] partitions = new CommittedPartitionInfo[2]; + if (i % 2 == 0) { + partitions[0] = new CommittedPartitionInfo(0, 0, Utils.localHostName(), 12345, + ShuffleMode.DISK, key, file.length()); + + // invalid one + partitions[1] = new CommittedPartitionInfo(0, 0, Utils.localHostName(), 54321, + ShuffleMode.DISK, "/NeverMind", 100000L); + } else { + // invalid one + partitions[0] = new CommittedPartitionInfo(0, 0, Utils.localHostName(), 54321, + ShuffleMode.DISK, "/NeverMind", 100000L); + + partitions[1] = new CommittedPartitionInfo(0, 0, Utils.localHostName(), 12345, + ShuffleMode.DISK, key, file.length()); + } + + CssRemoteDiskEpochReader reader = new CssRemoteDiskEpochReader(cssConf, clientFactory, shuffleKey, partitions); + + byte[] sizeBuf = new byte[4]; + while (reader.hasNext()) { + ByteBuf buffer = reader.next(); + buffer.readBytes(sizeBuf); + int tmpLength = Platform.getInt(sizeBuf, Platform.BYTE_ARRAY_OFFSET); + byte[] tmpBuffer = new byte[tmpLength]; + buffer.readBytes(tmpBuffer); + String content = new String(tmpBuffer); + assertTrue(set.contains(content)); + set.remove(content); + } + assertTrue(set.isEmpty()); + reader.close(); + } + + clientFactory.close(); + transportContext.close(); + } +} diff --git a/client/src/test/java/com/bytedance/css/client/stream/CssEpochReaderRetryCorrectSuite.java b/client/src/test/java/com/bytedance/css/client/stream/CssEpochReaderRetryCorrectSuite.java new file mode 100644 index 0000000..e91a8e6 --- /dev/null +++ b/client/src/test/java/com/bytedance/css/client/stream/CssEpochReaderRetryCorrectSuite.java @@ -0,0 +1,327 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.stream; + +import com.bytedance.css.client.stream.disk.CssRemoteDiskEpochReader; +import com.bytedance.css.common.CssConf; +import com.bytedance.css.common.protocol.CommittedPartitionInfo; +import com.bytedance.css.common.protocol.ShuffleMode; +import com.bytedance.css.common.unsafe.Platform; +import com.bytedance.css.common.util.Utils; +import com.bytedance.css.network.TransportContext; +import com.bytedance.css.network.buffer.FileSegmentManagedBuffer; +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.client.RpcResponseCallback; +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.client.TransportClientFactory; +import com.bytedance.css.network.protocol.shuffle.BlockTransferMessage; +import com.bytedance.css.network.protocol.shuffle.OpenStream; +import com.bytedance.css.network.protocol.shuffle.StreamHandle; +import com.bytedance.css.network.server.*; +import com.bytedance.css.network.util.TransportConf; +import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.RandomStringUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.*; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class CssEpochReaderRetryCorrectSuite { + + private HashMap map = new HashMap<>(); + private TransportContext transportContext = null; + private TransportServer transportServer = null; + private CssConf serverCssConf = new CssConf(); + private int failedIndex = 10; + private volatile boolean retryEnable = true; + + class MockIterator implements Iterator { + private final File file; + private final long[] offsets; + private final int numChunks; + private final TransportConf conf; + + private int maxFailedTimes = CssConf.chunkFetchFailedRetryMaxTimes(serverCssConf); + private int shouldFailed; + private int index; + + MockIterator(CssFileInfo fileInfo, TransportConf conf) throws IOException { + this.file = fileInfo.file; + this.conf = conf; + this.numChunks = fileInfo.numChunks; + this.offsets = new long[numChunks + 1]; + for (int i = 0; i <= numChunks; i++) { + offsets[i] = fileInfo.chunkOffsets.get(i); + } + if (offsets[numChunks] != fileInfo.fileLength) { + throw new IOException( + String.format("The last chunk offset %d should be equals to file length %d!", + offsets[numChunks], fileInfo.fileLength)); + } + } + + public void setInitIndex(int index) { + this.index = index; + } + + public void setShouldFailed(int shouldFailed) { + this.shouldFailed = shouldFailed; + } + + @Override + public boolean hasNext() { + return index < numChunks; + } + + @Override + public ManagedBuffer next() { + if (index == failedIndex) { + if (retryEnable && shouldFailed % (maxFailedTimes + 1) != maxFailedTimes){ + throw new RuntimeException( + String.format("Chunk Fetch Failed for Test with times %s index %s file %s.", + shouldFailed, index, file)); + } else if (retryEnable && shouldFailed % (maxFailedTimes + 1) == maxFailedTimes) { + System.out.println(String.format("Chunk Fetch Success at max times %s index %s file %s.", + shouldFailed, index, file)); + } else { + throw new RuntimeException( + String.format("Chunk Fetch Failed for Test with current index %s.", index)); + } + } + final long offset = offsets[index]; + final long length = offsets[index + 1] - offset; + index++; + return new FileSegmentManagedBuffer(conf, file, offset, length); + } + } + + class MockRpcHandler extends RpcHandler { + + private final TransportConf conf; + private final OneForOneStreamManager streamManager; + + private int shouldFailed = 0; + + MockRpcHandler(TransportConf conf) { + this.conf = conf; + streamManager = new OneForOneStreamManager(); + } + + @Override + public void receive(TransportClient client, ByteBuffer message, RpcResponseCallback callback) { + BlockTransferMessage msg = BlockTransferMessage.Decoder.fromByteBuffer(message); + OpenStream openStream = (OpenStream) msg; + String shuffleKey = openStream.shuffleKey; + String fileName = openStream.filePath; + int chunkIndex = openStream.initChunkIndex; + CssFileInfo fileInfo = map.get(shuffleKey + "-" + fileName); + + try { + MockIterator iterator = new MockIterator(fileInfo, conf); + iterator.setShouldFailed(shouldFailed); + iterator.setInitIndex(chunkIndex); + long streamId = streamManager.registerStream(client.getClientId(), iterator, client.getChannel()); + streamManager.setStreamStateCurIndex(streamId, chunkIndex); + + StreamHandle streamHandle = new StreamHandle(streamId, fileInfo.numChunks); + callback.onSuccess(streamHandle.toByteBuffer()); + } catch (IOException e) { + callback.onFailure(new Exception("Chunk offsets meta exception ", e)); + } + shouldFailed++; + if (shouldFailed == CssConf.chunkFetchFailedRetryMaxTimes(serverCssConf) + 1) { + shouldFailed = 0; + } + } + + @Override + public StreamManager getStreamManager() { + return streamManager; + } + } + + @Before + public void startServer() throws Exception { + map.clear(); + TransportConf transportConf = Utils.fromCssConf(serverCssConf, "RetryFetcherTest", 0); + MockRpcHandler rpcHandler = new MockRpcHandler(transportConf); + transportContext = new TransportContext(transportConf, rpcHandler, false); + transportServer = transportContext.createServer(26789, new ArrayList<>()); + } + + @After + public void stopServer() throws Exception { + if (transportServer != null) { + transportServer.close(); + } + if (transportContext != null) { + transportContext.close(); + } + } + + @Test + public void testMaxTimesRetryChunkFetcherCorrect() throws IOException, InterruptedException { + CssConf cssConf = new CssConf(); + cssConf.set("css.local.chunk.fetch.enabled", "false"); + cssConf.set("css.chunk.fetch.retry.wait.times", "5ms"); + this.retryEnable = true; + + String shuffleKey = "DontTouchEpochReaderCorrect"; + TransportConf transportConf = Utils.fromCssConf(cssConf, "TEST", 0); + TransportContext transportContext = new TransportContext(transportConf, new NoOpRpcHandler(), false); + TransportClientFactory clientFactory = transportContext.createClientFactory(new ArrayList<>()); + + for (int i = 0; i < 20; i++) { + String master = "FetcherCorrect-SHUFFLE-FILE-" + i + "MASTER"; + String slave = "FetcherCorrect-SHUFFLE-FILE-" + i + "SLAVE"; + + File file = File.createTempFile("FetcherCorrect", "OUT.M"); + file.deleteOnExit(); + ArrayList chunkOffsets = new ArrayList<>(); + chunkOffsets.add(0L); + long totalLength = 0L; + FileOutputStream outputStream = new FileOutputStream(file); + List originList = new ArrayList<>(); + for (int j = 0; j < 100; j++) { + String content = RandomStringUtils.randomAlphanumeric(1024, 2048); + originList.add(content); + byte[] bytes = new byte[4 + content.length()]; + Platform.putInt(bytes, Platform.BYTE_ARRAY_OFFSET, content.length()); + System.arraycopy(content.getBytes(StandardCharsets.UTF_8), 0, bytes, 4, content.length()); + outputStream.write(bytes); + totalLength += bytes.length; + chunkOffsets.add(totalLength); + } + outputStream.close(); + + CssFileInfo fileInfo = new CssFileInfo(file, chunkOffsets, file.length()); + String key = i % 2 == 0 ? master : slave; + map.put(String.format("%s-%s", shuffleKey, key), fileInfo); + + // one piece available + CommittedPartitionInfo committedPartitionInfo = + new CommittedPartitionInfo(0, 0, Utils.localHostName(), 26789, + ShuffleMode.DISK, key, file.length()); + + // test retry chunk fetch. + CssRemoteDiskEpochReader chunkFetcher = new CssRemoteDiskEpochReader( + cssConf, + clientFactory, + shuffleKey, + new CommittedPartitionInfo[]{committedPartitionInfo} + ); + + byte[] sizeBuf = new byte[4]; + while (chunkFetcher.hasNext()) { + ByteBuf buffer = chunkFetcher.next(); + buffer.readBytes(sizeBuf); + int tmpLength = Platform.getInt(sizeBuf, Platform.BYTE_ARRAY_OFFSET); + byte[] tmpBuffer = new byte[tmpLength]; + buffer.readBytes(tmpBuffer); + String content = new String(tmpBuffer); + assertTrue(originList.contains(content)); + originList.remove(content); + } + assertEquals(originList.size(), 0); + chunkFetcher.close(); + } + + clientFactory.close(); + transportContext.close(); + } + + @Test + public void testMaxTimesRetryFailedTask() throws IOException, InterruptedException { + CssConf cssConf = new CssConf(); + cssConf.set("css.local.chunk.fetch.enabled", "false"); + cssConf.set("css.chunk.fetch.retry.wait.times", "5ms"); + this.retryEnable = false; + + String shuffleKey = "DontTouchRetryFailedTask"; + TransportConf transportConf = Utils.fromCssConf(cssConf, "TEST", 0); + TransportContext transportContext = new TransportContext(transportConf, new NoOpRpcHandler(), false); + TransportClientFactory clientFactory = transportContext.createClientFactory(new ArrayList<>()); + + for (int i = 0; i < 20; i++) { + String master = "FailedTask-SHUFFLE-FILE-" + i + "MASTER"; + String slave = "FailedTask-SHUFFLE-FILE-" + i + "SLAVE"; + + File file = File.createTempFile("FailedTask", "OUT.M"); + file.deleteOnExit(); + ArrayList chunkOffsets = new ArrayList<>(); + chunkOffsets.add(0L); + long totalLength = 0L; + FileOutputStream outputStream = new FileOutputStream(file); + List originList = new ArrayList<>(); + for (int j = 0; j < 100; j++) { + String content = RandomStringUtils.randomAlphanumeric(1024, 2048); + originList.add(content); + byte[] bytes = new byte[4 + content.length()]; + Platform.putInt(bytes, Platform.BYTE_ARRAY_OFFSET, content.length()); + System.arraycopy(content.getBytes(StandardCharsets.UTF_8), 0, bytes, 4, content.length()); + outputStream.write(bytes); + totalLength += bytes.length; + chunkOffsets.add(totalLength); + } + outputStream.close(); + + CssFileInfo fileInfo = new CssFileInfo(file, chunkOffsets, file.length()); + String key = i % 2 == 0 ? master : slave; + map.put(String.format("%s-%s", shuffleKey, key), fileInfo); + + // one piece available + CommittedPartitionInfo committedPartitionInfo = + new CommittedPartitionInfo(0, 0, Utils.localHostName(), 26789, + ShuffleMode.DISK, key, file.length()); + + // test retry chunk fetch. + CssRemoteDiskEpochReader chunkFetcher = new CssRemoteDiskEpochReader( + cssConf, + clientFactory, + shuffleKey, + new CommittedPartitionInfo[]{committedPartitionInfo} + ); + + boolean loop = true; + while (loop && chunkFetcher.hasNext()) { + try { + ByteBuf buffer = chunkFetcher.next(); + } catch (IOException ioException) { + assertTrue(ioException.getMessage().startsWith( + "chunk fetch retry all active committed partition for shuffleKey")); + loop = false; + } + } + chunkFetcher.close(); + } + + clientFactory.close(); + transportContext.close(); + } +} diff --git a/client/src/test/java/com/bytedance/css/client/stream/CssEpochReaderRetrySuite.java b/client/src/test/java/com/bytedance/css/client/stream/CssEpochReaderRetrySuite.java new file mode 100644 index 0000000..5cf6644 --- /dev/null +++ b/client/src/test/java/com/bytedance/css/client/stream/CssEpochReaderRetrySuite.java @@ -0,0 +1,322 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.stream; + +import com.bytedance.css.client.stream.disk.CssRemoteDiskEpochReader; +import com.bytedance.css.common.CssConf; +import com.bytedance.css.common.protocol.CommittedPartitionInfo; +import com.bytedance.css.common.protocol.ShuffleMode; +import com.bytedance.css.common.unsafe.Platform; +import com.bytedance.css.common.util.Utils; +import com.bytedance.css.network.TransportContext; +import com.bytedance.css.network.buffer.FileSegmentManagedBuffer; +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.client.RpcResponseCallback; +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.client.TransportClientFactory; +import com.bytedance.css.network.protocol.shuffle.BlockTransferMessage; +import com.bytedance.css.network.protocol.shuffle.OpenStream; +import com.bytedance.css.network.protocol.shuffle.StreamHandle; +import com.bytedance.css.network.server.*; +import com.bytedance.css.network.util.TransportConf; +import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.RandomStringUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.*; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class CssEpochReaderRetrySuite { + + private HashMap map = new HashMap<>(); + private TransportContext transportContext = null; + private TransportServer transportServer = null; + CssConf cssConf = new CssConf(); + + class MockIterator implements Iterator { + private final File file; + private final long[] offsets; + private final int numChunks; + private final TransportConf conf; + + private int shouldFailed; + private int index; + + MockIterator(CssFileInfo fileInfo, TransportConf conf) throws IOException { + this.file = fileInfo.file; + this.conf = conf; + this.numChunks = fileInfo.numChunks; + this.offsets = new long[numChunks + 1]; + for (int i = 0; i <= numChunks; i++) { + offsets[i] = fileInfo.chunkOffsets.get(i); + } + if (offsets[numChunks] != fileInfo.fileLength) { + throw new IOException( + String.format("The last chunk offset %d should be equals to file length %d!", + offsets[numChunks], fileInfo.fileLength)); + } + } + + public void setInitIndex(int index) { + this.index = index; + } + + public void setShouldFailed(int shouldFailed) { + this.shouldFailed = shouldFailed; + } + + @Override + public boolean hasNext() { + return index < numChunks; + } + + @Override + public ManagedBuffer next() { + if (index == 10 && shouldFailed % CssConf.chunkFetchFailedRetryMaxTimes(cssConf) != 2) { + throw new RuntimeException("Chunk Fetch Failed for Test"); + } + final long offset = offsets[index]; + final long length = offsets[index + 1] - offset; + index++; + return new FileSegmentManagedBuffer(conf, file, offset, length); + } + } + + class MockRpcHandler extends RpcHandler { + + private final TransportConf conf; + private final OneForOneStreamManager streamManager; + private int shouldFailed = 0; + + MockRpcHandler(TransportConf conf) { + this.conf = conf; + streamManager = new OneForOneStreamManager(); + } + + @Override + public void receive(TransportClient client, ByteBuffer message, RpcResponseCallback callback) { + BlockTransferMessage msg = BlockTransferMessage.Decoder.fromByteBuffer(message); + OpenStream openStream = (OpenStream) msg; + String shuffleKey = openStream.shuffleKey; + String fileName = openStream.filePath; + int chunkIndex = openStream.initChunkIndex; + CssFileInfo fileInfo = map.get(shuffleKey + "-" + fileName); + + if (shouldFailed == 1) { + callback.onFailure(new Exception("Chunk offsets meta exception for test")); + } else { + try { + MockIterator iterator = new MockIterator(fileInfo, conf); + iterator.setShouldFailed(shouldFailed); + iterator.setInitIndex(chunkIndex); + long streamId = streamManager.registerStream(client.getClientId(), iterator, client.getChannel()); + streamManager.setStreamStateCurIndex(streamId, chunkIndex); + + StreamHandle streamHandle = new StreamHandle(streamId, fileInfo.numChunks); + callback.onSuccess(streamHandle.toByteBuffer()); + } catch (IOException e) { + callback.onFailure(new Exception("Chunk offsets meta exception ", e)); + } + } + shouldFailed++; + if (shouldFailed == CssConf.chunkFetchFailedRetryMaxTimes(cssConf)) { + shouldFailed = 0; + } + } + + @Override + public StreamManager getStreamManager() { + return streamManager; + } + } + + @Before + public void startServer() throws Exception { + map.clear(); + TransportConf transportConf = Utils.fromCssConf(cssConf, "RetryFetcherTest", 0); + MockRpcHandler rpcHandler = new MockRpcHandler(transportConf); + transportContext = new TransportContext(transportConf, rpcHandler, false); + transportServer = transportContext.createServer(16789, new ArrayList<>()); + } + + @After + public void stopServer() throws Exception { + if (transportServer != null) { + transportServer.close(); + } + if (transportContext != null) { + transportContext.close(); + } + } + + @Test + public void testEpochChunkFetcher() throws IOException { + CssConf cssConf = new CssConf(); + cssConf.set("css.local.chunk.fetch.enabled", "false"); + cssConf.set("css.chunk.fetch.retry.wait.times", "5ms"); + + String shuffleKey = "DontTouchEpochFetchClient"; + TransportConf transportConf = Utils.fromCssConf(cssConf, "TEST", 0); + TransportContext transportContext = new TransportContext(transportConf, new NoOpRpcHandler(), false); + TransportClientFactory clientFactory = transportContext.createClientFactory(new ArrayList<>()); + + for (int i = 0; i < 20; i++) { + String master = "EpochFetch-SHUFFLE-FILE-" + i + "MASTER"; + String slave = "EpochFetch-SHUFFLE-FILE-" + i + "SLAVE"; + + File file = File.createTempFile("EpochFetch", "OUT.M"); + file.deleteOnExit(); + ArrayList chunkOffsets = new ArrayList<>(); + chunkOffsets.add(0L); + long totalLength = 0L; + FileOutputStream outputStream = new FileOutputStream(file); + List originList = new ArrayList<>(); + for (int j = 0; j < 100; j++) { + String content = RandomStringUtils.randomAlphanumeric(1024, 2048); + originList.add(content); + byte[] bytes = new byte[4 + content.length()]; + Platform.putInt(bytes, Platform.BYTE_ARRAY_OFFSET, content.length()); + System.arraycopy(content.getBytes(StandardCharsets.UTF_8), 0, bytes, 4, content.length()); + outputStream.write(bytes); + totalLength += bytes.length; + chunkOffsets.add(totalLength); + } + outputStream.close(); + + CssFileInfo fileInfo = new CssFileInfo(file, chunkOffsets, file.length()); + String key = i % 2 == 0 ? master : slave; + map.put(String.format("%s-%s", shuffleKey, key), fileInfo); + + CommittedPartitionInfo[] partitions = new CommittedPartitionInfo[2]; + if (i % 2 == 0) { + // two piece available + partitions[0] = new CommittedPartitionInfo(0, 0, Utils.localHostName(), 16789, + ShuffleMode.DISK, key, file.length()); + + partitions[1] = new CommittedPartitionInfo(0, 0, Utils.localHostName(), 16789, + ShuffleMode.DISK, key, file.length()); + } else { + // invalid one + partitions[0] = new CommittedPartitionInfo(0, 0, Utils.localHostName(), 54321, + ShuffleMode.DISK, "NeverMind", 100000L); + + partitions[1] = new CommittedPartitionInfo(0, 0, Utils.localHostName(), 16789, + ShuffleMode.DISK, key, file.length()); + } + + CssRemoteDiskEpochReader reader = new CssRemoteDiskEpochReader(cssConf, clientFactory, shuffleKey, partitions); + + byte[] sizeBuf = new byte[4]; + while (reader.hasNext()) { + ByteBuf buffer = reader.next(); + buffer.readBytes(sizeBuf); + int tmpLength = Platform.getInt(sizeBuf, Platform.BYTE_ARRAY_OFFSET); + byte[] tmpBuffer = new byte[tmpLength]; + buffer.readBytes(tmpBuffer); + String content = new String(tmpBuffer); + assertTrue(originList.contains(content)); + originList.remove(content); + } + assertEquals(originList.size(), 0); + reader.close(); + } + + clientFactory.close(); + transportContext.close(); + } + + @Test + public void testRetryChunkFetcher() throws IOException, InterruptedException { + CssConf cssConf = new CssConf(); + cssConf.set("css.local.chunk.fetch.enabled", "false"); + cssConf.set("css.chunk.fetch.retry.wait.times", "5ms"); + + String shuffleKey = "DontTouchRetryChunkFetcher"; + TransportConf transportConf = Utils.fromCssConf(cssConf, "TEST", 0); + TransportContext transportContext = new TransportContext(transportConf, new NoOpRpcHandler(), false); + TransportClientFactory clientFactory = transportContext.createClientFactory(new ArrayList<>()); + + for (int i = 0; i < 20; i++) { + String master = "RetryChunkFetcher-SHUFFLE-FILE-" + i + "MASTER"; + String slave = "RetryChunkFetcher-SHUFFLE-FILE-" + i + "SLAVE"; + + File file = File.createTempFile("RetryChunkFetcher", "OUT.M"); + file.deleteOnExit(); + ArrayList chunkOffsets = new ArrayList<>(); + chunkOffsets.add(0L); + long totalLength = 0L; + FileOutputStream outputStream = new FileOutputStream(file); + List originList = new ArrayList<>(); + for (int j = 0; j < 100; j++) { + String content = RandomStringUtils.randomAlphanumeric(1024, 2048); + originList.add(content); + byte[] bytes = new byte[4 + content.length()]; + Platform.putInt(bytes, Platform.BYTE_ARRAY_OFFSET, content.length()); + System.arraycopy(content.getBytes(StandardCharsets.UTF_8), 0, bytes, 4, content.length()); + outputStream.write(bytes); + totalLength += bytes.length; + chunkOffsets.add(totalLength); + } + outputStream.close(); + + CssFileInfo fileInfo = new CssFileInfo(file, chunkOffsets, file.length()); + String key = i % 2 == 0 ? master : slave; + map.put(String.format("%s-%s", shuffleKey, key), fileInfo); + + // one piece available + CommittedPartitionInfo committedPartitionInfo = new CommittedPartitionInfo( + 0, 0, Utils.localHostName(), 16789, ShuffleMode.DISK, key, file.length()); + + // test retry chunk fetch. + CssRemoteDiskEpochReader chunkFetcher = new CssRemoteDiskEpochReader( + cssConf, + clientFactory, + shuffleKey, + new CommittedPartitionInfo[]{committedPartitionInfo} + ); + + byte[] sizeBuf = new byte[4]; + while (chunkFetcher.hasNext()) { + ByteBuf buffer = chunkFetcher.next(); + buffer.readBytes(sizeBuf); + int tmpLength = Platform.getInt(sizeBuf, Platform.BYTE_ARRAY_OFFSET); + byte[] tmpBuffer = new byte[tmpLength]; + buffer.readBytes(tmpBuffer); + String content = new String(tmpBuffer); + assertTrue(originList.contains(content)); + originList.remove(content); + } + assertEquals(originList.size(), 0); + chunkFetcher.close(); + } + + clientFactory.close(); + transportContext.close(); + } +} diff --git a/client/src/test/java/com/bytedance/css/client/stream/CssHdfsInputStreamSuite.java b/client/src/test/java/com/bytedance/css/client/stream/CssHdfsInputStreamSuite.java new file mode 100644 index 0000000..297e351 --- /dev/null +++ b/client/src/test/java/com/bytedance/css/client/stream/CssHdfsInputStreamSuite.java @@ -0,0 +1,243 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.client.stream; + +import com.bytedance.css.client.compress.Compressor; +import com.bytedance.css.client.compress.CompressorFactory; +import com.bytedance.css.client.compress.CssCompressorFactory; +import com.bytedance.css.common.CssConf; +import com.bytedance.css.common.protocol.CommittedPartitionInfo; +import com.bytedance.css.common.protocol.FailedPartitionInfoBatch; +import com.bytedance.css.common.protocol.ShuffleMode; +import com.bytedance.css.common.unsafe.Platform; +import com.bytedance.css.common.util.Utils; +import com.bytedance.css.network.TransportContext; +import com.bytedance.css.network.client.RpcResponseCallback; +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.client.TransportClientFactory; +import com.bytedance.css.network.protocol.shuffle.BlockTransferMessage; +import com.bytedance.css.network.protocol.shuffle.OpenStream; +import com.bytedance.css.network.protocol.shuffle.StreamHandle; +import com.bytedance.css.network.server.*; +import com.bytedance.css.network.util.TransportConf; +import org.apache.commons.lang3.RandomStringUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.*; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertEquals; + +public class CssHdfsInputStreamSuite { + + private HashMap map = new HashMap<>(); + private TransportContext transportContext = null; + private TransportServer transportServer = null; + + class MockRpcHandler extends RpcHandler { + + private final TransportConf conf; + private final OneForOneStreamManager streamManager; + + MockRpcHandler(TransportConf conf) { + this.conf = conf; + streamManager = new OneForOneStreamManager(); + } + + @Override + public void receive(TransportClient client, ByteBuffer message, RpcResponseCallback callback) { + BlockTransferMessage msg = BlockTransferMessage.Decoder.fromByteBuffer(message); + OpenStream openStream = (OpenStream) msg; + String shuffleKey = openStream.shuffleKey; + String fileName = openStream.filePath; + int chunkIndex = openStream.initChunkIndex; + CssFileInfo fileInfo = map.get(shuffleKey + "-" + fileName); + + try { + CssManagedBufferIterator iterator = new CssManagedBufferIterator(fileInfo, conf); + iterator.setInitIndex(chunkIndex); + long streamId = streamManager.registerStream( + client.getClientId(), iterator, client.getChannel()); + streamManager.setStreamStateCurIndex(streamId, chunkIndex); + + StreamHandle streamHandle = new StreamHandle(streamId, fileInfo.numChunks); + callback.onSuccess(streamHandle.toByteBuffer()); + } catch (IOException e) { + callback.onFailure(new Exception("Chunk offsets meta exception ", e)); + } + } + + @Override + public StreamManager getStreamManager() { + return streamManager; + } + } + + @Before + public void startServer() throws Exception { + map.clear(); + TransportConf transportConf = Utils.fromCssConf(new CssConf(), "TEST", 0); + MockRpcHandler rpcHandler = new MockRpcHandler(transportConf); + transportContext = new TransportContext(transportConf, rpcHandler, false); + transportServer = transportContext.createServer(12345, new ArrayList<>()); + } + + @After + public void stopServer() throws Exception { + if (transportServer != null) { + transportServer.close(); + } + if (transportContext != null) { + transportContext.close(); + } + } + + private byte[] addHeaderAndCompress( + CssConf cssConf, + int mapperId, + int mapperAttemptId, + int batchId, + byte[] originalBytes) { + CompressorFactory factory = new CssCompressorFactory(cssConf); + final Compressor compressor = factory.getCompressor(); + compressor.compress(originalBytes, 0, originalBytes.length); + + final int compressedTotalSize = compressor.getCompressedTotalSize(); + final int BATCH_HEADER_SIZE = 4 * 4; + final byte[] body = new byte[BATCH_HEADER_SIZE + compressedTotalSize]; + Platform.putInt(body, Platform.BYTE_ARRAY_OFFSET, mapperId); + Platform.putInt(body, Platform.BYTE_ARRAY_OFFSET + 4, mapperAttemptId); + Platform.putInt(body, Platform.BYTE_ARRAY_OFFSET + 8, batchId); + Platform.putInt(body, Platform.BYTE_ARRAY_OFFSET + 12, compressedTotalSize); + System.arraycopy(compressor.getCompressedBuffer(), 0, body, BATCH_HEADER_SIZE, compressedTotalSize); + + return body; + } + + + @Test + public void testCssHdfsInputStream() throws Exception { + CssConf cssConf = new CssConf(); + String[] compressTypes = new String[]{"lz4", "zstd"}; + for (String type : compressTypes) { + cssConf.set("css.compression.codec", type); + + String shuffleKey = "DontTouchMe"; + TransportConf transportConf = Utils.fromCssConf(cssConf, "TEST", 0); + TransportContext transportContext = + new TransportContext(transportConf, new NoOpRpcHandler(), false); + TransportClientFactory clientFactory = transportContext.createClientFactory(new ArrayList<>()); + + for (int i = 0; i < 100; i++) { + File file = File.createTempFile(type + "-ABC", "OUT.M"); + file.deleteOnExit(); + long totalLength = 0L; + long deDupLength = 0L; + FileOutputStream outputStream = new FileOutputStream(file); + + // remain all real data string. + List originContentList = new ArrayList<>(); + // add random data for target mapper task. + for (int j = 0; j < 1000; j++) { + String content = RandomStringUtils.randomAlphanumeric(1024, 2048); + originContentList.add(content); + byte[] bytes = addHeaderAndCompress(cssConf, 0, 0, j, content.getBytes(StandardCharsets.UTF_8)); + outputStream.write(bytes); + totalLength += bytes.length; + deDupLength += content.length(); + + // add duplicate batch data with same mapper task. + if (j % 80 == 0) { + outputStream.write(bytes); + totalLength += bytes.length; + } + } + + List failedPartitionInfoBatches = new ArrayList<>(); + // add failed batch data for current partition file. + for (int j = 1500; j < 1520; j++) { + String content = RandomStringUtils.randomAlphanumeric(1024, 2048); + byte[] bytes = addHeaderAndCompress(cssConf, 0, 0, j, content.getBytes(StandardCharsets.UTF_8)); + outputStream.write(bytes); + totalLength += bytes.length; + failedPartitionInfoBatches.add(new FailedPartitionInfoBatch(0, 0, 0, 0, j)); + } + + // add duplicate data with other attemptId + for (int k = 0; k < 100; k++) { + String content = RandomStringUtils.randomAlphanumeric(1024, 2048); + byte[] bytes = addHeaderAndCompress(cssConf, 0, 1, k, content.getBytes(StandardCharsets.UTF_8)); + outputStream.write(bytes); + totalLength += bytes.length; + } + + outputStream.close(); + + CommittedPartitionInfo[] partitions = new CommittedPartitionInfo[2]; + if (i % 2 == 0) { + partitions[0] = new CommittedPartitionInfo(0, 0, Utils.localHostName(), 12345, + ShuffleMode.HDFS, String.format("file://%s", file.getAbsolutePath()), file.length()); + + // invalid one + partitions[1] = new CommittedPartitionInfo(0, 0, Utils.localHostName(), 54321, + ShuffleMode.HDFS, "NeverMind", 100000L); + } else { + // invalid one + partitions[0] = new CommittedPartitionInfo(0, 0, Utils.localHostName(), 54321, + ShuffleMode.HDFS, "NeverMind", 100000L); + + partitions[1] = new CommittedPartitionInfo(0, 0, Utils.localHostName(), 12345, + ShuffleMode.HDFS, String.format("file://%s", file.getAbsolutePath()), file.length()); + } + + Set failedBatchBlacklist = failedPartitionInfoBatches.stream().map(x -> + String.format("%s-%s-%s-%s-%s", + x.getReducerId(), x.getEpochId(), x.getMapId(), x.getAttemptId(), x.getBatchId())) + .collect(Collectors.toSet()); + + CssInputStream inputStream = CssInputStream.create(cssConf, + clientFactory, shuffleKey, partitions, new int[]{0}, failedBatchBlacklist, 0, 1); + + long resultCount = 0L; + int contentCount = 0; + for (String originContent : originContentList) { + byte[] bytes = new byte[originContent.length()]; + resultCount += inputStream.read(bytes, 0, bytes.length); + contentCount++; + assertEquals(originContent, new String(bytes, StandardCharsets.UTF_8)); + } + assertEquals(inputStream.read(), -1); // inputStream already ended. + assertEquals(originContentList.size(), contentCount); + assertEquals(resultCount, deDupLength); + inputStream.close(); + } + + clientFactory.close(); + transportContext.close(); + } + } +} diff --git a/client/src/test/resources/log4j.properties b/client/src/test/resources/log4j.properties new file mode 100644 index 0000000..3c1122b --- /dev/null +++ b/client/src/test/resources/log4j.properties @@ -0,0 +1,29 @@ +# +# Copyright 2022 Bytedance Inc. +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=DEBUG, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Silence verbose logs from 3rd-party libraries. +log4j.logger.io.netty=INFO diff --git a/client/src/test/resources/xml b/client/src/test/resources/xml new file mode 100644 index 0000000000000000000000000000000000000000..d428fcc3c0d7de9b2c5c74eabcf9a77c552280c4 GIT binary patch literal 5345280 zcmeFaS#umWvOntQ6EWX`*Drp!Gh|nWB3pJ_uD7mJbnD+d!A8&*z&*NA{L^ z?c2Y8^T($@JU%{gZ^<2g>$`t{_T`fo&%XQiAJ1PrdGYl5w_SIj1pzb~XuOBY|@4@<2bFjIYA3Qqh zc#BKahflwPZ~q33eB2y1{u_UrUl;2C@Z`&9e|@g~d2;#F&F%Hoo10JJcyipVmTm||^0-4X1|n1qh>yL)Z?!~fq?pY&%MLO7=IPfjI;Hbu zu{c~kIx_DlzleBVR29$p;V|L+_U!*2EI=^dHhr@B#dm*u@-?vAKQiLhfVf5PNpDk< zy`Ci6OF!a9v`PHF`_n)E_T>BTeh^7fM*rve#npeaBwEovnH?T8!yKi)mrfY5*D`xh zqnSy$=3^;P>-y7QpM&gbiCwJ(=+(isA4+LS@n3-qt-GLskRgpW+L#jHI*~4>OOn>5 zKtd}S2|)9O(j!$k)|*A3H)q%9FME`Hz-1ZqIRP0jLZ9{T4_3cFSpNQCEDRkVkAb0P zxdbyLheBZK)HD(nj#JBRgw^ZKSQ`PT*#;P9N*R_6XdQb6V}Qg6ta56Pw9h?`n9p+( znwQQLXKQ0dij9M$RN1{!gBIe|<@=0f+{~Lp@Ozv{WCUjgDxgSk{@9i&Ebh^gVgsTfkrA-j(UW;IX+z0Q{pFw#|tS<+sW!bhY`nz>m_hv zQp0m$Y2B+mEYad~=hvmnlJ=!QL#r7HNb}{Pk|b4n2uTMA&|PxZ56Ex+{L`a#vy~m;qC}cm3(^2MdHCsp08={nq*cg-nAy9wX zd@e;=tWfUhbQKPpSy9T3`hJ{7}5zOnr{KQ!XiRKMRa6bwigUKOp^`cM%Q40!RfM?eVt1_n;q){EE zmeCrUWZn>PH9!-8wc7MpMYj0L;O2LI+w4T<-rPpV~xPEnr_E5hf5$wxx)l7 zqYWHBn}LRyPe7>8 z@cg^4pM7b(hOe$JUS-6oIf1!|_1e=vquDne-t#gA(ggT6MwAxKVvH;`i^b+}1-i>f zoD7h98Sf%_5{;~5HFUc0#JP55sy!CdD)SPHK|u|iahf2c-u#;`q~H`RRYj>-B?Q$d zXdS7(sK%_OzcD{t#C}nUf}hVpq_*mAG;=jAcdgIUCbun564w% z840V|W?S>3%R|Kr170Emigjg{2V`bwVfI5QEeSBkhjR=;C7-cbkp>Kf*Zz2-cuv;jLJ2w-H4p_SK$dGrAZ8Dj}S&>9ADF#15+9#oh^ zpVO-F16d%$CU(#aLLbFjB9TACEE+i;$}}biYEYo{5cAc>8b9;HIZ(A0_%uszp{Q6_ zvV9gh7}K%m7@~^eg=i!D4WHHFya7sW5qp4iGgWxI#-3olEQM?cda+|BK**NQ+4cL2 zi#!0G0M+Ng{7eRcK$*%YZv}uBb4(E#4nQlMlFk7TM0aXtWE>uA@?w!6ruL5qgOT_H z21$8*q&dX2fwAxdxuO-}XTGiJWF3d;*wa7+WH|VMM~*PPMeu>V+qw2~*WfdsX<%y< z_FVnkWBSb3p2sq%)69JSi@_a{=|BD)iMKL+BxaO7y~42N`qio@J?eLq%1&rk67u@v z`OTYmx6qEx>8$ze7^Z2_9T8s5bojV*K5I%E&jN@#7;0C`C0Jj1usx4PV_MMb1g25N zCMW{i|B8cN)pB{j`i@LQ(X`5LJ_rrTJiyuQ``i&>0FG!S%}mellfz!kM(r~-sVdoh zkeH^?v>NC;*zVgL&e6W!BJ|=a0*%A(8hRmCFv;sYyZ+}{kKH%N*_M7VI`MBA@NIdR z*-Tu8>VHPypfZw%>XROnrkran!iyhpW>Wf?$C4bgKAgih$w&NEN&~9wj*Kfz${qb9 zEX&DHTW%mZt4!Z4O2e9vKLInB@OEqPnMg*TD0Otsak1{4T<+X0nY1)ucj#- zn66jBFH<@;iK5C}XalA~xxP68?bH?o9+iH5Q`6ROmWLZ>vDVf(5c}aiDRA+sGf~7^ zTUn$N0td$$QnX)6aY-})Kj9dMCi#%tmev(24*juCh&S9>tTWVkDab?8!~utc?y2A} zPOpE=PrZPtR5tZe8vF!|0!;+w*BFdiK`?WKS-b!rEo&5iQ5BvbHmHnS1Wy`(cv~R` z3hSo8ASef>6%b}DFhRV-&&o}7xoPv;EH0#`DHx96Eu#ZIvUH?@Crn)tvOT-^fIp z*$+by5I@;A49)K>D;=_2HCt|!6$A_Fvpag=lz!m1XQmS{OSQ-dg6RZM)?@sp1sK;O z{3e*$0b$AEP~Dl^H8?M#ddut}nM}Yu_gX0=n5tI}ML5E3h$@95T=QW7OF&O$G=vg0 z-ny2-L7uc@=+yU8v{W~2Hz1vOH6KT9Eeq< z$|w{%y*Ph;b=|jpSFrU!28S?tstkuZLC8Rba;*2NaQiOvitvznRj(tfMkOn1ShJ4FPQ!in z4Xjm!hUWOI?*3B?$ZS*LEmK!+Re6OO2nfCn$Tor*lL<2rP?u+c$wo7z^Xu%5DuGbZ zO;A*jji>g*bXA4JE5?X6vtL(%?|HHYT5VNZE!A+{wYGw|-t1UaWeW3f>=-M2R*sA( z#}421Sk0xA-lek*0NWk*vcU<(YNx8!7@0J$M5?O#CQ`#3aNKATH<6mK*(8ug^=pR{ zJ;+D|+0#f@O7@Zh0d6VS&&fwDc_|*#vAPqzdiCS!#cd{yYz0p9Vga2J8f46^K=<}U zpGI$?wrIvB>~7Yefpgq{6&Y%x{&Z*tE}R6K3DB0QQJ&4;)QW(kW-RykaEp~5xnT{d z#4yozoQD+Fn5;(^n}riO)liibDLCRXA}Lwc%F0~62Ey4ozV=Ij)TI1Ff)xx)@)5Tz zt^1VdCA0WTr{B|MN%OXXLDP(T%q_|WpH?v4f+;5jC-9P zuw~E5tZY|nOhhRHB8ZuefuNHmAe{jz8FvFOF>ozrFgFh$nwR85jauDQ#URLEQ{7vJ zLA-pj0{+0d@d2@KNHm%-4F{{3D@GJxL(GjvVpO(k0gz&b@aBYE07#`fi31<~5e10q zp8x}H6@lhTyj=qj1R)jsMgh>(+qY+60(7qGSwlkVvdj(wW-fxC`RJW&=EtMb0h+}M z_R7se(7UP-1pe5d{6$Jfl0E7w;1JkBfDPq#&;%HQG(Rx8s@Xxa+4k%pjYUq!Am1JTm0Y#)O|U%vaXZwjt-DP@NhB(}zv@gCf2|98xg;Zn*) z&n#JTDW#Y=1gu}azj=Fo)o1Fh;m8;#N|~@1n$tx<+a08;P(}Nn|2XxJ)#9PX=9i%14C(Cc*J<-{;y4d?xFvh_UmM z&HM~TYK!{z2vbAEd@LkMT3=wh0FNdg)e2EH8})v74T!-3EMpMy>Gh92t|8d$s8592 zIl%DTlaEl11Pln0k6BaF9QSOi^R#Yk3bp|m?ia!)pz~UZQMg^j0KfpBWjtWuMn?u# zL=M$l37tEY@2CjN?J|h2&mbhST^L#`{dNvR<;Nw! z2)E8Qe0qL)b9>f%yx|J8TR#wCzJe56tnX?9I|8hyAR zz@rxJHM^Xcm*k%H;S%VQe8kd}QViPvyO^DQ`tJJbZI9Z)nUS)+ZP!4s)Nr?gkK?CB zK!=P{J+q$W)=%rk17Ibt`itcu$t7kd)LEj0miYk9ljL*SiomE8u$5jEvg!VFi)Z zyuv;kO zHDIrbkW9xQ$O+Zq)n@i%5VT;q<^A1oLt|BbyG9{Mx177}p~m3Qv;4l=tJUFg*(O7Q zr*2H&76!rW$XGXghCBS2(4O#S-o8Gbfv}lZ`Wfx^p}LA3RYaH9bsBmCM~=YNxnWnPK! zA(k4gok8Y^@iK;)p+;#+jdV-h zjI?-9R%6`m1v=|FPKpD*IZoR2XAJ7+K3W4gH{;=ajvv=T^R?>0lq!g{M41x`tG5K; zw8(zkh4I+2d}w?&C3}5H_|<9OR^MybO|?uy7Qp#M5;D>ngLODyRt>hs7Bfir16L^{ zC12IuItM1) zjuxv?>j#=uPg2KMb@$PMbJ>L*%CLxYIyP0euK;`~$0C60ww1wMFqVSKgo%|`oTFU; zTsTou5f8mfF)5@`gF@v^&HcO}En=y1r|s;wQ(%rZZ^}HI=cQ}q6tU@I9Aat0%K!!tR9BScr7Se zx`4}cD1w0nQ+qHN0spPGeJ~|p`JJ5(2Ri^ByOh3xi1ZmC(q|%!l!pK3z5_#+fJebZ zT?URo8f45392vVtNg|bERpQ#e%q65Rf%;u4VS5#Ro=)`CPiDtu3=ya6&tVvwo8W z4hCOsd46pL3;*vb(;gUhr3Oxbh=C1uGx7WrP9Qx+$`sKUp_Zf!? zc7CVR2?b#?ICKmOvIvL9uf|@(KL5Cv(_n*Uj^TkGuj!|%ZWEfxYFrP7Bbe!`l!lEe zq!f=pk8~M0GOn3Tvi!2J$x~bq5m3FZB3=uMmM;1tJ({#N1gugT?mmHkh=H%IPI1BN zIBD3f(FisaRHJqb9{u_1q9^gFSueqApB1H~1RB!#OfAc3?1cLX?zuM@jG7H5aONzd z+FSK$$Z-%5Rok5%_sTEDAjn^Y!59xixWA17tH|dGfkX|>E)Q)E>yBI)qBauM-1QbF zDq1N4B}P%tMxJTxP^y(owKwjFl5|-zI4PviYDPwtVFN?O=2YoeF*AyfDobNHZgYCm zvv=Nb3o;v^MdlqDGlQk^cZ*}#5H5?i&!RDy=7^$x z675@tx8`7;w-uJYfMW@RQVO{+PYcGI@|{myT=Clu2V@KA>>TYl>6>QcP`z14CFq$eATlI7*4@G_PmyXq#$mV zha`}aF-SCDjLJn`g7=CUng*=|!9*QAgSkujFEi^jDB27SiCHMBl4vNMc3zjSOoi%V5&^N zd4wnX%3FCX!c};1R@m) zAO@?v)S68dk2N!Rq$6T~xqkol_9~ZJc?^k(CGZnyL<2ttFtoQC{AnOEFlx_}1{0t% z9@k&$nm4dHzf!pzRAd|G`N7*YjDgx3Q3R0rCbC+TOeP?6JYdc*lhkA|BUw=}3o{v6 zKz+@(1TAo&fyRQTn>OHCEMgO^A6sBHc>4?6=68C=+{vWq&)27yw?Cg=W)H||7O?-) zEI4x_69NjhComw>vS&tyfZ!Y_@F5Ij33eN-QL%9_)|`NfF|WvH67~7I?n+x+#3Tx* z435y-s(3Inu0)I=OZuxQ=i|+=g}etpf18`%4@L{Gd?tg@0?vqcnp#bdN`?+Nz&a!q z&^S*xYNWIp$7wGYV7SAdybcGig6V7oc_=CS?^{%rDsxf zz>hIKTGq))&u%$i$Ig-pOysNRzQ>rY(6{%o>}H&zn6_`U%R5CAyt_TS&8q^)-lr4H#Rg;ci7i;Jwy8mKPJ^P|7&Y zSm2@$^NQ$?EJK~()VX=XYMpN?FU9Z&O_ph$Zy^Y>sIv_|>eq5*gG*FYc7`h(s7eNV zvaV_7n-+}M(l&~Q($)zpFd!VLs9vsXpksKtM%%4pX&gXff6SPuMSo<_W7Yj@TC?K>^BUwnSeGkWN2Dcmk#`RjPN zg()?!Z70Jm?1X4t41L1?kSeU1h@WX&97c{vb@r<>q%MC-=1X`g2CW5n-d@V$v{xk0YL^dw6mIZ0CUseD4ADEvJP-d;+{ZH?Om-Q zpHk4i2w7aDr@}HYi1YxFN^xbvGBDU46e_A^+Q9a%svs*v3gls_xJ`!?2=c_S|9(V) zy=zwp_FKdh2=d&Og9S03rQ7%_g;{;s2j8B*{O8$4F7*IX3r{dg)^FP2$r|!j-o+Z9 zasVfb0-6j)p=N{I{pYNky6Z0>M`j2F{TF8(99PRY7!Yo5;8@5h#!X=LPBv~9FrT1K zLS-O=&8THWrsEJqt~I8st?b7k$bi~@x`9h{*|lv0zOibMQ7ClvW8aw_5XyGwlC^F8 z^oIc=8d#~ado|A4+uL_N=P=9{;2$mnzbP}~7^g1MIzGE(T;A=n8FeHBC8Xq7>uuHO z9{2$YPDSRgtQsm=G}N6TJ}F4&$ImDbgVV~eJ^Lwvv8Pc5^SKCfhXCQ$Jjq9_F;Yyf z?HMURuv3$0qg3I267=1#?|aT$qZ56LY_?L9?GGa}1Tbhi>hJ(adCg~uepgj&;Jj$J z6GoBTIJa?GB|!j$pww+#z@DGuEjEbpfORO+VFXTXnJ}oWeBQO^7__CCsjZtTKqSvd zqZZmD%L}H-4JN*lex2Ilo=w?+;*LTF)6& zjeDq6(m2)%T*wg~Blm!cAE_cI=p!0xRGO6%>dqk-eW zRC7;9CD@tR>6~&@f-?-Z3`vc4ao3P^?AJey0i^GLzPfq)^VRh`5XH`=Gz-kYSpCG6&|lR;%*^ur}=vi2Izob25kr8CX}9UKoSYd>@bQjF}D z3cg0k&Br-WUG*jSq!?UVGt$GDwEgneXQz;H(y7MhGYG;vS;*+{Ac&aP;jjt($QZ|b zhR0_OR^MwlcrQ7ZQ=aA)iBf>(1pm_QM=Tz=~L0>6cXX+_JO=R+uOWcR-$*bTVC* zv~LS&wD5afr%K0)bJ33PPcJS`dzM2qGuU>f2$h1w)I2Id#ABo$U(Axjq-Z5d;#e6fyCTbQv%R;|@Q#GUkM&vO3VS{K@E5f=4 zB9%fA%x(7DLAp2#JXjWcOXx|i>J6&atI|Sq~;<~Dm1Vc`_aR^8(swqiOI8~J- z3%<&Y@d<)((PVg7z!@m5sCsDWiwGJK*H-md>Uc7bV2}A+inOAxWp*M2rjbWT#zuY) zBp?8r2Z^+YuZ<~+7P#E@bXn586k=#KV@(c!U_`2PtT=0OT%1DOKf8BA1N&zM=agrx z4m_T<$m$p$EPyPYF~I@=BG?@&k43jtu?RB!mKyvX0!9rUWqAUKRAncHA;!ufh8fxc z1r|3M)(jxw<^blcNWw!rQJ(LT^A%3mDfJX0!UCjhgD6^P406OhxOq-)9G7;%TBhJ{8>%=aP&N#9{jIULDH zEbVE@dJsYyL_5a_=?{H#3v)Ob9rkl75>rgLmO;sjDcEmgf_t#73&`1U>BNv=Mmm3} zn<1czuFOn8$sDz22MqBEsYW11Kol9Q$sR|8r}?T_R#xzER;-qs5_qkxY;pj?{OVHL z4|DJV4cKg3$yynaC)5JZG^8LBRMQgCHk3 z|1cOp7NB)7VxGg0${7<7m{hJW!x5aItg=xOZ@XI>_W%?EFdyq?z&wCg8Iw_H4SCIx z^K_#9_dnHghhBMNufzkrD|JBm8o&BL$*zO81pachVD(H^_ zrFXYKpIyJcy6&-#=0QqC29_F}dr6r_GC%vn>nOyaIyp*H3QudC_jn$^!wmqq*7Q=m zAAauJskfQoR+)%a()LUjgA9^9^2&0&aVFs6WEAAi=RKz<{_t}&0D_eoMHo0ij;l8m zILc6$Q`C;@Sb?y4>wwuW$WogYZ1%i88F7~|zu^vfQ7Eehxf+X|?ux>U!bZcZ+RPHf z-ST->u=wQ8b`Eb=n%6K2a8F<5cdEmK==xCz7Z#;()_9F7)S5jrhZJ|<=G|KQ}5b625i`Vb2Gm8x& zk84(Vyi|wL(t;bLMwM<%&t)g>`fiFJ{U@Z(Ah;_tY`y^)`3s^?V%umnTMchp4kaH5eBOSRSn9- z__gdge(8DLL;~0!WE2j~2#D&qX1Lc#DHefi2BEj{SOfxA3yL5Q*XeW>@>)>jPY9CM zvTqmpkqjxyL0ujsIC{g=n_yyM1+cwUFmlw}H5NfALMbv1i%x&My8fwWAqD0q_G1z3 z6yshqU?lTR&tnHjSwE9Su$-Dj@cjJZ{AGUX4vu0yE=wXnyGrO_I@mJq;Jwzp0)ub9 zUX01uYSzavKn99Mx|H&DPlCbxZn0a+3_chBs{B%Sp!G>6_E*JiJ0?0!0t|5jOBX^7sSG z_knXqLC_30lNfC3HFQE}4uWd$apR_o${>i74XW)Tjw4ZyK@E^pm7ULAU@?D&Vy&y%U#OFdc^=_|-atp&y7=w3K42IMlea?spDEev7nGrrz!K z)#Ys-ig2@XuktdXuZ54+=aFb63^i-C$cDoZtliIBLf4f;U(O#XgCSl?Q^gFzZIeqO z2zH2>;A(<>0bd)SioCZFLHc|+T+g>9O9&FpbzXKl0{LYlijzh7gM{8H04aL!8h$43 zHTvT8^7iy1zkA3r@aYnlL(1r700@`e%3FB=g5elr2dy@vT(QL*C%Q6r(2Kh3fg$P8 z9YJ4)Kta}Jwatu%O$AOsB1nYw+%ef0f+I5`S0}_Vfj9#sifujnhQKtSBAod=Ld~+6 z*J4Z}H!k1aDhj!Stac4UD}S8o7%S+-+4YZk8>oQ=gC}#`yRRRGf@8<>C^TU|wbiIy zjGFZvSRe;Pbr%(bwp7J51cz|7_v7R~4JjM*#(% z9-*O%AtuY#PdjO}RlCLta3VD_HV!LZ{(O1X<1(FiPbg!qEIpj$fdV20W6YK13O%yH z{?dGj3%_$HQ8!fp21IA3%TR)omR69(T?HjDo3R`v08++BZ7|P3WZJ-lN@i$kC|Yz* zd(GmQZ20vAL*>{F$8`8{_^&Vb_oZ^R`OhQIG`3 zQ084HNDPHj--7vy3?`kxOm`6`jhstg&DLY)(qVK!CFLxl%1iNKy@B#W85ZHDZtGWl z+bhs^ixIx4nPHZ|q6-_nCk90ua3CRhib>=x<>@fAT!53lrRYu7c`TTcQlTryA=nmi z+lEo&w$R@-47n+jQ6O{{7&if6VQFq;KQJwVttJTV-IQelWmL!LK>Cj-$O&V%1t7#XrM z1>&X~MBr;ha9QG?Eik!$yAIm?uA)P0m^%@Y^dV7tHOY(f+tZ#!|FF+fuyJh$kAnR} z@-7-IuLUJMCPuVajk1WElb%CxU)0SmT>2c}GBo1lB=cI!Nfv+(~YLssnG(Hgk|_LKoodyc_4x*EzsFxfT-EvK2yUDqrO=6Mb+FMh*i5i5X*olm|E8j zjjEi?fm`pCA`-3#n1V=w^*kAoU=FgK#8(TFygf4=l9sRu_h91)*2Qh#NxQR?g_qF1 zLgOfnN$+|f2^KBDqVOOznZcwr#N=d@doc;r#28GP!yb$yFsV6idcs9@^Pb^oLp9?E zVN$hJpVfe{HShB{-Z~1-w407aC(6!hbkqVOSo|P@G98E@olCtTZDzlh29kQbZ7;1+ z`)}8_Q9zL~w$Wd&`#dz*;0ewk@c@&7p%WiB$ivXolY*8@+$e6afz&KfA!TCMf30gE z0f1|0vkZ**q#(eeS542lzT)sTDv(m!2*TGupYeDUB)w0@qa{pwu3}lQ1xfy#zY35v zDys+r*f6iKRe!BfMYwB7!ddqTrqR30cb7dWi1R@6W$>uMFmcK*l4YNNeHZ8WDeq@i z+=AzlKQgFy<8JKWB`mwm=6b`HQm*aUE_TG}T^Hj}ctIDBYxMA4PvmcoDGf3>*$(>Y z3Zqn92j4}YUC_aksU7^=>1E&9U69{$vJBRRWcrAfb&5?}cKV8T4;;UEak0i}oxyr@ z-t-*6^S5XSMBC})!oc0vgAG;6)FuZw7qtSM7-}w8nMfLCx#4RUgZ)XSt4#1oV>i8$ z#)Pd*)smU6F@s}_hZymjZQJn6V~sNIT4OF?|L6%C^ToT%pKfl?vJ=0@3y9V^n8|<< zlDPPir@F<9LE^FoQ1Im zFUf~ueExggv|E4BT^Y{UG?U)5AtL{Mx?T< zif<*;2M)Vev!H_M<1~r3vOjzYeZ!M?w~I0^)Us=s!Lub+leSO(kK5axwA*6{#LddK zEJcAYvVLGhTJ9O+r2!fhG6kc>8h2+huM}a?rpMy>kE#~W@^Gp4=pZDjwG3(4?lI`E zjrYKUvvDWG5FFy!67WhegvZWTGj<^PU){NK9D-!KZ5umxXY07kpfQsufI$7LrZ;fN zqt|#j2dpEa*MghzaiC_2(ZvJ`(X@gwcwzv9Etf&u%q!L=`j0rKDhQ$l4Gspm{e;S( z$T?q?7ETXZ5ee8n9*J-_*jj#9ALw?8_>rsipXZmbAh|RBA3UMJz{y|`?(I@!1_2hdaH+ryf^`UE0?}(Y z#|AgH$-vKl#`7K1;0F#6R;5%1KeBf_@0sk;AevBO5=D-dBBXK1hMWWnTh~EcY&-zf zoWE^gw*{q~3U?(4s<|fwEbUOUN?X|vK@bl_tlp|ewm8;y?HYm_m&-E77W&WgYuHXW zKeq{cEY1V(KZ8TdQXGQe7=uI0^{Df-=Ad5lD0E%bAA>EzHHWGUg*Y-aujP+n8h!*; zolzkOngK$O2O)?l)iQ!su=Yq5xk`5>3~H7Yv)bl5xZ22m2!cGgZ6nZpb5{t2`yCV- zL9p=X(b4x$e|YxYm(RZV$BQR_{QBwR-+dDQd;9WbbNK6t;6FY*q)KfD5%* zFU{}4L9^Jv4$$xfqUwwFda;?qz4>gp`Q%_$vk_foy}P+Ry*@bjn+_0`qun}f&vpM%TOH)o%J`s(WW&!^XKzqq>m!4o$GL|mxrfax`!q|VCQGCy zOQj}DrY1|LCQGO$OQ|MHswPXTd6ri5EUo5QTFtYxnrCS>&(dlh(CYln+eifFM60*2 zK>{vcg@1lJJNuWH6fWfa^7i1D(+d%tBYq`%k=l}c@yOQ@eM7xVenX#JLF-gxnNFwI z*Qf6vKf3-OkX-l<{=PZCe0THs(e3%0Gx%|3QOWLGqQp`1og*pWNQyqfN=?P|F;_+p zxjW%s-#_yqjr*_`OA}qgZ8wO-+711+T~2nVO-vmuAYi-0Ix70G)zzQQf^pHenL&t~ zfkA|$%$r(i{HEbIddc;l2M%8d8)5!0EDY=o-GskCo&D>JpU+z8lbbUh`ZVT3&bKm1sd2&Tyfe59_imm5bQ&tH*``+^!^ zIVtkvvs*8lp`?=#yuCoBYk9(5F~mf#YzG7i1`GM2Sn7!=&~au$r4anKu)s>c)hYHF zU6tV-GD*nQULxTxw4w*I+>mC5kF)`gjU!&7y+_gTC0;4(l91ljMXGA}fLJEAS0Hci z!MStd;Oe4L;f};g(GJ*xG*n9~%55$I#9G7+1sp4sRb^OBefD<&xPYy=dH5wV8-rHF5oE^2#;D~1!T~ixgOrj9%(Nz;&;U8oVeA7?YVQKE+_^f z<-4IOccl3H79Iod#}-?H@f<0+^Zp}0_U;wtRR2Kw+3s7UlB{dWF}JADwrlEMLt7WM z64nNPz1m6COD$yA6y;xN8k3fg$2GucVp z)1}C9S;&+Ehk^^c1OsCE%wDyuU6~i=X+CD9*hH}$C4`ojTsMtP+_rtC6$xvB)QPhe zj1fTnhZ@uffFX|}1{z=2H_Ycc4*C$=z|=`7c=VTkfXBr_frt~R8+C#LQOTD4+NnVM zmWlU7c>(rQw3VT!%fN{y3Mh#1?5WbjaS2n&7NDO*-8KYu1&rxFmDDF&)EmSDSJej^ z6hJ1|;D9&tS@Lf}MCB|A@qiU3QsJ@Oz-f5QmdwD$MF%d$UiNSK$f z2?8G>R-WWDD_PqF-)2BgG$Ft-n1B3N8v-}1xJ8mh;1|b)ta`9l^ubQlLyROkJeeK@ zf-f|g7M5#foSPLB$lVvU(>6|oLK+d2_hcw+>`xGCIf8dh(%rmE;ftY6LXCA)(X{wm7w$Z2B4dNYyNB(4n(XwqOe(_G$v4aC?CO zXk)~9EQAVxF!-;~wzj=#du_<+hVeIbL57&yRFtyNf*4JE^D!Bi>`h5PneozUH%@Gn zVO4r%MkNMz9!<@CS_R;3o*g1qM0XB?bx%ab+l)pGJfb)6XkI5i;8BW1=z^_Khshe>ndR>2cZbdD6F zs7ELvQID1%>qWSR`I!$ZQJa(}&`HSVhaKQjVPTrmrf*_&1i0IMB5eOj@WS=-6O|e>I#%};la(w($X^* zkP8EC>Rf7|4fx7V$i^!M!;gt(U_9u@2&Rmz@HZVI^HF87*{^SYK7S1a)q&6k1>u<^ zeMf(pJ@~JghN1-2j59FU?zfhaMqrPq(ysa~^Nw_IxQce?Di*GBkbFO?R2Y$w%j7P` zlFSiV9cf2#&o_*G@He6P7re$2AF3t>V3JGuQASI=E8lqQ;Qr-Ska#2|S#5U}M(D1U zWDR`k3!x;(Sr=JJ?wV_kSOr()M*^o!VBwyxc(~*x;e?9_5VT(c2>gk@q&)1|BRS(t z-IHo;D@nIRt&B2D3FS2JA!b+BfVKxED@T4je|7!_R*)z++B&IK`iSi-(3L*{Oa>}b za86%Kl%UeXU3Nm4P?mAG60h`z)<`jHYg~}LK%Yka(ju(dw=J?)6U@qi7EC~XZRY(n zf0e)Jw+Z-+I;?F^@Qgkx%tDS^DrdLF)&mgvb@PrU2c{{I8$KzqGC{x`GL+s`+@>^FGp+{iB6;H@pjjT2+?di@{HLQ*K^ zLY#CmjPfh?#guIz2~EE2(0`8iIiR<~7c{Jua=yu>owTO-3B3+9=|{wQF0LA~vSrXvf26 zAC3t)*eVSzsD4oOdCw?OfLb|`eMqC>I>1sFy+SwbD`mFzh2K>Rd;u7__N<+fl|ceW z)CC9CeF%T0#C}r;r(z-!X;nl{*PTTF-81TVVOOC#rpwH3Y{~R!FCP%Wx5K&yZz|A}Nu;ME=wsQ$Wrl66s)kc$kt7`&g-$D_}RJL%@_N zX;7!@`xRGjn29wC@ZY8Srq(!>Kn|w6qG3PcRtKK)!|Rqss*Y|wq;az7 z+|eP_Mt(j~uJ|6$R4H4tUB~wl4h}Iza^6qp)YR67!tK1D@{_(0x+OX&`V1TBTmWP5 zhVflll?CsLAp$-07ie8+uYxz>E~(~jZNrfjEJ)v#-++LLNCtd>FN9C%{{#H-TJLxG z_4DhWe%zvO-)0wLraR-7c6%6MAJ1195H-v>O~6=n13K&r{Sl9!@4d|L;>53Pg!aKIjFq!%7eH%UxX z9-JK9m8KHpR~a_ypt=?lAQ=K2_kLV;B-IvhBU+4XL$t2v{ZQ@Rdm=faFi4We{K*1Po3z9wsmtS~$31 z*(d-8`uxEH138|mNTSY1&TEbtnO*l<+ngnmcG8sCDs_(N#aeq5nYSYOj`ts?WJ8GnTRCbYDNR}f~yQqu9#lr9Gik*u9K+#nsSCzK}~K#Cs`A9L;IAA z;uO^6Lhv{@648!2QBqGduhQlj8wNK;Mqm_MneDQC*bS?d_=>bV>VO%1GbFhe4VS1w zaLtt2O{v%2W+-)pczt?ta|X?W{H3pN|M911Fki;mVXp7;fy)`A0$~n;f$hF@i@A&2 zl$x}2{SRMsH-B5vz`JXDsW-cmDIC#@eL=kRY2Avsh)s zj&u;iT#6h&a%uNv=$OG;5P%8|@<`-IiRI@Sf?N}L2N=@(c{!mm(iJ!l{_8uYOVB1Y zpSXYF=<1WeX=x9*1GCVuBGz`C*lAb|qI_cKrR;e9$m7(p= zFM>%&UuqT9O-QAe132Y}MiIC|-)ie3M}$b)d*zo-yy2VX0Yy+^V3#s)F0fxy?e}6<{I1V-q7{+nT@=fuGTJ z{X>7pbSWBL9rp)oda0enyIZPY6Uz1KRr+@KiZ!80j9N9E6>+z7by-l`!uiqRySj_s zgN%GwhO48|Be;Ohbg(4aJJY^-0#r zav`8lNv62?0Jfi(yn-OV^jAYKK9{UJvgH5K>#h&w5foR0tl0;q3a_B za1)g)h7s6p_7+%90?Y^`?YI4?ZxKcUimPcj%O-f%33D(wvk&v&x0$b}hHQ3@ z&PCr6h8FRxdN@V_I<}4?C9HzmhL(%GRhV~ju{CUQfp8jgk;PEN*Y#qnc>qZg3EnAj zz5CejHk$sFx{VKjPE~ZeV-EJ)A3(SB3ULPva(`)5PklK2L8n`3N`mAtfX5;2l zF%7-&K5sbo=v|~dkXw?cKanPZM(cVzq-7cwoP|iqtM{t z-e9doitPiR+!*Vw>DSk%??EBsr$Kyxl~?-^X7@R>i1+w-@Yo{MQMHktP_c;qe|^=! zK`1x>3#Fn_z?Y}bzJN1`T}AYnyP|3#to+5#XD|O5yv4n7ui1}&BP7JIvwQ|efMOlm zPNJVX(sNL!r9dr-dYmHyla1d1BFN`kqXSjvsPdbPc_AT$|F^&jJRXh;Xcumc1XPzN z0drv?1qQJ~`+P3T|3duT^g5CUX1l^(EAH{Yo0Z&M@TZV>ME|*Y{K@{`eWbwKmoMjs zzrMNHX4{(CY_nM%d~$Gbuvst7FWgzJmdzY~o6T~uUN1KA|7Jd0tv@-KZL?8Y_Pd)~ zSe$?G$$!86`RwxSz20v9_C0TX@Gx%hS)exuup!Fz)z#~R%hNY!pMU!1^w;w@=l^>) z%`9P~2mwNsA*t1~U*CdiZ$Bg)?s`bcHeCTC^BJXZkI z^(ZWW8xu5%8fV%?!yFd-QdK*Sm%$CsEvQ~4LW5dFr9bs$oL&oAT=<|?7 zF`?ScBVptwktY_Cl4m4?he2`XJ>tyZz1)Gga+Y|{ah#DW;|# zV+yni_OrsP_B1kMCXyLnrC)f-T;qK$61v#4)+!)^YrLYVP-PTx%%HfE!5Jp(bT?Qr zu}conpsnHHseg-^T9%s!l<@=-iYVtK=oHfn=@uQL>Y>Qt^$r*x$^BBQ z;6jDdXUx3q#t1FoCqV@4tTf#GVq;V~^*1S{B#G4+VlSlwxB!Md-~cBSdnskD&=%HvElkDkgFCS1kt1 zbAUAom4FBYpW`R-FW!d7@-+AvR+7{Za;rAbmR?q@6o*9s9)gKua8Y~6Rah%cj&SchB0@)WdiQ ziZY{sEFO&7YtJ&Ncq8C|YlItxVDO{u7;L4L5Q@nVvxj2{))v|V=jR38bgxo(3Ojvg z+6RU@<_~5JWttRESf_3aGSV7y5ges^X5>*egn7ouNTS{=*lt@jzKPvBYsNL;n2v4ulrc!M9{e5G>091niI+2q6sM0JCdE6Z_OK`;VV z_=b%Kj*C(zF&@mSCF8+SxT5iB*Rvo3;E17-BeGXE_>Lh+#kW>kFiS*9Mpv)@2yxRT|x z8my}UWa?!aK~e)t~$#V+={ z8v}by{jBDO{A5olk%9EoF6QV9|2B4xLG>2aIcB-WvdGK}^ zKpt9F&IsWA0l7=&S8SHybr5+VGIUId-cgLNXF}9kkp3z`19(U@Nd0y@fD4N7N9MQ8 ziW;>+Ixw1g*^XSBYIz1ib0MT*+Ivn2=Of_(Qq+mbk6K+^z93@bJ*?wO^nR)m?3GwLJ#iE#MZuEWhb=?ckzj^2fgA+Bh#Z{W zCIocQnD9tCKuQwpuqc%XjGm!u-5O{;lkQ$b&m09X$TYtHQHY9ZfK~-)QYGh{yeZYM z<`sMs#ZO2RyeaI9<#?GM(9R(JK*fM-{>TJgJ0GmN@4l6g6ldZXNx8T9@X(B$7Jej# zzyWk1s`s{^_m(p{06YS)1o<6O1{*xwrM07AX{~!_daxN8sX&H452Sb5{s5WpX|p_? z998#5WS~HDLc;#by+r{xpdH>;qHCNn^)cIujsk%Kawt@8D@Rbv!ViD1?!*Un09f)z9LfNi&s(A#K?kzfX*eE4NVqm2NL*Qs%E~Tx?I3Hy1(e6E3&9Qn2 zD5Y4&TBZlT2${OWLDLgyT0+$#;f{hs3H!!*_;juS1g+u_WS^cub)U)ZcrH(NdTj#P6C)wF+5<9o361DJ8cj`4loxm{b~3#*@;mRP z?ifT8o)aVMKTDDz`781<9ui5~XsN_XJRy1S*@$ib&cy|;$r`;A{7b!}I&-}nWE$YC!5O&6_dZTynZ4bDseB%sNz@K3mKkvEXZ?9`$8GiE;% zQvw+VLrXxUk~W#bDAJQDtZx^T-i=F|&P`B)CQ;+UcC|M&8M-|SuUHCu)+(X1W$osW zksB5UY7C6QNSVnCw4T-UhWyV#pEGg7zkUGQ&7%M;H>q7Y@n-Napu1@#8B@V~?P}Cd zMl%>cWed_0!HFWh3^`KhaEJW}s;3&z0{uDv==rdje*oHWjTvkwRE z2vfUBgh_66ZD;687Bm4HY6fDQdWTDJH9Zdq94xuOE0{i)A0-)YwSpsr50k%`{ia2B zRLj}I`jV|FxPp@UtH*TxTKKw`K# zU}$y7X^1&pQz;cO7Pq0DN?NfI*qC5|X;`9G=r$e$GD{gsQw*hy*57%pw%vXBb};d^ z-l)I1bvF@1=P%K*2zLOIHnx5ee4}r{exYw>kK$Iw`W~=l1Gqh4 zgOEgtlg1#}19rDTpjsmWusvYYCK~}6a7TcG%NVeMuBcR|05%Sd0ySDwP(}C`+9k6# zj?b;_nYaqJH%Mf{&<8;dk5rn77?7BW7od9O(iRkjbPf)G36TNDkSQ0NfC+ZR%*9a(~zp{sE1rf-ydZo7)7hWloG+lG%>0r zM$!>@q)#FIX@7)U;imR5;aaF&f$WJa9{$FrAJV|K)D=ub2BH{~iD;&jiYz1`oafn6 z61KTHoZiZfeYo|2HoAQ9;~fTAa8;UXm|d1_WYx1+Z885Ct++L-YerCM z=^cJrzduM?1U{^ zu$yLs9Y8nODF?88w9^hC=uxnqREU!n4jJL%$5}6zDwiiZEO%<)Ol{C~C$P-WNNGkQ zv)HRj%wUZ&h(pYpy+*M*D#dQxRQlEP;@D1TPmOsT5DQR#9L2n8ujXOyx5sR}Gc&2s zI7smkQ7(49NHKjpu2)q)uHku2ZO|T=zznOUX`C4}CcLx&nC{;15m1xe{! z>nG90uuDjY@LDZMF)NLA3Gae5wEz0t6LE~pB3e!vGDoS*cO1#1_H3tZ<0V`*KxgAp z8`E|ZF1cc4atxFjXNHp(Cfe{TFMbmO+egV}RU(?(N>^%j*PbE$|77N&lnc6N(^Gap z9Xgk`t(yr_ZL37=RO9v>`pYR~nHa@w#M*IW#G!JTqG<{)hcV-h2fVv+N!hO>D*ceu zEhNciRU%qE;HI6aq3v#FN|x&o&oxpou{{y!bUdcEI17SWM1d%8NVI+8VploUTYDybJKkjA}uxu>>I5^mj?4)XpG6Rjpnhi2SIzI(|(*y&MvuCd$ups z5&;JH8Qrv3y@ow~lQ8VX>PrlZ4<^d6=hz>yP<@fs93?vl3+K=n772(= z&#G@YtQ1L;7M8IZMln5w964M|lb7RMCARH~vO zCo@x87@|}~CMN&Ykk-IS(WIN89lSp~a13ctLag18mI-+}tBe>2Wetr=)q^P;QteE6 z6@OaRi3(4FSpv0%rt2^ZRND#q+7V&+(xM^cXZ{tZ-_=Qkw^#kzo|Q;!lC!puyV}2a z9ApNuozVu=v;$Hw5;y|#Q8*IR$EafJeLE)K8h1zJR<$hD%FgKvX%W_Zn68d#1VsL|j=66p^r;7eum?eVhs*=m zzEDfzu+u2}Kb87iWc|k*D8e8KX=EVd@fKorMr58Q&< z+_B`I+O5^Fe(0ztnmP*cBK*IXJd??CpOOH56aD}C8%yA|d$Zg(v1N{fUeb}{5G}r; z;c)=Z;7YWhzv(_~t3P;23o^fQAMKBD!1@0<1B@tVCCFXbX9E<3D&lr0YxEUWzShvA z>I>~LnZz|&&5`Ryy^r<9cmnNs#Sb-9X%>u%=Q+8VVLc=}z7?{d7TGfly%rag3qzIR zQ-yGzsY_TcmTAlE7;U}uN=RUT3J4MsMvH{55F8;e+ooxOjQla-2*~y04zi7KE-d*f z^KyIcNu<2>qkS)!3{;}xZ%kCCizRD3dB&6Ve)25mwROq2RfN3Gw%$iK&FXs}Yc|1S$KXWnryPTu_i%vH9e5u>MorFoP3<8>6jhPWny*{f zLx?IDi_o>f>gydZf}h-?gpgVc>=jw=vBIfUbitLk3Y2mQzz)p-`PNyaT7y+&wz2pP zl;LXNq*A#XtHlVRqf0g&6bfB6*_qABqroC99Omk@999OTRFaY`7>1VmlxL1)ZTBG5 zn(}t>pxjj^iEf6pW0RPuUeRiZcrv;q$Ltv&@evHMNSgmijI9b<{EK6)QL>u0ivg)|6eIZyPF<9sZ7yi<4A*8>J82BsWjjYc77v%b99zwxBXIPXTdV1)js^} z1Mgx|3zWc>BU0OzM(7YywtfROkwZTOG|(jQHU~E$qNIui9=YM&efYT@vhyT~*{4kz zRiq(>0*VlOGPJuc5(Brh5T?|Mwfikxa>I~NGdLwXSvS&PhJllVZn@fC(FbKLFp;ah}5GQcqz7xVKHFiy)3JF@NvpA3a`jAa0ysT!zFsO0xgl~(Qq+ricL{4GOV|todtP; z>>`!&qb=B*zgbvQcuV~Tur@$Dw?<<4fM{b-K~_=LwoSq^@RMK?tP@+TzuFf3RasW0 zol=<%t&4~aGaCUbFn@o%y1GCgnD;PH!$YwZa7CNGmoOhCHFVvI!D|`GU+HKRpiRh7 zADo1VXbEBR5~hR*B~2wuRH)IUAeEv|e|Uw7iRbXwM(i4jXUDQ<9Uj9Y>DW%W?AfBX zpxr!&35p6d$yJ8!LaNH!glkCkfbI77s$%NHvsb4G8?uKAo2s>|*UwSyy?YCSMSqmN zh|R62jWtrY?9NiNZA-^mynQVlOMxRc(^D-SX*wjKaAaAq{uFUlw@xaDLTlk{0SYa* zdtg*1El|D4^b@gfALn*yQGN&l-2^uW3@KbjI3M@kR)u2+7`x=*PDBTHaa8?rK}yO=hH+InhtBqFJA8`4&JQ#7a$)9@ThyapCTNr2l_=eFZx;{o5n@ zJ-9r5bN2bCH;}J*asK-K^B3R!@buFIP%uO%B$?_MPL%oeElkR^A5!q+tm7f2R&_n3 zaCFy0Ix3>;A#H)y^^jCZ*F&=Ux*pOl7F`c{+osN^SsL*~Rp+-^D)E^no!@5Z)MN?8 z({-ICXGz78#jbBTKG^v*ODjGPv-8_5tys-qIMkL7;*2uX)ovk4e&K1Dz zHQnLH1WlsGnRd}Ihs8=n)z41$HH@g3Euphzt>!SthJ`^p!cof)8PBf``aC32OsIDA z2)4LMEq-A+4ju-@g(Po?GlTccL&DqOVc0&;TAX7JuLYYwIYq!Ti2TaMnP+GQDff{q zg2Ok0mS#()mP6zW7jK!8wM*vI7+-SO3HiUsFev+kloOdb6+dV`$-Zkoa{|?oEsGEm z-{AQ(S*hTNr6bBMCu!)&>*w!}d}S~k(0undSd9z!^a}hvtt;4)S-n7Y#Dia-S()69 zQVU~l)20S#l1N`@R+o9%5D;-dC#D5Gas+05s5Qj5xrFv^pi+HeP~G;C&_Ii{E=2?+ zy)9NWCQ!XTY%vgbSncXk`kA2)=$DEe&56Hd-aeYW3m>_`Q7Ja^Am7J}XjokB!)TUE zdf%CAQ@cqdTYkjb3#ED&KoE7+AGYFvSBVe_NjHwIi{7h{=r0P17TGZ=-rmh884iIG zb*K_T8P+FF59`fkBnU=m>p%{J|2l$Zcj006Anc&6jG%kKa(P#&ictV`y`xC1Dbad6 zZ1zfecXm8szzn3kpA^LR-Y zg75Fg$RtSLZQ9clF<;To4Sii2u;5vbVUJu5e40>^a5t$UL(_b(BHdo8;7F-e8k=5d z>8n0NdJC8c!uTII2p(GCVbXd>sL>)js@+@dpg@hywYSm06$rYfu$dM0XlV5 zH@r`!CS0(+pYZu-`{=$%A=#KHVS#@#%*a%S2BH(rWZy^kfhyfDrfe7Occrq5P_FW* z2v6PE39!~t%7V$QAJrN`NH|93@LH9!{tCajQufh(4#zl|*+bGElF%R7PdMISpj)@o z5Ns3Pus;q=dSD13-fuE zlb|h<7(Nn82S39~!qMTT-SMC#T&M29zUB(4*&dqoy3|^qVfUbu-bv%`DoY~eLXvsy z2zU1e``%!;#vV%I-e4D@N&X~MgMdsjx5|gJ_USmXy(N+r_{{0H|CZB{^RY{ z&3+Z&qpme%_kR^&+tuRPEpd?uLBcqW)w$d=Q)L;#Q&e27L_z72~`G7&9o**F9 zsz-ZPqaGIm!AXrMt(5wsdYvlm=nj4xbO(k43_qU^>N-z68)SSIyn-VGg>}V_Xd;xO z?NQ@(;+2WS>1XnaYY*S$&pX$8g~7bUKsf@Y4n!}eV0<2WyCPoCj68$P6Lq$@o`~_7 zxbCk;4#CFLVV8w?@QQP;!AJB1SKN?b*zbN!6XYe{bP6g23*r*snG}%3aY6wSFWQE- zN>n~w65mCNdt%xwUMgEZF|6vpxO-BD;vuaGZzU*Ax&SG}D{YH8$cZg06aKmdJ4|p- zp{RI*F(oRAjbd8Bpu+9)qi-ifL7cj~1*imxyO}u-UWbfMUj>Zb!OWhvmS5>}B^K4A zS(^Tfh@^oa+zV2MoSR_gvRm!(;D!60EMKnAyLrACx*ryA_Y6pa)I`iqQj^3xAR&%b z`FD5w;N{~aHIKvt+q5VZWq5sl`Re=gU#@PyIsJ7XWq80*hW0*Ctf4#OCIk(gPs!uz zd`f11*VFsI*HD{Qoa@!uqb#jB$*uF-EUgvj)R8*-NQoo;Njpswqb1-o!3C(0QnCTQKD7_#nLq{V79O86(b9L zte&NP;A(U`L-|UiWW3>CqyD)f*i__(@n8GMeI|?8+Fb~V-r=>vThKCQ1u@{FVbpEnLHC=LN7_LYjx!MU*T&+7^V>-P_L?+jKdquofmWl%0p?+(S z`Lkqhm4`U&_>!hP3O=@hn*XRo#gfJEQD843d(`OikxO+s)r z>$YlNrN;3-tBzEAl~8_Zw>Rg|ZWEmWKM5_N+ChIqW6#^a12k5oz@p2%KJ)EPv@~p4 zXpUmbGy05THC`+9tFoQrr`3UkU|_aK`i_C0|AKiHN8>E+C9cpBkdItZk)r|xc4TKw z9FtkD@dv0L{FHqvJVrb~b~Tg$7q-+y69V2$eIf10d>9mv2*5GL{R4Dzd>iE|VJT6_ zOjI-V0u5grHf;%?*5N=f99gI$3cwM?MR{U4&4ko+KvOZXR#AMILr_%v4RuenJX4tl zJM|$_0Awuu>kX0wU8r!DIA=jBNjMa)S4^OeJclSFFi{a1Zdg%Z?%AT<6^HMOEKBGo zjDFz4L^h$4CromUl%Nh5zhbWj*d!YFfEKmoO>SxrhexzXl_2FDgHqF6fkZMOgTa16 zC8bRp+nILQG=Xw~i4Y%1Y>=B;AvgQYpif1@z1XH&;=1OW5FkV93v4C4lKC(wz#~X8 z#{@G2f-})ia2!}p08%NwEL_@jp6%)owRF_O5ri?C|LVZXSotB-iZYA{FYCAqVew~5 zNq*Q}V<~(T}$7{fZnkj zw?!8X1Zsc8fsQVY3-Cpl9D^T%5V)^(GrAXZzwNZWDkDK0r6$#Le` zI+&COEn-qNh%!oj^jODC8c(HEU{am|b4;3OY9%HGMiV}iAFVS1`^u!@?|3Fn!cy1j z8$P{{$5@O$Z74r-tw8XQ=J8a|Ej~BoY5d*AZ&m9C?zNINGTd=w|g;>ZH@GNldOGiuZX<8 zkj~|!M0jIHsne;CiHfX~>}YnB!y_#Z)sE@``?ksp(B0LlDR{dWa#tyJ$6oc)St4CA zpvGE1Nm=jt8n^8 zI9O#bi%gF9Ovqa2XogK{jH)+*G_Fls#GTxr!#EZI#HijHD!YFE)^IL>yHVEh`wj#- z8b^{8i;mYI9eRM+M)AM%S4Bq5CcR8bP^j;xkL_N9ykqRxI#RLIsq9MLOHeF9R}=1V{uJ$UTmx=!F05kC`s#9gr-hnY7L6YE6v? zs<*%L&~mZuN+%W#2@4p5)wD}R)wJ&(<+N)1l(l(6Sno!I#9IIi9rcj#QGpYjuEJ5@ zxrU{7DgF9KJtR&ES5ki>b*Qyv)j^Ug-IsAplO2cNY4w#$+{OiKmNDeYkRXW)Aqn>U zVVQszvP6$*A0<|>9w}iR*l*;@ed=44_*A?L<7D_m0xNxcvBRx_-(C1cb+5 z4;|%D|Lq)HLu8NSKYXOzpS1>tD>owo_t4}D5_3{?3vva4CG=Rn`W_&# zHT6k%MdwS6bjzV_GoR52#Z-6q<>U5rr?~K1b*_?=H7gKqfva0@r>ej{X?L^vPTn$M z2g=HgTp148+a?bSBo#HR$Ktq1E!fP&_J?epNL&EK1x*aNQgS8cnAOy<-wgUR5KGor z?ypXB!yAbv*s3{b8hn9m&wm&cum{l~KX(L(CU8}5<)T8oM~GncNR-_hDc1+*uRj0m z9vUesvle9|OL89fK61|uJQ{I(V(*8^C3n#@N8d<)?KknK z`?zv?y5!>$#~@}146=_a2NMJiefo{#e$WjKncahKFv|OoNZ^)@2#F;88fUuTwCCP% z2`AaA%Fu%~Tt3q2a2q_{2b_I4C`@h$uvx>ctcm67qnfo-3KUM{r6SE5|Jp|&KhOwd zGNNQGu;jdLXo2lKCNNZW?b-=H1Od)Fkf~(%o-ww~TQLlsr(V5VCTRaStXVZ}clTfF6OKgn1q9r>8 zgR-d|f(C3QklP_>FS2`)ef{k6_Q~aY2Q`*SKXwaPsS(I(;#NB_u_UVl#n;%$irE7; z(`(J{_Gw$H*&$cx?%12|iX!~(iazLF`A2U-OH?0)dX1};he$1a37{sow3>BQQe%Zh z`@lKerJ}9M)14tc9Btb@ntje|&pDR~HkMlnZgyL-VmQUv`}b&?%ds#|StTPs-5Rf( z(5_r!Yfu+Q z&C#xxlr9YK;_fE#miCPX$n_z%jTMGeY&OtT7}mGjm8t&JH_YeSYJC`1CjXaSqr!Rf z7XG5IuD$ktc#!z453Lu@1nLlg-XzK;*>l*GAcayZ;-d8`yPBCq73RpbY!&RgYbZ6%6-d;LZ4siC(xz>R zKy|Tx`0B4V}^J}2}yGTkPKLp94pzg5|9g6yCJm$E92{jSu4sgBD_ot z=|2iM#oruIlCU~AO&SRIuvq`&%41oThp}k5eg;-fm(p?k>wD9|q*Q@r(x`*ngh;8= zO-BQ|DwETgNy(V2d+q|qHWUK*;BMPgdsWu(Zu=tvuEYj0sJ5=P$f@t49iBIHed)0H;aHO6+VOWGn>W zh;{OpWGx2ArHgg|2JK*GuPXwfP^NY|Ls3e9liP6<9?CR`{HPktvZ-T#!=Jk7co-y& zsyBf&u1#CSogRz003b&7)==5?bH5#TEIKWj5eZVB4!r9w(ZTyFGGZp_Wm1B|82I$D z-Ahovik(j7y#%4_PIQbVsIu}A2?8;Miui9$$QC(4IC}Q2+|(&4LL!!6o}+ zCeX=Fu=tVN&whgSP1W|+9oj9nS-Vq8Zu;-2oT$HNPc3-x#H;CY*k`-V+>xc#v>|o2MA6GQ@MjpoYfc-|U+^0VD+hJ0W>k>E?J_X6j-d^Z}eQF-o9?e05 z2e1bW_~$z#*`_;*B2nOyf^hbJoL$!!#BlN{#Hj0(e*oV04(67@g;j&@LDMZXDBu zC@Cd4aG`pfc-hQSce`8a3`A2s);}<_SN75U2y2A2DQI714dRBuSPhQud{@Ta-Q_6q zN9(VNx8;A7h?zY#{LA7emAgygcg6miccH9I_6kJEaC>3APm?!6ln~fsA4i6$D#t$Jga*8| z9FiVwWsgw}E$em2Rj>0A8ehA1J_gb)GQ(_S;=?mcow%AvKu7qf6-hwqQ&c%`l*7ie=^^?t6lwS~YA(_Ip(RBy4ogc}Mclq|_324YkD z{}7%!_c&+!=C=RYOYB}^x4K`wCZe&q;`vqYr?SWHED3~ZFzD0!tZXx*+2^;0>ZOd0<*fuayA4=*@<+thSpD}XO~UeWhbh}JjrDQLRa<1O zC8+|HkgM)bhsl*M$5!2I#I`>S3h0Wp-xIlVaqG~=>lZ8fEOMix9(4yCilifLHD(vb z4|KDqhbM;Z<5D|fVzs8dB9`2W4y8e7=yK!85ei+s9H>7pt#VV%k6@#bKvqaq>oxvr zkv+OwkTgKf;J?0=t79UPux;((F#PT+rGc&ljp| zC$t0_y|yk7K*Dl$hsnSj4}Fbq z1Zjj?4?WWhwgfuN^yJ#0K)+Z$qO=M2PKhJ-R#V%n0eZU|=RNCd2ZyU$0$l96V@ztA zE0FpLR518Rq*BsEaOltp*dhoa%atpaxEPrT-=SYsNWNG2RCF2KEcjH4fF@vx9W6wGAg($dP@(r;aT=i+u;!zEoo$={D5gi88(EM zb=2+f;?<~<{GhwWD%Zdxsl>IcTPg6%(nJ4cT!3d>sTeC$1_qe`aV5=6f(4~%&360~ zXkZ1f*Lb9k1VjVcX;n+##(>jiEqnK9kM~wsAGM)zRhs z*6l1g8SagIe#^f0ipF;m`+6B2$G*Nd`>or5IlcJ&Ga~31>FTyUJU}|sAvRDX?EKXt zr@|3d0PxzHcfISa+x>h@Xor(mU%YjD_DCWs$`0lbiAiunZ9gl*r2J)!SYkc3eiBBM8cnHRg(yKR_E7k*K1v=r5u(=sq;k5g z0z{3eHi0~@Nn6CBoMzf9e{~2Zxd1s5N0PN=QEJZ)*^8tzs7HW|C2c~t&EW{9zEYXto)bz|G(>@2gXkBQdqXDA%Ub!bf zrdxePkBZ9?*n!%F@g7LfTywTD-UHdO@_6eIcZC1?H)`j>21qE|sC~~HUaXAJB(dOb zc_WDhP9jMcK=2Cs<1;HlyGt!|P*WDitb`Fq#DZn*8+%0`Oqv`TIk)LL7btqThgXhN z+#9(YPhiMx`{O?Kt-`2aq6pIWK~92C6WD_>%usc!2nD(OE^t8>2Mj+9_vjH0b>5EE z?Lf@3U@&6l$CxkuBOFr3!qP6xt6+rv*xF`Tf`sT8d@OFR{h7G;!$yeD&2c#2=>7Y( z{4L>~5Utod!2}u$igINbgg@xsTx-XYGmtecqikPt1`2j%L)EFt&@pjL%EpNJ#~+^TCxPmbWQ1Xr{J>{btaoQbcTypvB@#O*Fw)4WHXD#Nx?(7!fmwk8e-gz;dLN>?HXP~%2kPbZ`t?@5$Nlp+orf|dalZo;`h&sl z5Er(@#bdOrB3FZkZO0rFGHZb<&M|Nt*zD%xhZ3Ypa*UHxj#qT8!lOlBADw$9&Xxah zBxGud;}eH`<+lBCpL)+uy`tzhcF(-WWaV)$h&Y2mL7g({u3J}maC6FTI_6sIAx!Ty%O6gQn#2ri6*F>!&UFY$p5-mdr;;AMXC6) z4)8v>_~8W?t)n4Vz5Yh1({`**Lrs1U)oDnXg68dF2DT$q)QTSioHJ6x-w!x3%&NmO zJ!rq#t58^vk7p_ULc5>0*5>!_iSF`MSVDYc>VD?2sBac*(4)6lX2P9u+sf51^A9)W zRHa(J&Of(x&t6{lChz6#lgsyP`+I|373@Mn;={6H0LW;BBUqH?4t z75NGHQ(qW+E5YC7)8TXv&ifdTDPQBhN`7(t$r! zu2(GL!+b4V_WXAl8v6qj(1L@*py4h7u@V}7-QK|Z`hq3eW=zvPM>NjF6{5KUIyOT2 z0ybB3U((dLyP|SH?xyTc3|q=z4agh4*w5{jS7SoZH4;dPs=w``%ZLkaxvn|lGDleA zS_}T364$bqT!n<_CoXcG5>+%&%k|sswroA-oHWwBMXK}k;sU{K|Njqz4sWloU=zN7 zdqlqnm#1&eKL7Om=Bul#SKpuia&`OZ0pJVu6bD>InZWz(*SBCow;z&7cReI9?RrQE z+4YbTLb@K3r_uG0W(B(*Qb@GxAz5r)5BW&d&Zk)#HCZAxSt{|7n4R@z>C|Ki)dZAk zYxIUlC8i^3o$yb%lOvaNdVPHgF7@^Qu%i{aTe@S`k-P)e2U;xwgHbz&mVlTgZ8C)s zNiu~sGwND*Ylty(1q*sw5Y zM>uMt<#<@zLq?AWeIAl1CRD3=B#YdlR^2pw!Ly*Skl>mg0G*;DT6GZD1?6zy0d8m)+5DY#qKrJH0q}`t4!VxXU$ptP$c}1x z@bx7rto;Ke$<0jlSY20hv<7HKE=tzkBS_l~R;ihWi#ULZ0EJwcK1C$m-k!i zjru!)og?m!E-9!K>>?{|Z2ctoM4OiSRftZQe53~u(rNz8ggD8lvKYB*Jg4?J0VISy zN>B7@7n53ujHpQ0sHu=y4`;QBZd~5E2d)l^E%`yHh5w(uFY9jO#@6lU{0b&7eu?9F z%dxt>UX(;xJf=uBNyX85=-+<7J)r;;3WX|ClU-R^E|ILlu(9{I zN1$>H44yP#@d5b0*q!B6b;~&5Xrs37k1H{9pe2GeMULu|F{uZxh8WbjKI3$_iB%mr z9x!7Qa(KbtR{ZEGi6IU!12x2w*;wywqDI`Dvqv2}i({xy+CGdBI&d#}B#p??r`yeO z<%ZdP4LB1SdVNS0tdLu`&CcF7Ye#K|x~xH(60&A8W>;VKtVu)Lt2dx>YT#-u6~mpR zI%ly|MhNYb!Ks|AVz!|&2xF}zn<@(zaI4R?)?@>>-Y{^fbFYotw`n4*xkTP_DAdOo zx)Ps;4GNX=1jw zqBccN{G_MdWl2x!3;MZI3yEI&8M?!J4}$f|X-oUSH#X{@pc}CO%q&QHZamqDC2jcI z55$HoZyqpVONLO#GnS+d*o>nVv#kO2wFsb`E{-Mocn?gK=mmmWL!jR_Kr%mQ>P^8{ zso{OT59t{2?8)(rw&5~22w^hd ziS+<|h^iroosHn?{!((aBk^bvg7z(y{C<3#DA5AD>)8 zwQespE1l|CFuXR~%9`$se*kZD0Lh|xTT`qZ~Li2iCN;_m2UtaNn9q3+2m9a1tx zFT2uE4WjLuw4Hp@n#o#4Y#7BW(PQ7CW)<_&+qY&wcLU@;-QG&r&Ea2z5f=eoA5sM? z8JXKBBm02rkGgIHQkccT`p*E(i9O#sATVS*<-kbDG-zFxG^9on2R|qXXNK0Gx{}l5p|ia#K!ayGIGyy7KD?kz;x(` zgMGTaDzwY3aT(6^nO#6sN9xTY=Qq|Q%aQnM1*L2Bd*h`Y=VC1uA33 zQqw4;-|=O_r7`WFx|+!y*yM={Bi1cZRh}-dURrBE{Vefz?d1StO%QoJea04XEzjIB zVi=2YUpJ4Sl`Ov2ZYN7_G&=KwLAh8`DSiAwM)cvSyFNS9$A(fm5}%85-A&x^&3Eo<-i=!A-Pyj$Ow zF>vmWcWybf-4@P`cCmQTyATRPX%bnGJDh-ofr0Q7KTvjTS!5X3@<=(S6;^k(Q>JtR zINxtCdZ&XYxoT{&u|q;=F~G%CStrN|lbF+k)i=RlZ<-#-`7hK;<~6fI6)# z;tTP0@v-r9gB_#Cb`p>QhJmIDsB?CUx8bjp`aTIF>gU*?fcm8)7GaV~a?hV}b&bx30D?4eeyfjpkOfq$4G}NJ#J_+utd2;8;__ z4g+&KiZW<8(H9=O$9y?3=NJjUX8IV>94Uo4y}*_jghPOU%wj(pP!=Fy>}v`TbYjj3 zv^Ug~+FI)fe6`Vz-MK-PV|SmiMbybD5y$GyQqfA5o1oH6bam>bRO>eMt6folY*)OZ z-55Q`U*(K$D`?kNw(|!W(I>Z11srcfXt0}W0fIpv6zBs}5u!rvQ$5I>h_<++V*Gbn zxAMoOpA{1yoj1|*?`J8bh zm&bYMjqz|N8gy$Py2P;+Wi*>GR9h=r$Z(liY67l4$o-zC(h1L2-95 zaSd-)fNy*6z+ko!N9%DfYg>-(sJz6JDss!CVH6t=jrh#G89X|!S)){s&@!uJxV1>T*zmW1wrtoklg|;8t-A12 zzalCz(MWJ1&|st-cp$#%0Y}xly!p1AF4dEC08sd;brw9}c;giF!@c2b*&OmRgpwoR z6-2j!Ur&zbcEg2j5W?YXSx2UPrQYW9FYPBxp32ry9H0xnG(`2ZKE5TlGdNmO@8<96 zck98hLNvXbCPSCq6LzRA=_S*>4a2Tvj98N`AXc}4Y@AfpWG`Qd?A1%u`>xAgkbe2E zFko1ZmvA^+_LDdk1mfcf;Kt^D)^_sE=*U#)?*3`W?cU&^f9ekUCHl$NhHN<-yRM!B z2fdQfP;e8vl60cpRJ)p_(hpF3_Awb*^9A=3^meTMd25n0{-4oj7{& zNgll4h57+IY$E{GCmw$#l0O(Y5Q5gkPULjvKf>+^5VFEV6}w0#z36d_jQ(dsMm2B( zI|nVISNYfOOCqRm^R&B$y+vj~P~`1Qzj~`)>1z5CpA45h9w2R~zXn(01#9wpLioMe zVL(u4qv^_qmkwND+`-)Nxtz|eVuLvo-KtB1ZbTQoDYz2d3>od(ijZm#S~ofVcPt*5 z4*hVjPq$ZPFXlqCP(fx<;!j7BC{@^Jb^%dc!<&`1-B^<>N8-rxh)sE_quMxzkqgeFiuhEF^mx8bjpKK>vh`gE7-=ho}u z;}(nCJEKAZa*1uD&TgrP7FwmmCqYF092*S0dD$52FU)oGrZPsUM0Xo{j?K1@HvZTm ze!uJ$?FPcR(Ug@CcZwW1)|9Y=&aBS_+gBks z=kitPGd6vu#IAP~-N4K6p}L(cp9^z(fh{ozhX4WPsawuy%UR80Vu$g*PRJR_MNU8q zp@l}_ybVa)X-qo(CKemY9bI4o3@2#T7?Q5UyjGh}! zIitrb*=djweR2y`Kp!`hwB1|_5DfaDKp$vzQ*@~x{h81k%wS|4ZhW?E$QaF*l4Fuw z!mIdf*|66kZ-8R$)2bNzfjd1bFbmA))Q^T-?C1f_Dg=@_t? zM_ar0706YixEp_we`lcJp+N!$+ySP6%h!nvk)F8h72x)-NsmS=P;+9OD|2mU^*MRM zP^d_cl7=Seyw=Gn_-FOmU0u%VV}T+m_Coj8Yk)P}cmvjK=d3;({`Sx63tLX4p&e;A zNJ)cXYh%U_!XZ5-L8VTfe8rsY2x8g3dM+CT;l~qqc z-hvb^Y=aQeU^r+fP*(ppTSBxe;leGIwTlDu(G8PC89kFcJ^{05QR%*Vyeph&!O|DQZV(0wF zOWymGg=*iCi4;nI3%(m5_v!Xl!fuXXzyk$<*N0STSlevJVO`2}D{F;t1(@dVaJ}6} z$5b1637e}=2_pED)|(m7>3fq@_1W!Xa(AyUxRCQJNg*gSSoq7OJc+{B9pZEY7ni6ji1qxeVN2< z-#6Y^o%ciMKHc6*49;;+8T$1hRT|?4gGO%cAaNK(1s83eFVpo@XR%nU~c?|{w#~D0^ zjB4Nnc8*v(9c`X=f1qU%~L8fu(nJ60YNJ-`g3`4PX{Y9GAg5wv5R<> zHfF{v%)y~s?SS$7;b7$I6m@%5XqQ{#GMwo%yMU;U)SEeSH`W-+k@#u_MQseTJX(FS zjhA+im)5iTiqK$?m&w09pW>#R)yKLes>)ND(MxN+(G`b4cszZ^rq5X1p??OON6<AGD5Z}-$cR4Or8+s3(vkRFwB3=Ej=-ly`-0`jQ@&t* z#uia0$2kQo)5^#G!nBhmH=1&?#C+^Nr#A*gRl$iuN}hzkoL5SzlLXRr%H+Vr zk-J$Nwlam!pX~yj8%;S`j9u*7I>;Nsx$dM03F&%)ypVDs^N1|S9d5vW4Kl6 z`6P&_pJRi8H(8_{8A7mkUpGr`r-iw0-c-gYm1v)*+J~NFv+bjeKemY9FMCD1fpBg# z^eNiSwE)2oFuvTg`rJ`5 z8=ut|DvD-H$q7*|;Z=NAU)bxAH;B&a<7KGj4Gimw&(HG2atmkmrK}sV{t~|=Y#sJ% zXi}YcmN_K?d)5PvA-Bx4&Y-=v6WdrEh)@_xd}15_=B6&LqFCepiEVaQmlNBV(9~<& ziESdSMFA9{Wwv}Xb$*o%fBPr4g)JwNk&d){XTQ{>5D0VM_K9ub=mWkF#~prZodvfP zmm@Pj+#618b2~#5kjBhQGUJsQwR&wmIiA}M7q%f?Go0Aw$P}hTV$*v|@^AZ?RpRMV zI|adXcfq+{bjW};X?C2Ty{&bQ57*3YXV#KM!u@JJHuBgelq^D9*kq6QP`H>+TkL)$-h(o#@{Sf z(qcDuj<`GeNMXP6-pljZw`{MOyP=`q+?#-$PecKUcVhz%4$M_JtV zvp^fG^M2^ur`ydj_=bTY*2(xPgMNKTH5gpv!8kWW)h&M}qo>z#jIsEt#@@u>qA{AJ z+I0(-Rv0hH2af_y=_kwWmR6SmZA;2^&CUFMz+*5DFtHQ-c6P@V*l5=ffa*g1fE_}* zgr8cY@z-V?X~+E&0kwFIN_y>Lyj2hlB}q3Fs?g_V*i{`^GH>t12w8*1NG3hHrCd>? zmmwn&kh|cEeDMFZ`;rLi+dSI^tUzQ21VyseTvA;?x@;mltg+R{0&sU=S8$6VJ^pk6 z*4e!r5cKHBI}gfJsRM#Ob!X+|Av5|lMo<--8jvP5PZNTY<}qZnYb!!VK1YuKom6cB z2m5q;RrX>IcFJ(3&+G!CI#S=#$a$MdmLu`i3W}N|2Pkx5 zHjtN^Mp0X>gJL0tV*;Y`BG!1K!iZrks>)M1)JtoQ*WuTwJ3gR(stcI`CCh(7(Hl6$owY+aLt2(hQN04$E?juzL=ORcKLd*Rd!nvg=Io6c0gRXoZ2DoW>nIZnboIl>V%AkQTrx(}~ zgU~DQ37IEEG5!>TR~QlqKk);faDc#B&8YxED=1~`h>-4`EyDpdh7;T3*rd;G7FNlz zSuE$xen=^|`6JM0Y#^7d2YeM2BAh>{lF^it#Tl!m^zjE7(I>Z1&hltpAvct?-CPS0 z4Eo@5Pi%8X#dJFT>eA1ODL%0+WJmrK%SC7igrE3+OrDd>zflr;{JQdOunJGCiHlSJTCO_0YX1ZW>~TBe<#LSE zpO+s&Q}C$#D~HzEPKkSZ4|0=~(yW$)huO6OVr?Cl-xkv&)7JfO`Hsf1FtKybKM*^^ zpNkB|8xRl89~G~!-|E3)G4pBEHZ10zYy*wssttxG;I|d6mmrHjL4PVN0iB-q$s>RJ z?XkFJ+Pg11M*QJugUX!1F-#HvQXT2<*r`xu%N>lmCt@Bi?&Gkd)&B|!@5#sU-m|s0Q;6vuwgU?@K2lTW8Q$hTp;n(X6YU4l=Wx&x zb%L4GwkQW|qEPi)uB;`BWGJJ~bw#t(whr#x4MqgOQDO=)yfiX|wEK_9Zci!@+mtYi ztaKNO=;~YzNm7h*35?O53bHs?Lj>;2apZr5B{$;>Wsl$h-|MoE%&xh|f2@A`%V+uD z+2P^CZ~i=;UFPKPfBWrs-#s4vWi%Rn_vBG<9gQA8eDdVU1N?ITyGQu*;dhTlqx%oO zee(D(qi-*BP}cVOYCT?#Mt}LA!;{H;a$#TBu5G{99=IJ}h*`ga3GdtY=kv4i;fKMn ze_CSL*WH`@S?5QTOJmDf_vb#NaO1OC2S~C&jn8IXAUOe`@!6~s^nle|)3Z{i+4w7) zMUB5^X(iTr>J;-{C53;lpL%6ZuEUg}9Y4tG6S7K&1_4_bOtA|;>dYGlv!z`^H zW@+`1jf}=IXKD2)ORGm&T0P40)uSw}9%X6uC`+qHSz0~H(&|x`R*$l@dYq-z<1DQn zd$g(tF&;CmN`e^DPR0%8F*ZgGWRRU67bg}jNf-+eVq{u+^zC1Hor1B{nh}a;n#s>c zmb262>Xh?Hyt&m!#(b1%`oZJ+{QQ7{uxX|Nm~`wc>n$8^@t@@N$=sLC% z41@VJ(?HXNazvLh2a5~{s5p;_g$iCH#gBPD(cS!e^6p^VJg)kHpn^Dq*~ca8*tB=JuolN>~OA*(Bs5eL=2I%tGr^xr6VhO~vKkWPK7Hv4)|Jf^2Bl{vL2MHO`gQtQ9a6 zF^iZyA+yML(JXHqvs8RR%u;=_70Ymm&o#y!cCCw(n2$kHK+Nx(PN5p^hEkOwlrKK| zkP|N>lmeoCtkF%Z;fS+BHF>jPsA4H)unfU`jxvH7Wt33(S`=)6-IO;8X(aAhs$Qx9 zkr>cirIVA{Om;?I)L~W#jg4f?@GABg~Bkhy;wJt$>4lMWej z_z5jb?MTc(6)U+1cqrkg#>WHZ!jL`Z%384KgIE^yfZik!ZsrC=Eb)X1UQ%ZBkJzJn zMQ8)aqd?9uM3n|d%-3>e6?h5abq1pmei zWC-R!%f>4X1YD#0OGoC5bv{(5h}ql9Hu40dK;Ysw0Xb*mXNZ7 zdiI@GjHHV;sadW^#0NLp5XNx>G@ zY2SLoY?#*7#$$h_3CyVsk~qJ1vF09EQi6L}TD)t`<`|-9N>rb?5Ydpvj<}lAlLZKf zi)ZS@WflsVOx#e%sIuI0%Vm_~hN7GJl#OF6A*zcbgqYSuCH%;Qh_A$TMpP9RVdm?~ ze&-Mgc)XxDA9!uK5z|E5~J|EVDdZJ?ykjzT1 zz;bI&;S6HQcD7?K;myc~7{rp|($1PU;c-M0ITgtY_)a{AFlx8`k|yrbM&|}4FhP>< zxfob%#Q9Bm)uN0;wJe9jIS6SH1ad19TAsbX2wMj*8NfuVblP-$027gVl8jSlscw^P z39y8vyZl`Q88L2D;=F<(6;>R2HTQ_}iyCzFX-6p$TDXS?JaDlnUdj zdYWFNb*L5vbAqS?)yinhM!TcggpV43K4C)Y-ZAfP>~o$-k()Na>mVo zp>sO1_nu_-9CZeJxzS#ZXrv6}zGVgtKLsHKk44>GOl_-BaLg@X5Bs{E%Y!}7$)HL; z`bj||K$N4}6*-k&WJIkXbt}NUO^SEYo=iZncjzM(OsMRdGj$~F% zG$@Hclsayy82p=#{!L0r7*9HsH)R0{C&vI5pxY~Mjznt5g&d5mMTCQj2>5#NBL+WW z@FQ#|TYBGIpbEFP>uyjHTTl^BH6*FI(PpzjQ4ES=@Fkjq_k%Ctze141@`Iv~iqS3W z$YxH9;J-NF5quMa;i_#FZ z@@i5UwjIYL>k*nc%_$7=GkPkT)97 z;me*(QtgS+`V6|@OPhjs2t}%hL+^G>qI(JOwQlYW-}%jlR*wHQodfX;!;eD*H`?do z9rjd6!l68OD9?4;Z%FbvY#Zz)smn-*z8%M0zP*%;;PTkmEE~eHLpXK_#}46GDJM4t zriDIf@EXFgH$NOJsNd{(^DpLyB%R6Bv+7-p|8+WEeJD7>UtR{@!*#9nQ;#vq;A)5X zCrelc-`d}0W5oWvd~1C}f~$G*f(>FF8CfDZk;s?alEFm(HFBE=D7;95A8wLOC;Fpp z2WgM1R4=Je#CKEWLj?yP7!>tloh`QXCwR=FW7sx@;yjv)rGj{s$G25j%(yM{J^_ia zLN=>*i-ppAg9~+E^{bx>y4ScI`q$Z{FWy-oi7i*I#6CEXz>#*ySK31OL4yH%2J9Is zD2EElr&sTUrd%I^BZt|d4m0^m(}(T}_B3#wd325%gxt&KY@ntzzvX3@pd^=CFG6hWacyGT|olo|HS<-dlvhg2UBEyTiZqPArattbVv*eF?lY{MY z$GnMvCeCzAe3h{M+;)3Pi-`S<4#e!fo(RZjfQ;f@yifPO3XMHtZ4>9f#WG044XTz9CozKJm<)AyD4F z2{U`RXmo0>NTG=)iSD3GYCuP1>4tZNgGN-lYAYd)ou;nxY!)4`E7}^O1WG(sF1JCM zIPW~KOvFCB3}xcZzNYYiaaojSw#XrvbucgQ2sMz}es&UDKm#J6k}vd1DmsPV=RVPi ztaak5!l}{NnAY3OK(Oi~fqlLWh~TWdJR*o5xC}&K(~0CfNEREFQ4%}#f2j4}O0B=sG%ZD-VTSl>UkY3Bjvx0DD(~=DQ&SyGjVM}?z$0hXZ0Q|8 z4s!!5X$Zr?;_zI6Aq=-+7%o$;LoW71PWcVG1f5Ij-iIF#N>C)u0dzNW_PjJ%@Mv4CT4Epgh+xHM4Q}sgM;sC$gkB~bf zg_}({{5o4KP?;0>l7a0`ApeP1KF(``RP|4NF^!U>XI62VolTG1gEZp*hO&q9= zZwV_*eMri}eS|i#U+|~vdgCP9YdPlE7vg-tpt1EyK3sn9D!2 zzIXLW{Q4HzxYzUbE#ck}-N%714S5rLLuk8#?)`9A+7m+IHM=H@26@Knf?4~~Ho)z> zX$>7EsPFVHb3$v;M5yL@;GNXG4J=iH?1qvUX4$}ECXsUydNPT!Ma3Pu=f|etF-goE z_L;869J;#$jlkNm-9**Vr^7pbT2M*z6$xc#3o_VCgS|A^OGA?HQ*z9cra_f59m27J zX+Lbr7&c`TZpv^zkJUpY42vOeS9@W*BU{o{j519XP7S3UILJV-Jlg)lG>Ys0wmxt;+%wH0dh%`&Dte&K zu{HQ`c*jqpBRJ?NwUFc-O);6BTCZ3th*x=h3n`^fzOYIxR=;I7rodPA3}mxvw^%4| z$~%5|cC}-h92N}uN<*C?{x-zl8eunOkVe=ucw)}}@kMFKSGHsZ92B>?P0@`RPt5e8 zdx9;#;T=B_Jz%2NkCUwI+sXnMPVZGZg#^rf`mE3DyxODS)znQKZsoSy5AXOHO^*=5 zl8icg!$Ytg_~xmmmQ>B>90E0UI+UbUAueB(1{2SdBMG=^M62tu4o;(%2iwiGgY%Z! zKRLYP$6iA-7wPELiY55bCC8D_TiOLC!&K)o69hfm#GJ_>k zST^7Udw9o>QY_4cB|fF`ZcrwJGT}{xPNE)3LQ3*ONYPtlN#9VHz;frE=aq?MS1&`E zu#qXLp~|q2^2~#IS!4785f=BqMH-b~@L2Kxof@GA^AaSfigwx&!C7~CMA*^`I}(uO zj@=JVuXhMQ~vYB!kZRUD9qBbdfBhGy`QS6h{b3zBondHHZ3{37uS!z9L1Kn%m3%qsE z?vbWJce1VM2fi)1gRL0vj^Ib`(EE`lD^qs=R&n&i+?|Oeqf}fAN^l?im9X9h>}N{s zJDEg-?WEJsj>5sd7YDS)9|e4RU!XtMG$KCj??-Ybsf^0{kXCEl9~phNc{sJR{2>!5 z+F2eoD?^V;Ubj;H9(BajaI<0hg`djmqm4Rp3_lR0`~VqyG_`ncD2ZW~4J_VSE`;s5 zvs`0ezzc|u2Q@Z@MD@GDlU>n-+gXlA1n>m2xvU9)hK06H>PLT!u@}XUwOF@;GLBaN z;UK*G$ZJ|gESgmn-?QeQFz69l(2sWQw8qhtcoD(@sioqCPNjt~q9Hu2<@2xLU18=Q*Kczlgo52a|2Sy7E8 z>rQ@h9Jw{${U#*<$a0IOoiarlOxDcn2wMwK2FTD?0S%Brrd?>{BtE5Msh(iHFThA1C8snuh$B- znQx2~xb|GpGF^o%9zvwO!~xlSogG1Jf7dNtPa&PRtic*pB;EqWs>XDg!%K#PY|-^v zagq);0Z8nBWn+<^j}Zu3bC@m?2_cEbFEJPLSSvxeZzi<`B!eGKf!U8V5#el&T-Ms- zxEMxYPx#HTjq3d-t<{>i&r|m9^V_hvh_&yJNdL};9-Rt`B7)YZxao$9QA9AYfX)g| z=A7MB%=@R%s>!5VC+RI?g*LuMhS?{MID}IX4`1)8oTVw&vB6WSz4zEyKAMd&hs@3R z$oo_lu(H;BgdH6P?|;jK=|G9<(IB(QvuGVR)6`y5>c%hrt$ke*iP{rVLjvl`g}wxk z=x^3Z!pe?NhaMt9;5eU!uoH~yih;ctd;(wuJ*@jjeHkujS>m6niQGb%%whPI1Y;u3 z_pOW+8_s=xCj3-u&uZD~k24AEX# zm?l*N{!(g#^}cbF&B5#29*wRgI!QtamYYI{aej;$-TIU@NT$aaYwbR!uihp=%`N-b z&h29T4BKlrFlD?O{D-+S?>Ffq__t!Kggmg~qC0}jn2lbb&pwHyH!*Ol()fM0PF`kcOFG^^%hT^OlCR*!~l&{3TI3R@K>zKrBo*bkG^$^6DjNJ!R> z-ME954dltFnpikDuEZ!?l?=}%N_`3E+`C1oFOzBEH9`JRrwD*JQf)o+C4CZZs)t9{ zPAeRaH{?HFB{i#z6B4S#O^@#{TYDNZ;9Ey-jX$Z+2RF6Phmg=|u*$~?=hnPhp6ewI zRZXPQUWU}N>|4xwh15@?4BHw((kt~)=1i2*oLAVa&j}v2lj32heBxFnoRZIo`&jIRh^S0t zJX0qwb5JOdL&odoWW2tUPz+4OZ1`GkyNvL4Wj7{+4G9CJd#Is3;Z^CM0WlOZJ;TDJ z+g%MQ>`G`bAO={+BZSfe7YCi6M2@`D;CaLl(}C~Vq`tlG0X}hFt(WjJ8khUDN&CWc zfkK@Xg$aGTNNx;*K^^1R&XDfyQiXhH#PC8Vo;0_c)o2Y?b!8083?qUt39KiHxTn)s z@|K}}uHri~F!ygW<|G>P8m(rQ&R($vuJH*sJPN5sBnX4jK2MmA za_7@*g@oM_MmF*c>qd*B09IYBi3~ZFtkoKklAuqX8_?;igut`!Ng_Xg8#A=+qcDpwJiRc7SW!Jrl2z1I*O0 zrc9r!RQP_UtFBJ)h|_2uj~E$6e1o-()QcFeofocc`D9Q*A?PFi&Mm?XT|O)kE!Y;9 zj}(#dxo`7^UEq@~nx}aatl${cZXZGr8iG(w;nTRbnvQ5gBx1>DC?@i!#IOold?h9{ zqAHC={Oe#;IpiJ~<=}@g;a+e@b|ZSEMGn>~MZFh#H9e)IS(UvA#u9;L<7geKu>w{R zh0Ry}rG~R!v^$zj_^9!x5K?y>d3Qt6S(-)iP7daAhXFegrZc{%vVJVAn`o!1usE?y^hy67$CftVg5U)Xf3>9X0g%>u2 z)SFow49wQek5%v?EOkmAkt4Tw%p+7aMV?9mKVqGtCb3N*a-foCvozLVxemogVwnTw z0G>=;SsMqOOjD~eGP*wz1#e#~?ozS2*M$4X9;jD*l+-HFG1huQYnrKM+S9Zqxi<-> zccF8cOz7#bQJzpl;a$D?ja!c6En<(?*gi3IS2OO(t6PlV1tdnUE1XXmI1~so#|^Yx zBc%Hk&c=~daATX*BiBDC<~TP))`;P!zBxq7b zg}5$8(gmB<#e_z6H~1qMG-YYmq|)RWD`VSIMjCnh%-_&ag8FV{BFo;mDAGlIm$kl0 z?G)<=Ybc^eVwnAygd6HxJf%9arYddFo{miEMnEt}xQsah&yv2R#H$x^&RoM6jg^1Y#Xb%yK<+X9Clu{u9Q!Y|PMNc8fUPmOvPzWnX#F1r_e{DsH zJCsSPi*ZLnsSr2lGXn+@Axuo@YDBfG{vOiUi0j!D9#Kv7>cnNW9P-pq;C+KI$c1|I zX(|aeZHz$)@DT=qoD=vMmqoVSfC$KqL{=KicsrlIQk)QHPkO6QM%MzxIUoWcF-ct2 zG!<2q)E$URkZCY`Q>_wb4df9)oWILJ1eSKv{DIeGRPK;39vqpu|I(y1SJ&9NJ68{o z@tH%0)8To@kPywrkfFfom%r5Q3aLpa6!Gg`LPM0D8>d!fbaF?KnZN$Z>h=XWfd!Wbp{Nc9h2 z&0cCKv2nN~jFyn@K#--@Gra4%N^G2!<;kmqq8l0*j=-Tnm~TOTHd~t4ma_|a>Dq#2 zj&WEZ!cXcZHW%0NGbWQg3L0E@*UhzVAgrR?C!Sv!72*;b@o|O{8_`JDl(6R2Y)@L^ z4lW?`GU+K-(Xqw0m_#hEp~Pl-^y8h&SFoB3kx~lF_6{XBaoXk)b9axLyNZ;i5hBB0 z79dqZH2&A=c=Z7WTzr<$XIB(BAePIv)vS54?#-&#gI9pqVoG`t9vW5=53)vA;SUV_ z@FZ@@Gs)Jo&0$~N61|voc| z{ofEVM5b$5KLE>i4>73G*a4eld==0P*n+DMZ`d(;b!UCl)StIE)y&v+eQ7-Nwhh%~bV{}}k2o&F8hKxJYTN~TXtAAabLVQAP z`<_h3{rKqHzg~_}^_Ue!Y)skb!KflDlc3YCrkCuh->WgI7t>E(4jIbJh-z)si7U~A z;N<0DOXbap_dY3$?#OMIVN^Yqqq%tl)lW}9WxU(eG8|xSungmkwN9BbbA_KiYs=6X zQ+Z!eWb%qFLugd0AIg>rJ!aD=icJN^U06Prd?`kkpw-P$Nn&BNTJ>s(U3Z#Z$hH=Ht`Jk6#^}PiIH}S}!M)uSTe2VP+)0G}ner;mMz8$Q;#wWbeA^NA|3n zeq=Oh`jMrv=|>K4H2ugf6dZJLZOY%W~p>POQ!o-I^E9_ z>VB3|_p_wBpQY6U#`>oDKFHGQL6)H&WNGyvODoa(jibua>Oq!P53;n9Xj4Uj zHA^cAZ8bjY(W=L5J-q>kN77Ni z-s8NMogOFSBQ+4FL#8DJ&3FNWv2a_#fS55e&4`iGG#^>XPLIoc{D0FE6c)-ftu^q$ z{QRj2hh`ekt>{4m4b*-lx7>m^+BsonauQWslrxT0(r2TLPzzZUyqkm1o7I~2!0s13wjP8|vR9|@ zwuq=+(#Cq0aa3Dv3LExLvaOztdaHgQs*njDToRz^oCMrxM;p1>ClU>epY8_ly;S$4 zW{aB%c%B=l_TJ*;MrOV0sTK4dR7-_KG^?EklIV<; zrTB^Mon@WzhQps^nJg|}E#q=^GCjsCB;CA>VE(-9CXYi>n;T_Zyc^kTFeH>nl-^TS zRZYPg4Mp#buL%;nCnR>mC$R~n-T?~*L1}&IhU~q=o!`>RhEKSU5CHoHf4a`oXP89e z4Jc7PN{keK(o9J6xVa^JACN7i1a%-Z|HcS;m@N}Q&R0kcR5}#b_6*8s} zQJV&}5j8w;&7IrNHOekp#++vtOB)jVm;zO0Qb#Gx`?4ZCKLb`6ABx`;?waij zLxys$?JN}XFrtnb#d~lv&#PPamI<$rJc-DCN0dDsWzl{yoqd4@uL|v4meF%%80ij@Uz(1@?nEG|UySbP7ER1u5$_GJqIf z(>8NQ(s4^>>8R~tojzm-#7)ia!lc^lA)B{Qp#s_tv+@k)>UmKbo#4Jwe4n278+7{} zkp$xi6N!5%6QeIUGY6T!Rq>)~h4jTs%jH~FugK7ZE|T=D!y0XQ5G`U2Nz`0l0-YEH zhmpx|&ZZzA@@-vHkooDm@BkHPcYp*3Z4!!WEbksdZECax8p=F{1S_OND50ienj3HV zu{XCW_cK~aBT%)sb3jqVr&Dk%RilAZFAHBi6zi9Q%2{!HQWLpnPbXDRI3n+s5C9U? zb%KV`QZ#{L3f8|Q&7!vD7QWjL$VvJQ_fk|Pj@G$`>_-}wn6CJ@3P>l$=W73?Utg)% zFq?^HcS!$+zivXdU^HiM6tZr{kxBO}h1PD8-syt()Z8|J?q^CzE%UI$Nw`nUpRFb- z--T&XILK(ESgPN)n^CyKtj4PMZ+UEyj}Z+&9UuFTl#^6Vk!ZC}gxhj6U|z9t3BS1Z zw&1x*i++qSA6j$ck{J_aW9X|J<-&ZJgqh}wKqmeH?ny>G3Rd*xO>TJ`^5vlFHjddb zjDdrx09toj6T9l#*g^{ihM(HoQ;3#@uPnIRNIXyRD2^!88%;crF;UQXVL{4C4uMo{ zp_y^XidggaImvV@DJK(&ro$-~H#u>5H4g;BCW-@u0N^=Ow1JAYjE((Rtc4%3!XU~A zTG)+@$?iCvZ!T4~b(JrtH8gi-v+xsH3R`XnCcKemPjtpZr_O|~k-;DB+;pWC>Hc(S zltaPMBGWlb;vo}D4>rwro+hj47Hy!&%xF6Uyw4~?lY~d=)`CB^YI45I7)n?GfZGzv zFx)>TlC3IfZ3{Bl1|qp1Wx^A?RxP&#Z6~Y_@+cX(ZXlg1}`JGCL`-C zTuJe-6w&40U%22(7m#U{umtLkAh;MNdS>{2qOTs0Yiv9UtC z8cVqrwl>y@_!jZ3i*2DzXANsTQbmbblS$uW)(s9&JlD($dwxgdK*i5zXyrYxgNZKL zkntmDLnb2T-7Lr`W-bz2*XDtuYz}{U(^500Vp=H>57n!D=U7FIXu0L47k7g7Oj0}> z$$ZSz)u@`uODGd0yaqOD$j;BECmyMBKAx>6NGAB(`Ha6j#!7sZ=$W}#{FQ$8imYm5 zdV;h@B|`(Fam7L zI6&er-KWq~;or)fwMdb9F)66+c5N-7oMs76DSDVw$y4#1J?J)EkbbuTA`e|s#ar7# zQ8(t7^-3GgFai<@70>Y8O59YJQ5V9-<*lZz02yOKyab*y7iq-Cf|Eqt z2k1oo@`GF@qe$H2M%V#SFPa_pU6+ z)-hGGYCE;1Dp z7Kk$nQc;VLHl}XUB*gF&=AQZ2)tm|~Om99z)Pk7F>{z-QReMD(Zm2UE@><4dsr(8f z!@yh?lu$+qFNRDVLt@>1O!HCm;jBh4O!hR2H{{%tLb#}ObdGb95Cvr3W__m{iG$0E zJ*&UmT{7J22csdDkn-85XZV-s=@v0ps>#1aO^auU=EAMhTu+Nz&0F3AqcMs1 z93XsRa9&6u;bL05>duPe5PNx|K6?~V$RQ0;)nVABn7VpPQJF;JX1K875WK=q9jr!@ zTQ8;LMKOwEQYjxXz3ns{eLxn+KL3f1UoD#lw`E3;V_N2mZa!jWx3X(i22ND-wWCte z?K{P1SB2x5khzjdDgO^#x4!Jf02VX3bc=Pk@k^@F^gQss zrABjmQ6btnS9koHD;q;oCtOESyv>D;B{UWG>{{0-A{iJMmm()(eI%+HH=Wv+BE#jw z7yw|t7Ik0{oyGAqmWJ%#0k}-kZvX()!op%0dN!v8hi|M66s=sbWzk3F*vfgPsOMsO zr3z|-_KjL6=x@xz23<7g62ae?Th;E?soQW8b0#y~2vwlWY_1KSNi@~W@_Md9<=8}z zXDH=89=Z+o$&19*qcz!z#jsa{9CK0~Bk~fJhW_1C;{JN0cOf`iqoFWSUtriP#LtUI zfz*7x7~i(tdNG$y@@k4~O+%K-D5xsJFVHUk){VY_pN@XK(=n>wi=UhWndc{Q3izIF z6}mE)5GN=2DTAwyGX$TaN;JT?nFwJiUg#?bhkoMKhs{&=;MvrP2x9ldP+j*_#fOQ! zb2O%0o}ekC17*%toKcNopYWqv4|FMw;nn9GMeV}q7f;Yo(vN=mNg2Kgd>tiavK4<$ zdXhmVYjHb4CT>k6HIe!W`)V;KH1tUmY$W$_jguUy4 zjE7wotT9fygwn#EvnU(48Q^aG+*=pBd~2mg?Lj8DFuZ7~%EA&_3EOoI9*I~N2an1J z62({=gGc2prl+>0;8FQ7qKOTH9an#skTL;Jw@$EY!6S3!J)6+*Cl$815<02HVdIG6 z6$X)gkcwU*J1@EO}oJf6f-ooMn#e6w`8u@Dk3Pv zWz_qGoWPK!(#Bon1R518sc_1C2uwBdj`5UHue%p$UZfY4-0D}{?(mvCS(t04L+Z79 z%gK0ntKN@yxE8MI9c9B?_55Pj;jMZ;HLm8ZdUcad-m0h37%j$vHb|6ZNff5j3(EWu zm%RF~)8nK-5(AOLe={vP+f(m>wVP^&BaxYA#Io?)@aNO|3cC8^G9Ulnh6!?^V`gNn zugBx)d8UCbug7ESvtwI4uYg*dKIqs=&;Z+lnFdP8?J<-DP0b6Fw{%vPBEJ$$ZmCa& zd#aBlY%6&4U@#FjW6@O<>*aowH=2W^QOH|lGR$+xcWy317rEsI9r1#K1k9M~5yFd6 zQE-Xq!8Hp_S29>Bo|gbNm)DYY4ECCc?s0vIXTf(wbdzUmCr4vAs5*xSzRBj2>hZ#{ z72Z~xf{l`>>5Ao34id9{^@H4AeU2KgeyIK`9Ab0)LHbNAC&N&vr$QN)clf$7%9_e8 z(TyDYDIKYD0LGQjA+ApoX1>iAcX#NC-4XoABf*gEN0onWrG^CwqA<=AQIf^KZgg5& z#>=Y(D2`5o54EgJqBAMKZwUS{xlN$tWy@vtBqqTUwg{n2&b`w3ZrHdRzXV6N83S(U zhzaGoH5B*qYi03LtSyCa=;D=Xd5L~2)xu3c2+m~-4~Bi)DCMoixHA_oR3^6e1iLIv zZY_xtv`?~zuFu5m%jWxjjb6R&5zIyR^xKwUW1P9#Vn1Wc<^ihZr%9};HHbcW0l=INNh+)q0ySunn!=JXSpbtHO1Z7N81+1`jllw+ zol)m*XEKC@`s|Es%B}gt1t5hqcGMDP)E{Fct`S$_9wBk%yAX1!4ZUh|H@tHA8D5C& zp5&ppPYhka$k=8NAuaKVwm#}ZP)F1Q&8|iP?@RpV=uv4VeCsq)1ASZ;N2Ln4LDHg> z4qJWAFV5Zh6M@fk5vt!fiYQvx1AIRn)1e*8V+rbs?jmtGD^86x{-e>AFDh~4v23&dcAo@ zOnNbPlzlgZ{Z7&#Dd5|dV4BC-j-u(KrJ~u$QYf9OM=8$`erm-JZpr2$O36kglIYsw zJFQICo=|7jzHXUvUAE!YXO5(b$z1p7n{4}9MKYQSp(HXRLOx`UQ2uiS2&LiXu~0Ol zaJc@Aq8poI)Nbk1 zH#39xw30V{$%dD!L!JVs+15Lyt{Y6pnuF>ux#>iFXZDUV7cSPB2gWfv;Ij)Ty4#sF zAqx#b<{`)&Q-F95N?k9Gd+I95vys^%Feaq$je%9f>|-Ui3D}a7nUF!#MqUWyVk1OO zOv1=ZJ(bC5d1JMxh!I07^P-W4%`-MvhTTm`DPWE2n6lAo)2iF@>T070D6dW_h3b91 z*_49ZmOXPqI`5oPsKON9Q2{reNsS?O<4HPl5FKBT2+@dUK!_r#)TF4~nMjtJZt5%< zlq3#m3{ie(uhTjnv%(SoluKRPikH_M$5|Q+rIenC z(mXBtmA&)72YQ-{5oU1h@tvDttg@GNz)0aGMF1ood8CYeIOM(X)4++#(G-VLUNa|xorn{a1v_z~&ymJ~st?g5Sl<5^k^qTf zq);P(eTMThXftCTsdNn!-~DjZS`QDw>9Udo47rwE5h9c0jmlTfdrXyQ}iUK2&V zR)(q4F=3ywk_+uqiHN%9$s)bQ1ee)>onphAty-IlNF+2#&+!n?%lh=f7sH&_ORcR{#_ZYi8b}M3J}(+ za^$91lTHSsc(kE6Bn+?lR^JF@+!Xecu zc=T}*p~8V|#%&+wcDLPTS%OCvu|&v*m&X!eKM7*Hh3LZ)EV=Ep#vSZ-LkGIvC0 z(mKvErieYnM-niyuiJ;3WQ))s_e_HVpAdJuO)#qYp?^RLVi}6fRhCr!El{lLjYPaa zk;&W>ZMfZFzikK|!jH|x3BjY`(ltLi{?~MlJ#gcH$@#ghbL%NK^|_S~u*1o|@`JsZ8V|sY5XucY;_xFzt{ z3fvkWyo58NBwS7d@PRutky*K)LN(XsV2+p4*fS<-w5BhTxy&Js0?w3uM{=O5i>Wi? zxlgPg5b{>32*1yL3hMLRk*yV@lT_%ZO0XRFH%nB!+Vt5ZN>^Lfyqh^Lk*T&giWtq8 zg6pmhi_G!z96`7QXms$rs_nJ!**lLk@suAd49p$rUE(y_Gc#kafA13i<^vT!EFoI2&A8xM6x zG%=cV)Q-!FoX}zK{Q2x#79C5h-uP7y7ubDbPzgV&_CbZKAz8HfhMzGm8urf5*Iv19 zM^Z9)THS&pDQm~uw0B;#X_#o@sj+w4S(4w`qj_=?+8P|5P&PlyOIzhUqE>0~wFAMTURB>jz+=nX?9|#Xo}X)96?wo6^5g+0>I^%b)K*Yv7-Fq{a#$)`JM>$c z13|7`2k0e&kmTFdSi`RRz0yrDj<}-X^;;vVwN)eTP-a*$pO3agupvC|*rmajxVkr* z0U$ztr%xN{)&%Ivns*D2b3C3Ho>D$16AtTXXdy&6(NKNkMH@rHwQOo~FB z;ayY)8Uq56buyUyL#d5r_UifW5R5iLp+V;ir8clLI_VrazQ0xH=qdl((m4sG z;!?w&M{#>}OZF{*1wKyYhz}|!w8r$4b4i+O76|_IZs!3HZVqVT>Oxez3@QhDLek)Z zFhgvqk{pvDx{_NHUrnOaRlv`kRSuKBUFFCMZLD$?6=Bw#qzWQo}fF!R8EswDpbK2z3GZbXi ze`LsP`jMHr=|^UvrXRVWr|C!5noU2lxHtXCn!M>pPT(~CD7mD@U$Zol*AdCHb&}nI0b&0i|LjL;XS`gFFMfKu{D}2{1cw1a0D~c z(xY$x%8PW*9}BnTH^T(av?6g*zX|_oUdv8T%WV9A)6-)X%Cu~+d}oD zv5QC}RzHX`3_}|{5ch;=HfOtLUP0U=Pb6~+`*NTDBYsA;`@(aWVTe08=H(z#uv=u4 zw`(<5*RZTl^+V01XA0zqA}u(gXkKA^`Ll>F+?^Z#f0Pq?5{4*sIjkb<<*0J0tipX& zBoQv{Ns&&hl6WCgaN371&ec#M#V2WZq+MW?FYI!YjZ-t+?(rY1pZ>Dd|ITLPi~HaF zc{^FaL9RGMP^<8arwGz45`D{lnA0VeLnMOqS#W{P@-VZ@&HNZ}1Yn z{N?qt{eQmM9ld$6^Uu+{f4Tk+_xFY(};pY{>=9HGs_$A{-`rw$ zCzIL5=x}j1#dv9IN7L1MIXyUE<1_A@9etc0O|*e~G)JP+?%Q{-U+wHVO8v4NpN`jP z`weKmn2#qFU2ykrZ(skN=OvH*^m6Cze~ort?Z1BYPfS?9bglk`McVnt-rwJip6%?u z+j*P1<>fb{o!ysvFX*lpyKnY>`TINnw%^{q`DORrv$sECCg-PrOqVN6zalD*P=0~g1dZEnS zM5Y7{UhZOfUZsb#^V{3KH*a?QK5dzQdHrJN=iL`Cl0UwA`}*16UqtkGq}hM|&$qj2 z+WfZj;zjzGm+=@=lYcgztj5bulYh1Q?Cs9}ZgMsLvGX=wi&U>~7N=*Ei>6+yY47|U z9P#yNHl08y=i~YM?t6gx63~Hlb3-=*S1kQ(=M}SG)1!iC=)eBxI~J|40SExTg5%0YXmtbQey%wH= z)_b>@9ZjJ#z8<}ttX7lxd;XjB5QvBw?YLzK2w;2%4fMs+ch6o4+P{0Y`)YSI+MAD7 z6KF&T$k}*F3wX?G_$~DBd^9;dn=VPAxyOhcQ2X!x{`3A%qLrYWei^x?07>j;+`98@ z|0lH%@aJf+(sURI`nMltFniyOMmw{~pP12dI)>I7A!2z-!sMEVFrAIbUci{tn0Web zf9JQ|U-sVqv^%GDb@0yXs(9}1AridV-F@~P>KK3Q{MYW=pX?s}_{Gx~dl=fQ@x|98 zsCJM8y839bhHXRCS)4CNiYa$iPw(-)`pD}$lO?tWeuR>MC389%ug;f~@9|-=nvAHG z_lw05EQra)Wc79U=>ExgJzC&-e)M=cn*cVjvZt_4vCOtCAIHn^x##0QCZqT0^Ue`K z?*r_o*_f=6mt*3S`QjrCm-&(WwLV|YYu(bE2(-AjqsbqOHOxTmZn_-(F`ax2d;I$u zejkt4FK0#1rL`u${jcw#o*>hS#dm}UrazF$r-^W*9J z6EGi$Z~~eySL?7(GA<9!4?plTbMEc{&#l&9kCuS<_e882ARhcWSw=nYfQlgf!Q^mp z3K6H4CNo|nqAz_u9?zrh7BKP_GtBb+^lWv3DIwiSsG^JQxarp!xM8|F`DP@%Clc{> z#jjt37|YR%G30jj^%v9`Y~<(R$4W-3^`q5;gO^Y?TwM9tV)nd-%20aNxPe&f-T z{dqe1-i?Kav_Hnm6R#43Vb#|Y(kDlwi^ch9bwZ?BO(sPBlj&(S4S5t}g5`HM1M5s@ zGk8LZ0Bq0Vr}2DsyjY&zISRYngMkkM!Tvo5(bu5SF-Z13h2D2P1k61G+ThrrwDpYz|3nbAjE5Q0`@)?ZKf-e>m{(Hqkh0(%L(RiFnRxe zyu2_*pj#LL2odF9-xmSptu;U zfJHDrq#9B@kC5>>T0dKSgx)!aNI>Q>XJR5q{@0^Bzbw$|6hwwaNq_v{*0zETpaOIF z2OtoyAqdNd#G(DN&wvUt1(fE{|_{leSTt_)d=T`6e z;v@403Bs2}tu?P3hAV6FCDNYI>I99ai`AMnX0&|&gqV=h6?(7X`7`p)7G!RHO>|w7 z+kZM;QM|+rjOn?zVsI8FmbLo<V#`LbUG?cHZBV z3-HYzf(+*XR~XEH18%HO{ucKCjF6t!ldKOj5n)KtqvKOCj?oTV%aNMgCww@7o;#V4 zT9qOGz@%dKIGB-YVcjsDBV^$Z9rKpiqU{qHj-b@{!W1iFK&Y&f$>akpw~H02qtobd z%p6we$r9@4E}};7&%yaBMXN(7lhrq)eF$(hqcVQTao3BFsP$>Mmj})0Bg?O|e0KkWjY~AgR z4xm{gzIr{!y5kmb&e{1Kwi2oh@DPEWS{MsJ$0UNa-+|X_S`5VW&L)#33_l7>DMfRb z(u@MTe;&eaK<~6-E0`&Zql%GK1TP@lM-WtsC&CO6E)=#P{AGIpavI6qo@n#QDNl>+ zAdxpb@`Fl}8@qG_Rwq3Va||Us$7M6i@$=8n=BxE|xr7^l7^v@p`&{^nKowZTDJl*2~E{7QEkKE|bNwBAJD)%AOTQ^YNR}?_{2> z&K65}YKuAf5!LGSW0dEr;Q^Tzk11rn+!Fmdg;D;Ipkqe1K4Wl=|KRSDH+})y(oN#D zKz~D=p^T5v0{)tf&)=U!%d^i?bVkBX&ZabShG-IbbFdBIbyBDrbPhQY&z+LL+@&L-mz%+?_6d2*S3&4wujEIoxEi5Z`$xkqli%RyZ8P{0N=O6sdF*9PMMDUEFX8>6khj5-i zRf!2w1P$t?Ay#IA%&$ijwuIHk5oFPKV=p-C4GGl?x+>i@|(JQhx?vT#4L3&gl_`iocCs zlPQ9g-GfsWN4m&k8o`1(034BYAcDmX0NLMw8*&ohRxA+~)?G;bm%4AZ}j_EvY8y|K~+EKFAN zbm_MF`&C(YnSt-k;k_Y-NpYI#96^jJxZwcNF+!l#sgUtn=4x<^J8n>=uy z2$tXsmUYxf_UA-uglLHsD5RbY-_>D8vQ&{ozgj4~@vzBjTNBEG z4{9GwJ)^SHf%U;QNaH^yf3o35Aan+9I7EK_5Edxt%?Zgh_;ADFK1VoY8-QDJ0&D>d z?~?oiBPpy*p4g_8-GkS4G(yOeyf}#Vnj#V8SP|Gm%vr|KXBkBrNL~d%gG54Z@ftao z0R!p*>H=2PXi|CS0vTVT=m(P^YOyztnwt>ISdqhD7*XVs1z^lVJYOsz&g4QMio@y$ zjyVJ|a!6;=I>%PWx*VQ6^36;z71&W)34Ut?|tC(3H>As|^RhRviY4Peo4z5%9@{7qydp~m2`%yv##|p!5^HMGEBl0+kZVfFVkMcm5E@Y!&~QF) z!C6II3I7ff3~!cmTVx*}ED$wiZU(#|I4g!HiTfXu&26e<4HdwIVx#6GB6R43uw;w? z-RVJv7fg^HkQrefIC2(bpeMk^U{`D%TFk5ad!fSXb0~Oi>!i|uq6eh*bf7v_}vgvN>m3pPIf~mWe1bH zcaN6mAU3FZP^#pT)(K>aO@E};;1{ySN1_24mlFhdri9tkW3mT0a7G!a4OeXU4xx(p zH&mEGg&m}`UrmuDvF@Q*kH-jR7Iv1L*l%ifA)tWZ?d0ef{6T7wU>Q+%3dO9{zuj<3 z%!Hs4kZ&ESu*Bnl-gGj73fL=&ck#$R%*JEvF@Ucot>0IUZjR4E(iT!2`ax@g3(-@%wH zWyG0CISY7#nKIpmtgy#HMm#8aOy+RpVGyvM7DnxpBzOh>h5rLhtI-h4`e_HeG`mgqV_n^zycq?i*KpA<+CLyf&rbTz>()revoybgRpd==IaZWxD8qqd%+uNWfxS|2AACjUXA5lQN~$8!ct zlJ9oF1>cN*p@F_%93iWKrQu=o+R2z{AVjz`&Q(;-BwJJ94=fFQ7w|8PJbrjhm2KqG zkjlg&)7Mx%c9IZhbED-3kd$9d?tTO~Awr3yfHic4P#2|qIPxsISIV-&$C1aIA@E7% zCFgTpk;xI~n5AanYfu@p6S{qvOwPjRk5E*H`Y&NG10CkDL;P@rNz9UQ?z)}d2s%t- z*K2Nx8f>H&;C^;9q4+=^q}~ZDuq4RQFky*2oCB-#l9Gh@ZUScs)I#A1rOWV{GPjgV zfC~jLja7vkCa(=Wa|aJlmb_Z`+zbk=F^))(;b?i7F6om-A`^g~AasHuOJF>uJ8N-G zm-cE)4_KXcEf)Q5{E^KZ3RQq#%JfqKl_-)${%vX+3EpcQ-|uXsvvwttmyIt&Wn>M; zD3aQASP1%k#K=&FbO!N9A|4DpMF|OFUocz{MWA(Dodb~~xY(f#7W=)d&cOdLU=d{i z7Lz)nGS-~t{br9Ia)%W<{=_Rf=cOcMlXe3U9_OsJ8YaF>RmXWNOl}B{R8S=u#A}E_s}d7 zMM@{9y~0e+P5Ywg{l<9+KW)u{sqyHg+t?y;IgOT`Qr5O7IRpJVJN zqv}~;E5gZ=dOMy@N(0H>kZWD-yhnr@jRQOmmXhs!aejz6TiKE~&tc~x3d)c>02j>S zMNo(oAK|)sva3kiD1{EI6GcmL)dB>7whqeXF);?f$N#fH{)!{Oq=%q7Q8sf&C#02w zF;|nBtU)vsAD$CIxohkRfNixzuySubo?S!c?tG0U`h6o{d53oBRk7!zsVu2aHcrC5nuLmiZwfQiq*j|K>ZT7 zZXuJx#G)E51e^{RpzhH(qu29@CH6U|iF+|)D(OS1jr?&$K_!+hO2gmThh0Zic+um( zqY9Y(Vsa!Au(;SsATYBB~kPa>mByjIxhBs<49H+A)qLzV{H1snwHk3H`D#a z_yCN0zPfuiWfI3gyDg9C6|9pR_{^s%mG2 z+Gu9h-Se{(uF#I^b)KUwtOXL94D-lsV*eguON5I2;JXX#4vL1i7`?bSLdz_Ui_snm z!NxzJG;BIst-ZPQX&G--j1i_7a|^j-v#@D zg0!$Jtp!_;sbUHkeLOy&t)m>s@5DS@R?4M2*!a!$gXo;9n@KS6FRke*B07gr1phg~ z7uUX%0d!6R06}G29Tlm-O6DK(N=~T4^eC<_ft`Rj7<_zAn^PDIdm~8XxJ*eZ+307u zzK}=qTH_SX9(*>G2jL!wBhdVt(XWIBpfw->(mfg@w}1jQ$E;cYYvhp~wLiG(*Hu5J zTJ=rzKd12M^IF{BS{_Axe*zG@-#^ZBt5&_o*^ka;!)^RmCa5hf#t`5ubo*HR~efohxhz#)I!6 zj9-rSVc2j%?H*RzN@^Y0u}{X;*@Tb~N!p&dCf+5|!X7}hc8MS&A#$+2yfP!j0sEXgpz$0n-`2xH2{-9bxq$?me2vh@_VeT$KRVk=Kzf*>NSGf5hj0!OxzgaFgE9VK^AM&B!gKtk zpB`oMsMIqJm_aDVJI2WY=#48Bp~a{`8GCdiKHUfXNb#u1a=?^BR(13o0bktxWj+kxvFh8&U91FVn+yavR>BkUp6lcTA)XNH zU;!AO-C9$LSs7mCa9YVOwks@ma?rqqFbH^OxkeLmcvUohS6c>aS`Xu%! za1TX$Iqweu!6qH7BQmEjOL(Ub=|lFPPa_sYToXuxU@taRV<|}3jwY0+La8LjZ`UUv zBW@wrYzuH~nu;p?N)w+2q#hw{a7f(XK zku60P_hfP!S0iZ`*o6xlEfACj@)Mt7mjm#bSci%pFbvpK)5s9V9ssJ7 zCDVfaO+>Y6zdqLD6mW>(2b=@c^Uz$zhimR996eg40ssO-z`O1g!+tA(FII)xy}-kp z+XC;<1tj6{Tz1T}+YQ^18JuI9sV+Rk+9kE;qJ0}8)DBLfz}|eWdYvzR;04k>H_%uh zGorn$Qj|G}4??mLQ=pQWHU^NylKMG=*b=R=(EyO32|#kfsQK_|)FZ%UA&&isuoJZ8 z3b`)%Voiny>q{KMg$9|Sb`y92}+run{e=LZOyT$1Wlh}jB+^HIZHf@o#v7e5N zDfzVrG*)c#1Y?kvf;QmnQ%FL-6IlS1z8}GO;qxAN+6fem=_-#D@K)kgYA1>zBtJ&wh?7t&=iR+TO#DK6MW&&7v%1EmlS;j?jR8r z@+s5^g<(j0;Fcw9X>y04U_WvQ25NP=KtTfo6$Bzu%P3mq@B>hcF?Di^{n|PNq+$`2 z_-3@nHW2Qpn+0>k!>H(5%m>oMvdY{h@Bng4@(U|M0erv31DezFf7=g1K!OL zEra39sd5O&dxZCac)&|+^qR+}0h^idf3SgxYp8w@o0cL{0FaSE87&BkVbMSniIHbg+lII>X*@&-#|QB1wrGIw=(@Sl2~HEb+USTIO~)QY z_9QJE?K#F%Kfvfan;cG|g=xbYI2V%Q481!P9v}n{R}F;$bUSQuOcC#RsJP45mu9+tfKu@BJ%sdX6Ld^gvFk%uwL{QI{ltqRNLAJ@v zV2?AL3av34!!&Pd9;ewu<#4c8oD%U?Bf+5sz#9hGE`sSx*`@{CIS7;@xkCCbwM`hE zu@y?4nXIx5@r2K@;SS#adVvECYR72dz!M3uQSpm~!|4o!$^xgL3Zcg$oZ@TL3sJc> z&5mLfC;=wCvcw(On8M%TGjFhSmZJ;^gMgS6bN?POl|R?`%m)_MivFLIbxbfWQgOuL zVmp`+nPw1-Y;Ihmm7^7q)7XeAEK!rOoFRdYzz8>D&$yHrMQeae6Htq5@gekL2*J7f zfRMhf96~o#{2$qt`{b_@PVmn#3~8ujZjGe%s?^9Y~c!Sq;o zXxj$Rg~B9fa4HdA8oi?(ZN$VbWD-8R4+){s$z1vDT^cFQa{wt3hJ(3}dR(8tRcap6r2FhfYTU?k>#Mz zgnB@0gjS>Qq(3B(3Oe+AatcWU^YLSpCBaFjkzjy%p@y)NABTz3IW*!(qOzB&n}{{Z z6Ac-2pT@w?NWl}Pe)!>g;Z*@t94CSk)>;{=G%nD%3S8-gAS~=`;48?b^QE<6{G~ZV zu`O#g8vg`U5TKz2I@w7?x!~8YlJT_-k-|k( zWGrLalsR#XC`>^hC84RSK}Ki}0kY)+MkCP$bEn@ooy9&ZHWXHn=?Tx72}GL$7c)s6 zP&^P6jm1migwAxDP$fVI>4&){-aQLC~-tfP5Bq% z6N;(>OL*fP8IoK87*3EB8C)Q`8^8hQo(qf!q-6I2T0#1siWXWh31eo96L!q6!bTJPuqLyUL5IToMAy4 ze?qAXu{Hr2WHig68K}}lj1Q$o^{4mg9+e9*2dFd3(5}9x5s32%Le8UL>Mr6J@S;&) z4WR;cX?TbLz(EHmB~BLqd4_6XyAgq}kE=UizB#r;@&0FeoKjB&mz*|?mX&5nJ&lp3 zLDUGr2iXG0n}j5{ga^4~E_(8{Y?~8BDfuT_J#n@ziow2f^xzI1$Oe%zmTX2iJCRk=OjPwwdm$&Bd4uSR>?@we0#0y9;cAeU%6WMUqrJX> zRR-R@Vs#UV3W}VO@Eq;I)Y11ML>dL0r$I8HGo3I?%tS#YvPsj?r`KQ-3KAi12u^^h zI-_$|z!@+^kbj`mBqk6}@a=?xE6jPwq6>~dmQbRJ#<)^Fy#F2Gkz*=Qk_;0q(Ojul&z<5E3z(Tmk*X%9_uo-VDNNs6 zRaML!gDx{CngK%fIK4Z9O=;RPOGo&VNkD{Wmj`l3@0Oy|x5E|KqjEX0GC#_vh`_BR zf_~37glKfC2`MwCbC7|;3Yg58Gq@PUVf7k`3%!W|OvGEFKxekjVBa7bh?L%Y%6w1h zg%?=eal}dc;69v#cYsp|Bu9kGoDY*D)L4s!H`BQCY8`c{At*|1!4zQ&l8kT6xX{pR zUYXZmB_OkyMH)7u$f&x;3X2QkbWHOPPO0^WN^@qcXiCdRr{T*+Ij0Oz)@;5 z{x*W+MqVbJ?IM@(p*^pDo;eR;U_8M#B2KoOka&{$M+urOnEU_l93Ibv+*6h)1og?9 zfn*`Z2787BPvl>PgQRs-f1I2bifvf&0v_VC;16Mr#|X3|eu|8qIiBEXd?dT+Ano)K zR(=En9NOccGhR?F=Wg7gV^2x3?LmA+EMOtkgJZ;&9Mp$D3%v!;;0M^dysH{nA+lqj zWa4+JV3@qLW7N?dBBoyGtv9pSGtrRrz8XZ3HP!&EhMI|tI5xb5(P&X zvfv82-(-CQUm^eo!^2gBkxy@ML1Yoqg69d05ISJCZr;iMP@ z1#aM^!q&nLu@!c@MC|?=XEC`$&`{O0LZJhdLCQ`Pr3OAr9iD#9lWu(k$ zgMTj0>4;`n46?rx-J;e(!J@4BX0brytiBLl*8Pld_ANyV=E47`IE!gkxP3b}4~~{0 zLIjP*hUDfhJzE%b=<6D}Ha->u*o9({qxj_-B+_vCc{*ZZkv0LS;J|fyqmSc82Mpo5 zlAprQKgZiE&JkMT3^e}D7P900ArK~cQx$a%%jtLv>Vi+Wm+%%WsqJj8e+I*KJt3fx zSHF;0kg;E}7t1Z`#pEm`k~*zDCi_oR78!f6TZq6o&O4-F^duX*1WZ)V&x->E@1Bj| zjdc$hjidy*G7%dBfZGz8226vt>ik8JfuI+-4KHYb`+}Me+&89Ij$SBGM|TrDG1`-2 z2L`=gpgv&>fpy_NNH6BFM0wQ(&U%eXhMz-2f;(*^0N=PeKlmTu2%9Jpx8edZ{1%e- zSD>YPQZ`If%a+qU&8WazWhv0bbz(D=$(+X}34|ALY}tEpy(9WCr*FxINVuU;>Mo$i z^7PFJw1HVbY()fzVFme(mj4}ZoWwfvh7o}9N(%T1{W-dW-uUoGFkVQR-3#*Sv6z5= zknMX}08w^8J=#l%uQ5_+%cxT*8+}s@3^CY%aLJ)>E|8l?X*)%y*?Fkw%;G@|M7%@O z=lHFi7l2;A^ftIL{Dnh(2R6o0J!A&L5<@NUg`5b4qBPp012R)|AbjR$jvY+sb#n*u zRvYw#KpF75vY&GdkV9*Vkpm9iFXTx7h&-=32m#UOB05MTN~tLQBKr6kJ2&HyBnvT_ zbd)-Vy2m<_j>j6lMt-lb&npP$-{iX+-*x!v4@NE5ax(t}XH(vUy_^@5v*{64lu#c`Nr)TxDzPC6elBp5K-|4Y zh!bikdnO-&iJjdn zdh{bqBgQSf0KttehY#)z)zOAz)%;>f(M))xl>2hcG9KD_p)>_Md}vDoF$xaZL3A8*g?6D=<%Q4wf8O3@IgTSq+urw8 zKr3#7ARY(O7YpWd$ks?*`UMvo!ev+TSxXj4mNm32U`rAI2r@ zi)7jiAG2Imy^m7fCcj{$Bfo9}omCAuI>s}oJn!-5iS1>|6aZ{Y(UjgD=#1~;GB5(xS4N3GAj{rG?$^YMeIW={x~<0SG>nx=%0M< zMThOo$$^iaa450#4d}I@RWTk7b#xz%z@k+x^-y_QH4^d|KWOY>LNVsN4$DJ_aAIcq zYtjFp?WZbvO-mW2&g^$Ws_fO~t}`>S3P5m8-qR(3V}Dr~|L9 z?yENC9_pg!aAy`2&?czjHtNaBLKok@pU*luB-s(6gsSjcSZOL6+Y|t~n~&aiS}ojp zqYaS7m>q@%UTY-)wTyIauvsh+v-sZe?4r9mK1w@I8tT`+y2pOiDe@X5k@=GyE2j1q z`jUb%(JIl8Y(%~$%bS!urN~%RVH6@|nFnnt9BpJs5z#w{fu}qGK%p&6UAIk_nV%E@ z$#EFLHYE?DkvB|O(Z7WKHk^#Fp1*(3FEM*eV4?SO@Jx)OBy{Bu04i8oV9xQXpBqC= zc9t%Z4y2a1<^UAjivfzbck&cYHQXx)Zpi z#<2sUNlGMGWKVR_pyLP0aqV^P?!>Y0P4iFWapKpNQ~Dmr%8C+F$netp5|Q6HiB^=5 zTdL*C0O_Vt&YZWZ2@Wr%X+;lVp{{}TIh+8dOu`IMZ7#HWtz?fXys!CkAh-~>hAi2Q z+EgW*xbdl}fRW&}624;wj%`^1N;Mi$5|&2S;mNsGUy=Z2mUM^uTa^bP!HESL`S<*} zwKT`R#JqSX4j*A`m$vbO-{Grq&ES;QLg*z0R@bwbG+MHX@cT<3i+-y70a@kF)RXbv zP(j5b2uf00H9D02&i%aQQ<1Z{#k7P`x%0PW2^J2jt?dv@NLq}`EH!p{z2u2{p9pv z>}S1VRGrgw9{c>3W}%ne2j3-n17R%-uUI{wW%=5=5{LHj5S@R8@FuxaR(gKD{_X71 z{2j_R@sgTXaQ6UoqoadS(@Ejq=bS`xw|TWo2YCtTb|0^fKeG-=%LvCEBYus@D1Q9D z+;7c%&2o9Q>SFir;gGIeYi4D5a-B{L2LypJBaGoXYHReK;3 zhsbB1_n^o~ns-qnEc)nno+oe8YA~FLX+nRSITrvoO*rJUsoC{kk`~FE)u3{2v@uCi zNG>~&SamcdI#NshydW|teKAAq(8aZE7HCMWmmc1B>&9M{R89%*W z!8vV+y^AY)fn0bIxE)ljT>j8R?f8-7WY{Dficc=U^T&~#iK+++a*hZu&*t|u0v&%; zV@syP^B|$y>B}56yJCEvvm+1NP&BJ?zN(DnY8C>%)byKt*q$tMdCi<=VgXuZn zZNaz~zujD#N0i~`!~IuZa7-63Sp%|%nCvE0f4yE~cS+c^+^p?uIY1kJm@QWl!5dgP{lN7AWVlYOXTE54}$ocP|YRVYMz3*NymqYPnrO1p06ZhlcVP!V8; zq`=<&&12W7)%pD%1+&)uz9e)&#nZHfF0awJZ0j9zk4uWovJ3HfQWBpmDvlq^@UPf) zEIpz=EwYP!gEwoLR^rhB1_Puu<{Pr;3MjxH>$oX#w73Qh(6K^rEE*WbaqEnh+}ssZ=#X1U+X^Umjf=BKuUr!={q{w{j|(@mV&Yye!k zdAZjAueNy48;e3Y?=)!ux#nPZPWp|1S~UszbH20123 z*gYvtKCw53YVX7DY zcDzkuyaEGBK{EPwgY-N3P+n`P>Vl8PPrVC}i-XRh{lB1NGN)xoe(sa2*LdV#I|CDb zVQ~d3$Q+P_)A8XMr%!O%7ZTa-z9>kdPwnX^(1ZU<0^}wkxN4auVzJ+*+Xq2jw~luu zKGdUYX5tZ~t=#uP=xeH7TrO|dehs+H%Voi7;J-4y;WDdl=2!giOg1;V2H@F!8515d<(^%{AzY_ z^(p%P2>^+{JGx~|t0%P9C^1Z){gogI77_}gcq8@-5_Ch`A-O`^R=5o&=P9C;lpv;d z1D!w=nf?lEK&Oe)TwssI?q$g;w`+5xkB|_RzOHlxt zgj|DG**PIMrTp<^cQhd~pfOHcLyy;#q+quTn;Bg)--n$XdlBSr%82Ufvu(OOT> zGsD2Pul-|$0;On<$*C-$Rp`M1_ndV`tlaMok_- zQxq{9rq7Dz1EY}dK0oeqw(yjv8f>jKK>93Wx0bp&a#~=KCD$iVy%v5h!A^ZGH=EbE zL^GjWqyNkzr`?C`yMLVw%%`V>5d^nuPF6Z~eMcD%NS(5Ad_RUiPKwadu4~R63+294 ziaN%AUeH;l2g6?Q*sP(_&v1XGA_MCNblx1=jpl@`#PQW`fVx<57s(!KBoYJS{JLc~ z(3xzd0`@X%mG@Ncf@g4rU1v@Jg$LsuA6}Q6Qg=*o_^%rp4jWe*j?y6<-8NsdDdfG8 zXj~*J#J}5nhQ-#@(<-NRD?Nr7M0L2_ECxYBZU+{C+#<9^NOUdE$8C)XjF`mVQ(k4N zFEPTUQHuT{bLQhjuWCOAz?v4{M>57~;p(QfGLjIzPHdS)A+%`e)wegYuphT3(4TUS zyN|hl^2fC6wViX3Cz!t?`n(U-&6wQrsl~#p$=Ddp2ylhA|J)c=JBld8io&L&Ya1na zWa$3Xg^` z6G^hI$a%jgkSeDrNWx`(Npf1JVp_(YYo}(CHEvl&F~2@{C`C$`{_9!2r39wvX1ZiC&E$J|M55gugCjCNMUT_YteI;blpp4oXr zzZS+XK)JY^o#aKqlw}iRqVg7;qHPXDNh=qcJoWL3y{6pWVlK)EGK03EQ;Q!hnRnG= z+J)u+*Z%Y4|NhHg{^mdaRscQ!x_5Z^xBqHzJ#R`kzV^5Ik3C!Py+on{t&uAA`sJG! zza#8$Yj!2G*YCdHd->wki|23mk6#=W&clE2^RE}ze<4F5P7-WMO7j;W|9ah;3aI|U z%^GfPa5!%x2;k zJNIeF6=D%hTgwijyh9_AWI&Vyp$Wkk(cA74KAQs%WX6mTwK-;OL`uaESv~R&*Cjob zx`qGjTxnn$EjXW6{r#whPvb&1b9;Al)+Z$+%IGTRZfjNi$cfbxF~%*_AHuhstk`uY z5RTbIyV23OYHNjun=@vMMQ55yCuxx1wdtHFRqa-y!sb4St_M9YXzjzzZHK*}1DoTI zFj0~ViSMir5 zeozhiSOFkR7nP&2%Z0xGshpxTPMCb5_m9?g`HLymM;DiWl7p4d_q3O7xhtIdskxDy zcg;MKJn(ao%A^q36~^HlyQBZD!HFJRGwjsX3$TF-45wspuQ#N79+@S)sOn)GC3Te9 zpJw*wqBYafUX8{uqXoV``9R(yqavIHTgDfuRu3QXMMw&4@R}^yJ$;NZ|8SeLzq|mi zK;efOXmjaUU6)CAFfLKI%fE{y>*Zm=^1-tQ-M1h2Ufj}01-e~xqwzb(e1`iVre3Q% zxxD#omM%uk11_AeGJ~12t6s!JFIBu(78YAqFj^Rf=)AjO_+%chG^g<=aUjIIkuWoO zpVPvxr3%Hw|0N+7HEJ({>;7~9?c2kD$Y2Wy_iv%JaAmaUKdp+MI@Egkx^e6+kfB0V zYn4&4Y3Ud*Z6T)#5nzgSa(YEDDqhNRL#h$b+0<101$q1@ApF#mqzIKuu2Wi|ZUJ>h zoX2F_2f4JZg*-g8ZX{Q2@*I#Ymx}mZ@Qd7ky$PGVzycH|V1xHdXvO?UU9>?Z@-Eu| zYYKF6xe#;+inMwUq-=``h$`a;*bYfv`!K)k!biWb*Ye#O#E8bAppW0S47mFb7hT)$ zLu{8%DJX=h^J}3!78i4Z01lzt2K4zSruG<+8dKysw<=IP zYz+RAREHcOR_~{aQGLxF=#%60ha;8l{Cb86;Agrb z1u4?I4BghCy76eW8PN*gHt9=LptC>yr6k% zwdG!qX=x~g!T^wHYczC>bv&^cm2UCL$L~nhCr}7n8KQS9e9aSz>^2FOGqftMZnQ-& z$xvpkD`tGY_&m^9_++`3ulJ)j`N%rSS}WFyAa!((rEN<}-$(I!kX1N=;~oDNSn1K)HIdKr{?duN@wX_-{fTG zAFPKEwCkGq6vN*$HryniFvnU`w9qQK-SDmzh4L`Q0w62i&t*xj_7!i?tzF#0D8e(t%*SCjn-t4bBAsd#rvaNN=#5BMV&~Z%sE4+nig!V&n zzA=sJV%D9>0g%i0dt|nKq_@e#ijrEYqIDVB?PGlz2DS!G+sTrb;CvcK*k||Tp)mIB z_I6^v@wSWficU$G5N2YmHL^aAy_7N}_zHUgLj?m}P`-r>&?bc=X}h!Nu_)iiWWx!k zpFldbV%ckPq?|0)>db}({T4A=h&Ymd82w`j7&5i>bmn|c=kF0}30 znI78(g0gAFLeI3*;Nsyubu45DX=i*}sy;q<>p=bnEn1Q5^|~RIK^X>=KhF zF?DD&W*STXgzZ40ka-Fn!OS5zykYs* z%S*Cy*MG6^H+3>!xW`|*MDHqmDurUqEl9ci=-1JPt3YfB!)ep{*I)Ie_0jri+KYCb z7ccn=-0n+DN+)(1|M3eu^xofbg92GrxiH>2=ruC6A^`eM3c356j?G`2$-<8f*5P|# zEcgv|8A}IVTy_+ta(Tqbp{`pb!2a#Kq zz>2$YhM+SQTg8-{%^t>2J^#SON<>8~dLztqNsz;;Q;>?q!niS*PQw)%zu~os9H}Li zKnSv%a|ELa#9;EtEknda)pAk&-S~0bGU?xljPF0SOl%iN65ObIdt_>mmO;aJBu+*+ zY9ywx2RtoF;Y){0)FKpwYDs%Vz%HAPSyHSl7?a_pB8x>UDWv1Da2hk2(Ah>vDdxKe z&BMSPL$T=~=5u(sAK?#P(|CCCwk_Wz-a}p`;1z@87C-qmP^*P(~ZB6!NFWnesib8$_%Ela-m*x^W; z(1DP6dql1HexT0OKReF^W6RDty%iCYjzG;APg)46$}ft;@UfXU{es)H>k3u^5@x{s zAmo^NKK{Kc9g^$nAhEP8lRgJ~z|vs@%|Go##KEkl-RIY#X&|zn_Fn6PL8ojz@x<{L zpH*8#`RyLoxIashOFAU+8WS-n^2ZRHare@LyZi9Iz@qPkZ%LQ)Jd4(fOQb}0gN5pr zyGN0!aXtMd_asDdtxWFrJGt|$_k8= zCqM&L(1MTQJ_UF3kxwGT?Myv05bX-LlNvInGS@T8qTM_NJwILsF8@>AN-AtC2f+h- zZ|S|s)1;=fnF@oMGhR)Um)R5G$Nu`_YG^N_jV3>YmxwI%lRu7_>G3I@5`FgehRP=+ z95g1Q?>?@2L}Qu(j=!inkxho`TD;+VRLC5SB7fyp`1eYw@sq*R%v~BnjxLRuAN5b) z%<`s>6>wx3TH~)L+wYFKSn4sDdjE3%Z)^fOX=hgZ_{VPYma*+d3mGV=28T+8-F$>{<3E}2xPonWhp?=#R(PxL{fn6-2nd# zbn;eJ;AMlAhb<%DUz>OV(M)J5lTSG{jp??*cSl5P2I=#7Irpmf#uq(=#rI^8REBQQ zGp%I~x}}Y3S*TJvAK41XE^~b?>CwY2SgUuw^mUX!tc1v^d|iY58)Mhqad4$QjA^9& z5+5Af@mREC%JLDYv3OwNvN+o>iIewMRe4uQL2?sfQ8g2b`j!ksMRS7qrW=XB=1-C?IsWjw z8_F=fJPL5R<@nAPZqCRKC)FU5Znd0it~ck}PSm4`UO0) zAqUoOeU4f=?JN;22Idk5rZ1r=4RL)fHXmrh(Tumn3UP*RQghtQs}q_MPh;zC5~9Tk zg|brtHx+7V(UtZ1#2FNC_E#YRo;GAzOv>iqh9szNe^5^II1iarUaZPu};Vh zl(bifci3qBV8X8atO}|yrEm|Io`PEtIf%Cb5e8ovE%elwq%33Fxw&_^Gy|=%T~5qHRc(gPHd#I zO9~!y?~6f}B$tq@mq32o<~KW#W8qiO6l6<_Vqibsladd|m2oz^Oe9SHoeEL! z+TZVNFYR98aCB)k@nTSgH>1s__`liduS92XFZc{)mKzg~6I2e+Tr^j2?7b;I@-aqupcNPh>_2VAJWDywR>EPt18{G~&U zto{D$0@B+Snk!BXVD7kWPeD_0e&vBd4$7CEa^J_La%|q~laBz0GiH7z(`hC5cTDQT z^nyW*BoTBqY4A~W03eqYiE0hR=xpV~vt_n=L~}#ybLD_+?{0eG9Nm{}1R8dlUlZi| zZdQXRr-`~TwUKX_n7mm_c zvJf8#6p$l8PX74&#|?$`$1QAt<)@vFn)w4Ov?Q1d?@@qv7>)drVmt)TEs4bEW;BWz zirrU&%=lxCqkOfZN4zU7XHjZZFtB%WIU`AJ{1l-mbS=MOxbVaM`>f5Vlzf~C?|Ry6 zv!y9N=8-??i_4Ao86ar>fJ}&#LkDq;^qZ}va-*mQObMGII@eB-4?-hJXp2J08PmbG zBj`Ssc!f9j{zWPTNskHZ4=?;gJrjmmu2ladi`0&<>t-e@C$W4E$P25f$R(cFh@R6(DG1B6c2{ozdc){WdYeSL`OWBQUv_ z566?UsY{gvKr>+a`4qX4p&YMByJ)SGf7eCw- zrDV5@jSll?wG|<*WtmJg&@1As`wRLr=NM2CIY1E-0*{37-fwMfg=mtX9+k8H6{+#F2y%{5~@%2Lc6zUQberz8(c5fYle_z53 zb&H$7Qug#1%YbDW$W7SN4KGhCac=5|%G7bFFZd=1PkGfSfGecLu#@qkMZ%3Kmt)9L zQ2xPDl;)1`!{{47!cnH+&20{SeIoSygHo6kV&4E`{=zAf+Y%C$e2kko4R+iHPaIIh zhEt9D{w9%{uHZ?*p6UsZrIb$W4WYjBc{hk?`A71j)M1I%aGdxgvoh6+Ayybj{*xMUu@i!sv z$td`?b2#R9;z>Cl(JegbJFc)lQa`h$BHf_1U<#xpxkl@n>ovtXI{%Oq140xSxNdU_ z;Cc$(Zn^Gla^fKG{x+g_AAB^13T}q2r(P0gzkZk)TG+i-#H6B%|65cCo4h=^^Iqg%1cn_bU5`E052N6SvoJ$K-OYB#ewKK1gB z)eOlvK3_N%AqlYYo3o`9evE971yc#pbTbjV?$nveg$$vK>pwv@i3$r zxm@9{nyjI`U=08ta*Jbyv=9Q*Y^PCY0t$Q(Uk1Kim@$JTh(~E*`+z%txA(G(sZN5!olL5rlhxd=i@rGY46%Wzq*B$dc#dXeVp3{Cp`0j{A5GyNITF#WI_(*o1Mo##tG+()1QWzt&_T~e_9v`2r|-OZbl@5S`Xzk z0a)uL3|n*rWrZKp$hYO`VF@KWVgMd;O$^j505ja|$J6% z9DG5KG-w1jTy8Md2`hGH5x~5wCU2Alw;4!Wp5^G44|YZEwAVBKT5F=fy-zp+v_X)j zgtTu-(-D&553g}T$?+d=_07W_G1UAzleP-QMDB4DX%Pb96pG_#TByx{52${7$zeOW+$^IybhKYFE-NLNG zy*9Lux9A8!IV5^b=E4lruXzkK?XW-1>)JiR%4>X~F6WQ0A;`!2<= znLYTF4lRU5MT(=KDEG4lE-a3q!m$~DOTwI?3!)wnMSoCy;jpY9dn9Pv2E7XdkYd8m_`l`-T3T)EKHfF~ z!c>n;;8h5e#W>C8m>@R~;Df4cPi|v%eiE-z!`@OJEy*12&!N4v3TBR>@GCt*&QBSUIXAm!xnDl zUB(YIF*)WSmfUa>U7Y)iluecZ!|f&G#r5D$_fu70S2gwQ^j~B_qec z>;oRr=c#&pt?6381fU8JqA;tC&c%!C>XL^Js_-f?5OPPKlDDM2mOEj_!{xeUM?_N3 znVVmgmLQ!i;ji+b<=y?sC=JhN|BGv(TE-h1L_d#1c&B|Bol`Xb0FRJ4&(aPo#6-fB>t%-71-DU$m_MP z3FsfdU4%2I^}S`Kj+d;Q zh^>QB#%CRSLz^yJ{VmM2@dK*J2weX@r3WfqTdX{iS8hHNZl%@LN#FA9h^0rA6)xW5 zMwFTqrWI;&sLYw~aE}9QAaqg6Eaiz{r=HHq-OcUo)QQFuIFbrULUMA&`gqa$SpV{Es~9d#BP4U3rcc|v&t7ZTsT*Kr5@jNys&|H4LL zt}>+!ggYv98+}eN{-y&(j#hf4?1SKrN{&Wlm&3}WQrAG^|=pXj!tbj!tYd2n(%5m@c;;zikXmv6rC&)9SdS6RRc0D?~|La%?7#%otZ6ZL)$^ z@3L1X@cjW}Bn)SP7+Fliac}P%umWQic)8m3fPYU5XZk#E>{Dfd* zyZSn>qN4rN-?u;DEGtd-FEVn;x_u7ynoJIvcG>TdHTFHNRz9!>p}KC=2HG7vbdmLq zsj9760>eVq=+W`v{hXN+n8ui>hzJf50eQoZmyEIN01F)6Ut7>Vu4MCd)|wi%f>KBT z|6#-f+=Em@_#_v;N22m4cILrIA20SeR^wIl`v`N`wZvU?3%dtL41>|;@$+yIJ6-bD zMt}%i*T#jU;7MNqA>=kfdRU}jm`*OY3>Kda48RMhA;XZYcDyCO(om&yd@G?}ryP^b z*>OYO-W?-l{<^oicH`8+&6cOOyv{58Taa@X!8*Wy?QEAHUVd%$eM1mnbNnD2x@4Q- z$E#Nt<$uR)8?+GFBVX)*zUzEZ&pFt;8Of~*D56G)>Er^W;MWt3JwFYO9W82{4_awg zLxq{(pRJ?nR>32|vem_|0?3v>x|`;M@7f>Q$M>OVTAKun9Wb6pZwC913yBxk_p{@E zSfODZQ6hi*M8<}Suys{rl^sC0hXu2R6%Xk1>sm2vL-0z8-^@W@Bl_s#?K{^-*5QKa zqhsR%#TfttM6XTvVEkvQ8(DH7|HFqj>##n8g}{BnH0WuYbfz)#U_!n7Lux@|GMpI{=&_UR`u ze)y$kMs@tHmYsv5lx3FWT$M6k#Um+ZNb=KHRW$6l?x>>ch-8C3Er z*wVjRu0QO+oi#5JQH0TyVy{3#!vu|5HZzm8ed> zER@t;w{qN4&DsY9;#yq81gIv*4i87 z4|7gCf^%eCdg^4Vf0!Y|ih`h81dp-n81I%urH*_$+6ZJVIUXisJ2p#dy72_BtzHJo zwoBUhsQzIK!3ZOy1Ntg0LXFXyo{RH=11$qkDstL)B{@J=oKQke!(E@V#{o7 z_E;_5LFTm+R=OT`AN(CrQHgI0-bo!TUz_Jd=WB9{)3K_%Vd+W{dEiqB~FgQ(6Td| z35nc#$Nvz2!oSmQ0sp@qpDIL76vL_jSkZw@%fs9yY_|E=_AOYAACEtmbaBv8GW1a& zPV`)O9XY8_bKB}v&~Ms0wo=6yt^Ljv!rF9@K;a;jNC&wjYu-Xg8W^oLjx6jqLEe?# z$4TEue9%vW6p?9*8;*I+lX%zUV#o4CH0_#!2!dUe*QD4I6P}|C@Pr!N^U@@dFLywO73}~0#oPbh$&BYlLn(p=vYnWMy^Sir>^S^Kh!Lg8{-RIGU-uS6 z1>w(I#+qfU>=uc#_$^j_8@vZt^wxn|Dw6$SPU1o@Z@^a;?^t8gUu8plx%2`<7GIpu zhRlD6^*M%)fw5i61(7%m?(*&LEQjaLnc_X|kVlwMC+4^f?b{!2HaU>1ClKX>|M>Ia z{wsF%pY2|#5EY$x-jCL>>4*@WK(bsA!@+ti1@@d^;}v1CX<#CB)UtzV!~0e?o60^xc2J$y}qu-M~nrxM_|zCZKO?lpc%f& z->Yr`E2nvr%flHT^J0Awg}V0yjAj!-x$T^+;S<*+(jcp$59vMWqDbTQ z#k+UzaqHDSPCy9MnPTkM6*~T|&qJc2Zfgpw+uhPmhh}2sd_;B!I$f-%-1pI=cv0k* z5yleWs|V8)$}UkIGrWj%ZQ3dc6e+ZlVNdTKS~&|Zl3ql8I~xP$qktDf5$2;yJ(h+# zjUYPS72uU*29|-sL}B{1f=ml|VZJ1){DvRd;(SF&FSG)A9k_q9JttabM)0b}SU4+* z5*!jp?C#fF>e2#h2oIgz(Q?F2e)e%spJE%#!S*0SJ+P4DB3FcLFe_$)D!#Xq2^j$N zH=o80t7{$$(wB zmj={ASwrmqQbZi-8Bz0&X%-)#1yIJJH3-f`MNZXhyE1Xa)y9X9n0$X}v~KTDaJcAf zJq?)DyIP5m)fa92+<|q3a!DL&e@HbIp4Dj_6^`?;hP97m=Q5EHMw^?vuHJLpz*Z*Zf6y5#uIx(VPaskig-SrvFrHT-Hu#M z792-9>;TIPM+{T>vFlG%DSS;$Oa%h@2w4tO1>kmyE8Ylk4q9x*zT9=)x08C&#VLa1 z$?*CodfumIJc8e?xinEVmE(Rgr!S%wC__^VP#WIxWC;>P0WSVrwTm}kek^58^aijf zUagb7xbby5QYsROL{=C`M8?r{Ubl~C9rR5`$JvX?OWs}JH#!m8*ZjtOL8gci2(Y(v zJ~$YK-BZWI&QzA~aVdw&R$Gvu+y_pHOfH7%1l1v#S56P`bEQI>T*I#?%QfsbNHMK4 z_02SP+s+feXuh#I49@RRHa}tl@E`V5DTTB#JLJIqR4d7^Z$7b|pMJq3Pf}}XLD17_ zdSl1#c%z3HJf_;(RVW09Wg6wSuQn}{!1y5F*P%$;m%A#f6W76hg~MC(p~o&*22BaSbW?7z>@> zG)vwi_;iX9N@B2FDnX)$>kpI;@Fek1JhA2FU#^Q2BItqSl(>#2MU?b3uJx(rg8kHx z0h-z2`jox*33t2vD`G}qWWu-#$rVaOs2;&$L$7V|l6eFvg-#TeOgo1iXG>fBi;}v! zvSS)0AyO65WVbhz{?>}fNkRaJ) ziZ6XJY|s*o7Yyb^oRN7mCh9S4)slFV!%KnOG~#VP#EAJOhnkL<_R8(&@z%NH1=>P2 zbeLASLeAun&0hM7==qtTzRI8OZf9(Nac|6yT}!Q5L4?NW%3{V0{oa#%+#fB3x~I%@GY8}S%_=#vl$SyWl{Xj2 z@fDipyT~+}4xN(tZl4mXB0Cv+tbjH4tD|QRK(ao*!v*tC5Eb{Af<4uow!Z@@>WF6M zoCYsHOMP##0AiKM4=D| z^`z~rFwG)7HmC*KL&B&!uESbkVxRs3|Dd9DFpS*M9kE#KKVJ*SPc zL4dt{JrW9&my^C#0aL~cJ|0w#_PHvD7NO0L$E8^4EJO!_x%=F{4kZ!`aqYN5ebN7l{7(w)hn~>Zmtq(%!y4ZCDR(H-5!CJRYno*>&f+E}YoH8*~0pK4qy`Dvhx8G{ZGT`z3$ zJM#KLRtLz7&<{ZA_-RUE2P(dr-SMV-ybrWkA}?mFDeU_IZj9V)hG(mbM0qd|=M<@7 z-mB%-81+9p3po2uy^?UDNyj3;PlmYgVl%s5zWr3b=}M2yJH|gJdK||sQhhjd)kpp! z9iqBJK>Xc4TpQ;^uzHFyDcNxuy8Pj4d-h!TBg)XBvDh>z=_EE1O$eOP$nLRyBmWfE z%~p*3#kvwU^?t;P*6Q`Oa+YEVZ55=JNmb6=G z_>2Oc4MFS!{v6{D*PkP|U_(nr;K4goHNk+O-p_2Hc7#*ioEa!jXI(?X3(Q9DImP^GnazM!q8@Zs! zfE@*;|FoX;mEqQ9=gYP1#bG)(&Gur>zj^)g&5Pg5n1AtV|0oIV(EgNRKHvW(&;l8jy7IjZE~ z7zHOd$>(42zH8}ga`ATq02=#v0cJ}nweUbRo615+V#o&I##TVdTIR(6fEVFpc#ykn zceYJuweE(0sgUud*60Ib%Dze%-D%d;4oL36O2MW#r`Drv%#j*=QL6_4P_$E5L*fh5YQ4Tw(2 zB;VZJ7(47TY8@j0@YhcMn<8ewOSzBGX`;SU-|t+G%WI zkmH&YU5Lbb0*q1;iZODCMtsRZEwcr~rn+ZvF66+B$&z8<<{0EDWgtfuheXBMB*Hp1 zsYYq%ic~nG_y$?l?4~<}XqxRK+!zZq{@g(utqWTNwm{q#pZbwJP(A+tdAKxZs0F^q z&r+u7C2xsY7j2=asb}=-=b>}&&W2tXah#~jQLBd`F6N4N13_KviF$J|{v1#~?AAKpJEFC$jr_~r!-fWR!47nM4uU6gB=beyip1ivP zLkFZHaGDM>q*`*SvDs!cN)zQa5@#EQ@yFp3(gSIwlJ`8H-Ci(au|R{53nV7)&G?6J205FOU=k1v)IA6`CpjEr!#j|*3<lEc%>Q-VnK0q`<(eS{L>E5IzC(pFE?@BF`W-AhSL2)tRn<0Or^OS zT3xbF+}#!_wm*CwG#38txz{BH`&juV%!S>zkTh6K{OZ1U_Ay}u>=YyN#tF5Z0Q~PC z=uVpQ+*o*Ac_G8@t8eG@SF=YyW;cQU(1k4R`-$3fTlvQ`n-Ed;h6q7wr2pf?{YnBv zuF@}&xn3@BG={y4ujyMKpCUa3mOrIOEz#&DwrU@22IF0+h$kEDpXr1-k zVBJo8?JlN{_8znfMl5wyAuJ?`~Rt z6_Rz_fkgd?1u)~*Ab7~Wsj%EMa%{N;zr*G^vq5_1D90Oreb%gv@ArZ%i60QJX7OSaCC~WgXj=QGe30>Vf|9 z{4!ofnT@N?{?Zd&_incEMZ|AIhhTkzY8Pmbjng?45~S^$$_NOb2})5X$OPtkQwj@x z>9*8`6ieL2j3yIP1N2d>B~k&cA~r&5QN0dqEs(yn$TP;mELeM%y+g(9Fr;@FQ4tW$7{#wle`;HOS1{!vc+^z_RS&0N|#SM3rDTl zj?S(Mydo{wr_pzZC<#v-wdkXr{Up~VhU41eWp4!JOi%<qYaPfh@k*Lt;(yz{NSmr%D=>IP5r>+oW;MRfL8+|y9C=-E;92gEkCR6 zb4gk`7Q(2u%Z>o#eA0<5lU>rmLieq42}wN{E7>Z$7PMPjaNmQ#91c?UQFn!j@e7Q< zsY&V)X!fZ0N|LR$GvoX7+Rk5k)Lc`{ICJR|ilj;wPN3s>--NO9sUkW(p3IgR#7=1p z#Qw)IYBB!guT=l*XO^9mBQZ9iM4QO^yGZ)`U`5tq^w=In(k4-9d1C3y%>E}>bEuGM zZ{4+2FHqhabWn`|4OVhZc5&4fHb2(<03VH`nnPm!>hbQv`{}l`{55QIYrhXVVk0Vq z_hm|fK zzstlMroTZ^j;2)}pfEH~Kc)h28VhV$eA8$gxk2Pl&;Om(!fY6b48cguh#FjPmx&IlZrSu7Qc(&w2KOGK7VE+Qowt*yf17 zEsJ+GQ@U&kyEjxI%bQuyqYEOhp#@++_e9R3Vd~mo4hbl)Mh-9AT3eCL8^>SWA7Uk` z4~o5p2|R6^Te$Vb^}lYcXtYcp;nyrjZB5%YqjHOJw6g%vU!2{kRpohQSX0VYl^r@P3B)0h#8et28 zpJQ!qI-#{7VjvCF`QFQl@o|G+yn6j?|LwDX9DO0mmM7MP^WPT1)cwk4Ui`O>nTA-# zV4qO&np}|T&+g?L^IpK4c;~^fKL7RUQ&sY8scjYwuB&jOKIg4zQ-nQmu zUt*(a$qGnj`Avjd3j^JglWa3|e;y5$t_0?`22lKPF{kx6k&5|j+No`(!2cBkU`C?W z0Z!Z`a@vz(t#^NB_=t{kvUNdT*k4zSp1dmhKwQsA>bv>A-x5HaF(bizswbUuN~ z?pi@y8KG|&A3d%hI>A$-*CZ;US{_BLxY)M4vwpl;E;L58`vu)CIblLNe9&esZX&;! z_Q45kB++;`WCbB9qky_ny^1!{BGr^OYRvMceRnfgU~Q9{Z!JJ@Zj;P$(+ja!X*2jp zV+TPQoU*VeP2`wf<<-F!kJ*7x7Hp?mu29D~y}vQzph6zj4{E{@`zEdnq&H%W+`={# z_Y^N*79t1V{3E-z?AaF-01YnqnwZ6e6DFNPw+0v7R7caj6cpl)pC}t4jFx@osr+sI_%DG<{CQ8t%c-B>A{s4$oUpBd~)uRDj+AX(Rs;2xVQBzljCKyfQ7TLl9WtpyS@d{Sxu!gaDB-4j)?5d)Hmt%tiaTHHds=8lMM zY|8<>Ak7ZcW64!H+2#?^R4bi|28UwWdZx(CfhC014CS(>HqnJ~wtxbDb12tBG3;um)X0)l%ugS*?h*W3_pDHNHe)ea(a;9I}c~uPS zGa2BOHvJJd1I+~ufDeywa!9dL3Vi(V7pBe5u7Q9Ni3?C z{nPyGH`xOisYgOaka?jddA2n8qrV=K$@#7OxQ=uBE z(EJUrCTlI?di-EZ??S7^Xlm*}?dm6WW~S|KHxX6wh@(KuC7**kpnd`B7tva2%PPhT zR5qFe&g9$4>h%lk(pu~BV*Xd6m{vO@E0zm@#3@MRYK~_?4S%~bN+FN-y*^kMs)ai?NlS+F8 z`vZ2~J!a?73&bx%1b%4HV!UGh-1KUw@&s9(&$%fw~^r-O{_fOP6T_TT>fFZ-{KU%&c&r81-rz|5yx!)`FbPIdcGG{xw@274OQ)w}7u-X^a1?4L0H#WMC>&kO-wfLVClVL={1eF4~GotGz(hnvK6<* zjaVmTA+PpR`cy_q7AAHJ_5AyT)r;6`#NAYFyT0OW z#0^rYe1#Yl)iw~il$NRCNDHWnUZKCIk!W3bt#OE6US`#B1f`$uFXy*p6;Xi7wKYqT zzbOk)t(V_&xD!3tZCg$cohEwpm77D6q>kL+H;oJrc@Ri#TE zD?5^5S{7*%_vGtyyzmiB7l(ZyDF-P>*J47g60;jtAm?VY>SaNdgw1^^SH<-TxXIe~0I z`TZ#qvvOtr64mtL`N4m0LMqk`;<4;`7o0Ip;2bCLg7(T-8e*;YXNJwL9#8gnn)_~| zVP0|ITENb*vgRXA_fss)ue>0OL&*-KtmgDbB+NFAwhQq&-s~j#Pa_Dk+1nNNU%%Ts zu5|4+&UY(61@&4QGFkG^#45E#Y#V_YV2LeuDX>+7lP4gs*1{z3pD4;i8b3mb5=${! z<0(Xmz)5sn@zRocIHoorQe0#eM5j95+M%Tt=ow|cxtW*-|CsX_!WpjAzB0f>OD1Sa zL%3H@WD_A*yj|Ciqob_+M4td9aoRcV_B}U}z1dv}e|?zSD+do}b_<8tr3|trStQeB zm#f_+Vz!Xzm|eq z=$&c-Xcq72=pMI!?@HEE!*cWsA9H-KtYsW3&fBf8$?OYJXrcAWAfhIdphnVJNQtWD zpto6?@c>Uh&<@_$|6sIrPKDezxP8L9+F#a_btPqCQqz>q=}WQyfTK+*Q%=$Q?> zM@JgbiqU{kO_#4K2RwmGrRj-ds=FleuqasSH8xXRUuPpBkdQ@Ty5NC?r&~y@JBs}Z zulnUi-A6H}bGl*y|10ta{!D9@4H#w2Pd{u<*4@on>;$~1%oZRLg|^d0tF|5; zyMz7$ndo;;Hg0LK3(6}1Rfu6$;9nf{4>w}w91KFrDQS*G7nyC=WRfsDW z_TeU^YtWdQN69OwtVI(>q{;Y0|AA&Y*2@@tfUiUVciAG$c6Jaou`MnX#HjAlbe*6O z--T;)#KR~lz9iPJJxo9V51K_E34+v33hdGk*!A|!Jo18;EVH}!51bp!DqcSAZCm8_ zFLCY4zl)0E7D0#=>v`7_ClCNx9-k%-urt|)&UvB4COWU!;%{uHArup9dA_%A2dqax2pGc+d8h&i4R@ZE|!slax^8H03{GyZcWdpl-OT2HAwsX?Rh~(?5weORS&V zL_}|2dC3i&wv^2~f!He_jkW}jIV}39#2J_RXYmM zvuKFFPH!jmC3?KM!Ej`Nx{zh`Yqhw&CS-conSR;@+WWL1!!S_^>sbW-&~Rk8cbUjN z)|~ip)QaRHV9LZv$7aRqm`0Zgx9nhskjZgi6wWc0?{4mv%(8$oj?*Y}CBV55p4GRk zR6I5@G8bHTNNZl?8a|ty*wujv_nldSK~3!Yat=Q@F-a=8-D$FI%k`uh6c(gBl4B`5 zwMt>95pCC%RB|k`(>S)+K|qC2=!ln6;B7PcU1VC?I>q7fLc*rp%N>#qzt(kA*I zGT7X9{V02pJ{|A#{ zDm$&QksV7m;GDAAJAz8}h+Tj4NSid-9prP6 zW$niy+FVwN>YhH@YyGdk@ayI3cNvOb61le(~GW;P#ggClLn9Ur{eV0=9i-ZN|^;V<)FXxmU)4={$JW`lFmBoW} zNKcUQdHv3JFy%xJ;#Shg5&JYJhvoDKaka*~&2k?6g2ij27 zCQtb2ma!o`t;>vhO_Y|8)f-TY6|o+1BFo+U%a3 zfP6twby ziQ4>_YDE;vw^{e=p~cE#nGv@#N4NrIT~VK!wX^9AEU1Ip-7gPmFp#gc+~Nr@UK zjUlU=Cq$z2L|b%S1SS@^|MwSf|GUYu<7f-}*Gct2Uf96PRV-xdi+|g(Iza4uET8pP zN3RXONoP#N;cFI^Cbl8O_~Hwa?L66y5f*#fKoO`5thd^HCxk=e!2h@G^NC7bZF#EZ zU?EdO7d>ktg|@h@KO|1R{PJ@1D-Q5cJ#3<3;_fXkGFw_zSUl2?*jhiFwR1nX`OY{H zxBm%PU-{`FjOPxg|;KTe8#gl`A_f)>_>Y|30Mr|JVNW zVvnpV}np3Fek3Lf~!aCNProTw5^nbp1 zb$s~q;XjUA{zRN)9>X<5eS^wIZyz20#!tTgi>5>6r_u(Jqy{>rpPdny5vBI;hHmG{ z@}y{bcKVn1H>arzd$a%i^;%7V$!tlXwfh$_g;&bLBFb(F3JLuC)V@!fE53Q^z zBlEVXf-Jv9)RBV9|99Wxk3OqCt1@PHS14P1O1zn!*l7qt-rZp4b?W_%j2Z^m%8tqDXzj=xctQWMH+g`cr_)sy<=V90A-rlG~z z!#Ua4IAX)Iy1|Z*00W{nkauK$3D_WYPu=I4B64$k^O1C64-~eNxa8klCZ!g0od51> z?YYSsa#n|&T~bVki|Hycfq1-K$)>Hwhf?0&Vm3DNxpBUL8-Cv1#SI1rZy7&kGtG0x zV)GZ6pQ>zfLSXl4hq9fiU+*0;8BKm@{~Uj|h1ml4&q+%%jF@K$yJ6$k_$fdU1wh^Q50Qi zT!0y@V5;?j1#l~WKt82Ud&h6D5lH5J5JQN6H%Lv;C{5#wIec{uKXi}%0@)Jihl(c; zR}YtqWMnD;FSeL06&GwPke+UC|^xggjK#t7K+Lha_Q@3v-#tbE}V9REn0agum!x2B4hphJ2`&KMJ*NfWTBeSp(!cfZ-nGvlZ@PAn2B-Qu+rPtuFodRYkhm$ z#Zx?$4-Z%GiO*9@%g|Gr_pqhl39R#)yWO1<900aHI|uc-jB_QYpfs>7G!2F7Qnxxk z?F0__8qNBU0+b(;ry5C|lU2e!Wv-4OBwb|JK5D;A8`Go1o&vPkCk(qAOO>xTl{CI1 z@7ikq#e4>lENu0ZBvKhn7Sp5n;+0AZY8@)eaO}k` zwiDhG$kirpn`-@TGHkVMZ9i`ICqnDv{ArsxB>Qz}3mb{c$-!;0d|PRj5D1D4VPjs- zx&!eS{=OANe!*02FI=;=-f;soOqJAFzIQR88FY@RI-i)0wZ5#iuW+udToh42srIwQoaJC!TLN3RbQsccYx5rHzxu)t%^{IX zGitrmESuBcju2KDhglkDl{gbbqWIr_N%7DU!~+6-z*Ozb3Wa8R>9WJZ z2TET(B?zjgz5XpIx4+OLj3p)tYX&jF}O%s7VfLbx}b zEy;lp+BW+Dl4Av8L7d{&=`D+c?4OgJa67R9;WjcQFu)-RC~@5Ui!e!9@}&s}40XAs z;s(sq3NTRj{>|6bRTo`Kd{jS2oM&Ub>w6Kg;=}KAlUTFxp(vwm3eeF8$U_Z6c=8Z zN+J*ct~7^}I1Guz4dQP(0{#d;+a6Ej#|sIH_1XX(%v?H-oc(n#6Al}zRODy;sMxva zKCh;Y#=N}Y!C5ZH7v{rWA3p}9^67%6{)VDTah{~2Oxg&iCCgUR)Wg#!e?GQ7%) zIv{pGOnJThU2*R5hmQGcITN9#Pt4P4Y|WFwbSm?ypTBsZZ{`xpj4U6OZ7hoLp;E-V z$JtwkaoO~C1kvJIKbh!5?PAjbdrc%I9)E8b9$r^$OLrJpbtRs&ylU%L<4c^}U^hEI zZMwVbFWj{WuSL00ZX;dC`qN6;fXcg$7Ndj8JYpfl&;;B#mNdONnGsVPKl=+xBA%a5 z6QK=DSavNWTF+*ne3*#ebnLayMH%i*(~I&;f&B&46@39)R?_1NpyJZ%I>y4UJe#O8cOBqC z&~>x_>mEhOz1OMdFJhx=u9zEhJw81j%-^#fIMwOj;`F|WfO}4qX{&utA??MdNCy7C z@6{j3j#0zLHXQC&ToH{77-B~q?*);gUOUN0*K>?A7GQJPl;hx-PrEAafiJikHu(iFG|6db=g(RRW z6JU{Y!C?@N6F_RrL-`dyqD_-O>&)k)v{dhnOS@}KnAZ;&A2u?bCRCmsGDjrZl}U2G>zL>g1Cq4PIzq zlBQNz3I^<=k*LbMvu=Jve~Rg105$WJN8Kvk1cMx*l&?C!=?JBWPIak&TgH%h+C&lk zrDKX$%}^KR-|g+^X;#g$jAhLHdR5wB3aXJXt$6W&r%yki>(Y=IuIr{Omt7n*eEAFCTYw@-v!Y#H($-cn0sei+ zE!r4eN#fAu{NLz(Q-=1Q|MueboBiXr`zSAp>q>oX38pN5`~BYYgZ;NJ5BFb@_4d}L zcD!zH|F;+aUq+1VBEsSwYu552fF>(clG#L+K74n~SqHOAisE&P;t+T-r``1JOZ*k4 zLq;J*KCZM*SwL2mY~}JvZXre9QNdUU6+s&vQsq6cBEp>f9+5s7kXcZ9yxRa%SF?X1 zWj=S(;D8#5!qVrT$Uh}IUp~8C^cO=b$G1zgH3cvV5Pu)Lu#K{QEES{3vMBuk@@dB3 zuJALsM2)(|PXTCK(fqhMhPTEK9k)y)s^_kdj0JXZxJJ-_<`Qa4T zcr{oNc}FLD>v)(&_KBX%cK-VL;uOC(t7KXisN$dqr(J{v*R8GBhJL5ERU(&#SlVMx zFjw`lYkE05ozW8myE#$C@p(F>_{I!wE{8vfOh7o5q;V{(f3X%a6l^nA=wCLcJECbN zD)!jDAL3qf!^QH{Tm18CYLAJn>~zEnVQZX0D<~n$V;5PB6e~$mB!&JbKjG5vKL8+H zJ}u`F!5{`XF<)UjhqcqhN{PT-e6=Q3@MXU7^iPwBCZcQGZ;A2)$)W2^2B0}ZnXp(f zn(8_h=D<0gNXMqk)~?NOb}-{zhVc%8qrR)A15DL6#f+fB#P<(6cI^X0cOPN9=Y&T{ zh9rUQba1}Qy$=VUl1+0)?$XjgXbEFj@=gADzFZaePBIoU?=3)cLQvh^IvY9>APwu9 zA$72Y+k2*MYC9{)&6K~ANzhrd2ds>l-?rFNEW1#Qy!vl+3|DDAZNZm`Rq1Fa%ebYb z;odr%@{#~Sr|$OiQUI3r6WhtDW2bFvJKG(K-b3Qne)6&6@AYDQHNn}N_vS~MqBe1r z+c*|INW~6ZkV?k!xj(P*Jb`{&P=Gr$^^041_&~%lRd05t(6b(YHq*~ZHOgqsCv>y$ zFnorO6WC*ZV(~0-+>T$4Ir}mMgS2?l_x|;8dQQ5*MBI5WQ`auuiy=@wk=|Q(|0^X~ z@TMVXdOIQe0Ph{qs#!OYwA*h|3Dq$Z(h`^xvZ-#K`X_z%?%2hP9WQGfQx^_q!4bfyZ`#oS~fQfyCEaej!p9VlUuMuNLUTE< zEM1HfN8%1Ic99Rp@V6t73mS?pf6yZwB5D?m%uf4`M6ZqXCJ@Wb^}y z0XjssbO~=eI%9s;gpTVJ29rz5xvTljT^ZI1E3N@Q!$P~E_ zEqXQo$ zKJ|@PfKtJtkfvw=><;eXVzkP5J2LjI^;ldD677?dQ$6Fz7F4I{1DVu(*_99;*r1;D zu!QR=b%}jIBL>tmFPk}T6I~L4+pU-TP*91Pk!+sdXex@KBb;nA%yOeC1y}m;ARdWf zAeGx4%qQ(=`9X)(#lChKm`h$81Cy_fxWsg~-#{moT0|*p0!{f5n+-%LBOywyzps!} z_t?=#3RBSpZiUCZFDWN_xK2cZ(Wj?92ucfJ^oGNJ{=CQ_G^z@4w}`fuT)RX}!`%p1-e4G#ni-*j)w8QHH-^oT53uGW*+)Ld$(K&fKoWX*n9lH z0W<5KmH;SeS4H(I0mjs;y2aUNr|v4bF90YyK7MkX#5Y75A?f+QiF92NE76qY$R_dB ziacBM;93)lK`bra7b#>^2}x~wDHp-O8t^BeAvz?+^sVjJG@30}Ly!K^1yP^*7(dP- zGOvY8k?GFDlBOeEGq% zN>VUnPbUYAtaSe%7jV4Q-+dt;YaE3o+PIi>gld{$E*}BgbOPFNF&xs7(OHmYFiaOk zj(C}~t_t_4+Tb|2sL{BJU4B61R%$(__YD?-BFNID4i7lGsmEX?ibl&E4ppY{#5Or2 z73pcMl~A{6`MRc}<#VTqwb1^kFNWDAp`RF9r>t9$h6`G*^C ztPGZs5X6LTXLfZ_nhgP><~;XpWPbM{@$4{i@CEp5$yZBt_z1gCv+tM_y{D=YMTU&f z+t&Q!Lt6r|>q#Zq&WpFN3B_Kav!TPP<;n8;MDg`ktVvj^Vh~wG->P}b-iy1Nm2}YM z=GR~DLe^f#CgaATKF8$i}6%tTI`om53q95FN28 zV{Im5?i+VhjlYYD!bR80L{Vyi$r=wQH1@iG^oE4?=;3u<`@oY>7V26dy8ONG~j=#nhS2? z1>=X!opIfA5I2`Xe6ucmB^wxE6*geI!UBJKoB9(v3K7MKr4%FnSaEL+La?rqe@LPG z`NjX%i_N4SuR#LY)?K++O)(yBkz1ljX&^$Q_{-THeCFZw zg+TF4-U83W*Kft7%h4W{m-h%BO&r0&6rS|JA|!9jug!0LU__Vq(Y_)MOpHz+x;(K7 z>Hr#0N{-r)b0LfTm;^v*^W+!c)Zsq&W_2SGSW{g&$;AbPy`t8_DAjAC|1`h4dSI1z z1gpQXKzNP%sU{xRvp<=}z)d@+$>dj`D1$D zNCv&x_vqDUfsu5@hb{@z7P66w=8|kxT8}XWmbLF?<^JPn@A>|Vqy4vEhJLySR)&&# z(l1}|9;T0e!6@invZlhHGY*d)@m^4KHUcM@1D#eiB~BR472eVx??lPr^-9-aL-sKT zlt5_f%qKe|zTTbVVfAOYvZct!r)k~7AP7UhwD5WVK!;YWhfD0qwQSYrm-8D+&pEy#ssQe=wRVF%mHzaI^lw*EW8^iLFj!BrANrO+%~SKCgFn z+JBF)LbIE%$u(MG&@p^eMp4-(15WpOV#3Xr*tzkRUH}|rmchVag9pSh&i_s5nr@Rz zc@foffR^V#O<^7tNW>yQ~HPq@OWAf z?zG!XzANVyU<(~U-Z6CW`MOg+1=J}NQwaE}>e#J0L! z)%bd0HN*fURa`4Xk2S6Bez~GKY z%wRaE+eV`SsQLm*w%_jWP-b%)ngP={eaAuI`M=*15@gIRWlA9Tm%-&KA zkh!p=0i-8>f?kR}yyL@+&rW*is|?M7E|_iGdO;66qr*b9U&GN#LuXT|6w za3|U7>0F?0{OCR$sYRRah7T)}ZI@5K?Yxi7J_uISa4l9u(@1p1)a&-8lTXW*W@03^ zUXoHV1)$vWtW62-M0ATymi=k)V<2r}A|^Q?K^eAj_b|WgjNV8z-Y=D^qum4dmZTnJ z98v!@5LQKQ>;wVnoVq+`fZ-gzF>DZTzO4t}5E-&=uN?4*FM8}hy1O%A|xW-;2P!d~#hV4Z>R`{@tF75-?;5 zok5&L-8u(cr79dBI)`Ji=(YBvl{DSmz83l z|NC}M>#n5GQVM0zxDyV3S1q)VLO8_2WRQ@+om$Lqp02wXrCy3Q_a`Kj)Lrw}_RFZ< z!Xdp+^LB=@oVJ(n2MtSaZXXCpa)dlZMsS3hoMRT0hdCw_f!y-+uI%~;Wm6ka9-Qn5 z@=&qy@naqfp&QKAMdZql8%xk7Bg$u=d+gDNYt8(A&4-?*-2>Zid=7nz`fNHaWFTG1 zIKueL*>my2niSkm1eZf`FWj`fCG;YZTK_?dUg9vSW-sjN`T4rd~qUvrbzFYT`a zdGPRtK>IOz&YKf!Sp9b>2dF88<1cU8^p8b)ROI1TxbRBjFz#*uj9}0c92FWcWOFDr1XrmfcRs&*US{2i-C=wIzZ|1 z0c?bP0m}-I@M~zf09w{W{JRPf6VKFWv(@nW9ebAXd>klg+%}|6$I_Icur9s#;@{P7 zs9zTU>&MGfWzWp!UdrKi(BEd8%{H-@!~l)=>3G^WS{Ji3i!Lj4;|+zYAWkF~5b<(3 zF*k@Xa?g1R;^BaNjVrKDfCcNJ@LQ(Tf)Bn-(Cuez-7(ax;IGqI;J(r!KkhA1q{WxY z{Wl%yT93G5AkcxG{&nv_=Yqc)Qf?Sad_|{`P5}IaATA@a1&eH^$v-4cL?Saf6dvQR zDc%~MP;eM`gO5a}&j14W>wJ}_oF-89{@t@1h8YcAWdg4wzkj=U|9SQ!CME4YF6lAT z-Qm}E$D>`==?yr+k12&Gq3ZJX!wtry?-;L;pTC*iez=@rhx>ckfm#F#Mb}w=-~2JSGkFQfH4dxS@a3zh9O2=LR_naNm?DC!H_@U!~ zV=)!9^H5cy@-i_C)vku-uQ(4vemu@)=*~-jmA<-(s#0i#%#YG_;ERI)%Q z*?sp}b&gaDkJ~KB?v-IvDMi#))9!OY!-$O6bSkb_qjfb^yd=b6O{ij_*FZBMcpr&g z^TBAdJj!>ipL3bF(X#nn-{R=%w zbs)UnTDo1CaFM2PJt5E1Of8=@wC?T{*kR zGwKWBB#rX&ikf_h02?3eVc`z5_rwOT3mwE^Fan!bcfIHNFqv< z0cJ98LwRG*2BKBhH+T-@N?|UwT;8b%Qx5aG^gPVlG?vIP`5N0}yGVPfByJ^u%cg~m znSQ|1#X5rxyZs?J!gQJ4?6oB@6)ZdfRFB$`@pN7}XQTEXyW4i+K|<0f!FnHSTEHU<9fa{|Ur_bB7H9v%`x`y5(olu^+aqD6d2QJHs5_GP64DE z6)yW`wEjI<4)ie6g{_iFdV03E5C8kWHrGv8u1*={v~oL z_SE>yvcCq&$ZVWTuwYu@TT7?+1u170dK*7_e23bTcXgU-5Mn`Si9Oo%Qw~*|cF##t z0;Zrd=u4a#s(R9!mUDy1*3{9oWBklZ2FH>JmY@g{;IXGlwEvH+Y!Uasm}`Bk!Qwh7 zH10qeKCE2R@2HR<9$|VKOrM#g%Ly~<0*Vhoe>mJkAJS_pK>+clMNT|{TaN`k-t_)0 zjj&VD%4L_x)jw|U$o|-oAA?pWmwx( zE3b6Y6s#?`nZnB-Ex*31g%eCzQKY%Z&-Y&)I%aJ}6n$+YsKb~nMT+Ff8eh2HG`}kc zor`n~_g{Z}Afpeaq($8|a4w2|Ol0mx6)KT*oqa?H%gck^wRA{acmQkg|D9 zw&pd%OA!Ty(|e(bJYCxlE^XVUNt^BB>p6{F6Jz5NS1>7l$n5kA1f_%g*gaRj5c7o} zx9QL}qzx?1pEavwzk>94^Y;&b^oa@u1y*aJ0K!;(CZNeX<^H7F1osDHx4TsMIUi@4 zFbR*-WveDxERS|gZ@$yTrAIx}>PsyV6=b^{(X>r&r*a|I-(d(L*_cc}F+RV-+hfbh zxJ{BAkPoR|0#LYL8SuWmKVNqgzaT^6+98MRSp_huHeQ|IzU$Q!1Ot^Q&>-X8YvF1U zv~g-p?ctAN%MnZ@`SSLUDYTE->D9#Uw}P=0gDt#yf3yo$A{o>xCmX?3*l9(zkf>&j z6faboX@ydd(#hjK!?x;pu0hucNGnhj%9R8}ylOud;tf<;U1kpdLWZ{+JhztGy^Y$*?QC!=acHh53wCEUD->y*};$~?dKt7hd*>36v z$OxrMLSW9XujhH^ip-3ak*UP^`)btLwjm>8UFP+jw_$XG*&}WhI!*_46D~o*SAw7% zE2$SPv9KA>9&#Se<*>>`;KrBv)E@B^7nHGPh*y^-Q|d$v1FRYa<7eXs4>G{)k%$&V zM2`H&L|uouh@=IgJ$tPbay7A}h65r^^!ULb-$DauU6LZZ)jTyH~E)Qq5kw1;2*+NTz^$nkZyQk2b88mD_z~6OA8W!V~Sixh75**nUXc zib~U12JPyE(yn<{17N{b;|AF;8(L9$ue9F=GLpp%0Y`V}_si9C#Kp^?164NR3gk8UE17WIF?HxgW_`(MiWC z>q5%_^9kmbO^^^3e+XKgcFf&@wu?2DTx;`cr)_7I zSfmyMq{>B`DzutljYTdkksObDN-Duy62kQuVS*LBt;ea@>y*we(j$k>4Sb|e-JH}`DB-Yhhh-0?=wM#$H%R9C;dUgkcF)Bga7@?AfbDW8kt~2fhB?vsVX?cYk|)uamouBp!bOrnx!AibJ9g{SWsUO}o<& z35uNFZFvxDeUkQZlbhf^ZlLv&g+d6nYj&sHkbDk03yyE~6}M6NZ7UPk!Og5`c@On( zsTXM->*9*qsGso&9S_d`Zg->W4hwstMgcXcFgw#nXz_ZoNY+31B}!*&8J@H|Nljy*la0h%r$th+v*;REB){} zFd2vnKTMyhj3+8W!*%J%?D6^AJ3dhLKMbSUHc?>h6}^5pgW^1^rCHWmAPG8ecmM0i z5JJ-kL&a!%9Ub2xqqjTV^~)7I*{?DM#V)qf)Ek#Y)h1hJBJ{M`OlcY^xvkFSEQoDt z39G2Z`b`bvKP-zxYFaRzNJ~xc8q8Jj}CDfWx;r0-lTF~;De0?x5 z4dB(Nkg2V(hp6D%2w<0uTzfd${r$J!51SP%(UoU~x5V7@dz%k*uUv6Fi^8T<*Xm`| zN>*O+vJB7d=>ER-U=7*{SfiwgrJt5Wl~Lc>Cp$&u2f=kZDKR9h#8ItMhIsNnP<5N9 z9GuxZlZtl{3oK@)Y$U5fNqBlQQ@N{+Bq0V7%JH3e`MoCk@cTa*qxcU~(f)sIRmNuV z*;}DZm57C^_}H3Q@{*MiMi^|0oo#K3r5CJR_#q-1e<=nsxus_$)!@+Crv$^!AjVM5 zgIw=Ds(r#bU--d7y=21PC60G|xdIGQd^%QixMKF2liR!D6~VWviF-#28k!($mJ6!W zsF9*C7Cdt~>4tv;vw!t14RX!MCy8drvIae)Q?xIVAg=}xtI%ow&W$I*xngO9RPc`% zTF7%9IPANsXVx&|y!oG&mb6Ryb%IAJlhZ7|xH5YET?HBZb<#t8*Rl4)qu9?QH)ihZ z?2^f1o!- zw>P*Yq~{RspeIg8X{r{!<8Q?m#YI3GS|z_JIeE*?9Vw}Orri-Z?m1(7s&7?te6_^r zw1@RJi4!hUUDWR3lU?Nf@^MdS-1(2jhxH3-U*Y#wL5kr&-rH_%s~LW`U&yO1ad-Y< zI~}majTt&d89T1k>7Tgf+r4_tp1i-bW`JtBZjt;5Xl;L*bQp#<7ht^4*WO`@0w~m- zHDnK&6|L}|x^IBSRpir5SZla)glZaxs&8q2ZE-OA{Z)(N>9l~yc4~9Y*&L7vFHE7# zwYV&={fK@990RL&T+>7ua78;vh2o48<`=h?kR^;RvJ%Y|9-$5 z;((~mosnxPlJwxwqvlo8pbl7JR_W7`9UT$!{i`7~Qp;X+S0RlSWm2cK(q|x(lUuyp zCQca1^lxGtkOt>ar%FxSWC{ssuEG>)C&UA!5PD$Dl;c(~(EjC^;V}{IktO$$^AQpa zNszkx50XQo%2rvB6@Jz^X)nRHGHx;jOt!A%$Qp`C?fE<6z^aKg{V<;qmPYCv4?~D| zK3$w|*$Q_Jp&df5bYv6XQ)4j7;LXbI$f$)u-Q01YkdOC5I{3vYDd|&lexF>@N=M5f zekYp7p7z<<{C3Vv2eFM33GFNJ`U_$_{_UKZyhwz0&L@{ezO~wpg>(l@QBKkXoo|8; zX&?n_jz0;AxH4Pp83kZwc30x;70u32CqtLfYHc};9|}m;I_5kj-R08-4mtq|Y!L>b z5<-}=5?Af*Ekp|X_mpOO$NpnV%C2aa!SJ|8zbyT1tYHq8Y+)|WlgC6*jMx&B^Q##U z9Vf*QBew>e#-z?mPM5&w33k}dp13_PwP*lcJm$d1!n=u{^Bn^D%%Z90A{Z=HTxT(I zf|}j^9;EVHn7isj7{f&a^1?*?9MmC|j6aufS%R5Xj^lqwm&BN+PWJCz7q8VG5Gqn? zo$IT9TxvJgqS5?}zs4*}AA>8G3TdBV#BN79Yp?HUMwS%alg=+^xGgk}G}DR&@ij`% z07EkPa}HcBJLhn`HcHZ2{2Msm=0YuK>I23^%RZJN%} zb?sSnT9@!V!0V(1f<9jS)e3+nliLn0d`NX)v(4$CYZ*3TXuD-^!PJTHiG#ht{hWOs zAJ-9_5S$$?-NKJ>T@$a|V`g2m9cYUT(DAc}j$uYbawnXPZbr-g{>Rae|9s2vyCvVV z+@jr(4YnVy_uTdP+H|iy!pje`^>|n(SSh(k4$2zcD#Xw_fO5yjwh!~cCQ%BBKqRa0 z!})`q=T+)smuYBXvi2r;JAz*Yki^Fv{?J|ymEN@rNT@@DHUSm9KT%v&mF@pyzT%YO z@{X7I0<64ccH3!*#8`7b4XFW5(}x-RFhlBBdL%2E7tLfzAVkN<6>ag+`DYd}jaA8q zSHZ=M6~*(*FAsvZe4a$sR$sDo>nks7J6Aw)ENajaE;$c-ky{EXLqxoF01ZAFiF&b@ z2rSZ>xTTHQL6M}E9~QSr@KSx!Uw<8Yeb*TDjJP31KjJ=`k2VFKA5#Q6g%AN;QLEvkI^Z4>CrWUIoQhb4|!nJFg2h9m3IZaAC*yG_5D#o2EXsIF;6 z2tKuTTE&ohwXc~jxo$Me)0w%cWXXMPg6i5CT}J+4PKqnZAozTR%+j%*ZD&Al(f}5Y zUyc56yOW%}s>4QPlHpertZiD~0cl>mYxtk-HayVb%2mI#7;(FIOp;2TXOKww4+P-M z#@z0G(+6hmAccL)P|zm&-E>lan2mshOzRB7y+}qx9EQ+*2K@V#G4XrCRKR@{ls?4H zX~+$w#FPhG)NlNl20fYq@5f>lQ3KcO;Ui6$TXXeZ8Sq40a_DpW5OGSEj{doZw!q3J z`rF`4H=+h$9)mP@TJq_+GjuQs7HIs9qLj~K1Rz&Qg8+V_Aiq>lK0W0S7>UTr!CX)<`Qp(vUX|DlDwI*BCf4*0&Q8>krEe9~Cf;IF540Fd)x= z+&TCLRtm;YFqsU6_YGN8AzKb_o}_p_HPh2HM@+=C^V3VK(~G_e$&G`Zw%Wp7z!!I& zh4F{R(4dO6`3o;K?73}cx=noD)IRy`;S+7AyXq0tSyxVwL5tm8wNs#~PZV#v%bKLd z))?=r$ZykD0FRIIuRgtN3?$W**&;=zKljZ#%;jUReQlzjFDvmaN1)RzJww>!V}#D1_4%7b(se-&e1 zEX|*3)7wBd(7189vKMnun5X0CpgR!@TT?6J%E;QbW8tQ2s^~=Bj^JQMRnlhAl)Uj0 zUJmU#l=LyG0Z0SGoVE!JkI;XcE0YDHivy0&{=i5 zH;_Xld$oGnPTXy}s#oC+I}3IjEe8ktKOnz_ah&$MJ6r3-U$IGt*>$A#kDuJ-ch{&~ z&b~R!_&IODx8kdP)-_GLpYcoRXTufWA<%jE{@hiZwpZljs;<+o+|E5m`;zaM0 zyf@~OAt&RyGB?3qtR01eHRYg$7`?=dExZ&v zv`B9p6N>Tnl!(sd2jlcN-Lw1YsfFeQ{~*pnelu{0>m`#m%q_{Gm=_W_esy}&0<-c{ z?UJI>Ox0k?e)1Ek(PuPUM6f2eNUG^}>k3I)F{*`(QEh%Uc$$B+y`kEcxaSEl@g!`% z9RK;s1h)UAn?+&} zYT~EkhYFFh4<#yyQHqy$cB9eYNM0m-(E9pzl=lkG1iG9&w?hGYB~=$enV0e%<& zo?7fZ5FcheboasG3d$-uDr{|jL*`QEQgF7=8kPEihFVD>cGxomnYelw#ntZWD`pg6 z?c6snPBTj36}{3aubU6dNIxm;fHy3kkJ^)yyqs7sMDF>svKP{pWo9N8(W6u>z zp)$r+N_3ttmdT@`YM6r51o=8=?|@q{4B*l(FKB$sx!$C8J%2zwp~_it%Y5|iNmFSO z>?#Rn@yESP=}+kdznx+F6TIji&9}awQPNqfHT`E|-)sQW7n=H$+^+E( zWQoc^`Zz;_;lLmSm_0=RiBnTloPxHX@-e>NeYrVbojg?$t>^$ssu`9-88<7*v&6BO zOW9;_UzzvXMb!`j6(EtL<1I7I7WX_391o@xP5QSSPI4c$9-OpM0nkh%hH52J+TsmM z?(R}D$j!%D_fp>rF39Xb7hAemjE+v)a+^l$$$iZbpG4psrMbiyo22IbrxMvQB^s9P zu&=lEza?Q~?lfxWzuZ64QJ5DRpzT`xsVudI`_4tcM0{=jz7+%QHBd>Do@Dc$5fFL? zib$mVZYhas)A_EhZ{-_DUU#jPzF|`*{WUns{?Pi-TvgVLbjh&R#$Ffy+`1cSJ22k# zF%GIwx1@JSx~oz0WlOCRhH#2~Qo2W~gUci{rvbTh%aC&?#l5bQTSq?SD=7(LKQ$2n(iOPi*VTKQ=Q6WXM>x_a)nqiV+$B(*L z#>1LRjmjbsdv^6B3QK>%{2g=)>OV$n{~V^Izlvc&G(S<~N;2^}%v`Jy=`?`uEjGm# zE`(ML8+DQ)GsymDYixZz+x-f#nTC-`5Sn6%W#JPiaLLhw@W?b@7ywcyH5qfya7opF zZBiH96-sx%rG*wmvq=mr7PG?swh#hfGdwx9(N*4G+Ud&6B!dd;l*z}&@$J1KuM}s9 zk4>4SQL8oI1#kUsvaD=%B^!3Nr~3u$b-Sfb-L$EoEttvTRm(l1ic1U8J(+zOvM>Yb zqC)`@ZZ2$L2p@u$b-X6+a#%5P-e^>4wtObIp~KkTd2`X_7X{A631e0`Sa3QzZ+BM`NPl$~a5)bHN&X7-(EokPW zZOL+I?7MP@#*cHvK=;|)TmH#Az^N}XHt?8#>m|F?)~r?}@gkhsnp(89*uvh&A38AI zj|4#EjY$AMRN4}m1;7BbTSTp!vn#wje{La&mZjuMJBWZ4BNpCdBW+>GEZnMHv&WXi z?9NWIFp@`MtJ$}pX!Z?%RISp6d9@Q3!|xB{Q=J4};}bD$!!G za})Y<_Jy!1M;JBKCdkndhZ?tlk-7_Gl1Ai-prMKEd=?2u&Y-zToOH}D%Wxs2Rbawi zJCTPm0pSUBO|?wpjf^@YT?#QU{@hbrHnh;Fr(T~htg6td2;K1a<0qBsC{!U^+b*DW zno4mfVqUKv(( z>{QE}rqD$NPp!aoG5qSg*Ddo9Cmc9xGM|TgDvfa_FKmEn?jm2>okaE~}Q5p#qSC7Pp zBrmp&15X-f=pD#cdeG^xrNNNQiojq@I+WAdDH{F?W3acUp5x3UGGVXa+H`$svAVxU zwA zR#*!2{Jni{nNqlJTSuJv?Wy6CI18)|PMBfYfyj6<$+j1(81F5j)*&Y0AY?9}GuQ|w z9QilE<2YX(SxX^op6H6rZnyN1*NaQ81O7M$>TS7Fv>%yx;7ZJ zMAmL&O1lNy?y4yhZmL>$Zbw2}NTW>__Riw}9nEy5th3jtQ37;}9Lgha-w*PVLCaiS zGDO37xG(|*$fEaOx`$)BAYwANwcsDF9Ddb?!R!Ie#2_U%eb&`L5)0rWkxZPc9aE*0 z``s2(+~%R3DPwURjUHOzqq(m^Hfxe-;;Ln=7Hui0@59foU9qAkM;{Ib6Ly$lop8nc z;(wS7grp~jL3po#(318sWi;112r4eiMp@j?BZSsIy(S#qemXRRAjYW<}yD5II;WaksM^6e27Wy0MR z=U~fgyyL60?J=$O94j|s-nfW_$iB_SA;HZ!n?j&PrnckF_3L}7lz`yn^_JPvo#SEA zUt!lx9|}+XC^pZ8@0PVZzG(WVkyMuZY|=z04{e7|9RQt{5=L4I4&3;0Zd-T)&xM{+ zcqX`!9O4@qVwdk687)`D1?kdEjt0Oe@Q?QmBR!gtb1qzZab>YUUfVOYCL}Z$o!sR)ED+cVC!@Pbsv=qmv`!ufD*<`N`|SK;ARnqqwt}b1+0S>LH9~i1cQ`?$-(*8x zH82tvBM0!mTR$K~lB!+PQ9YVK@EcsdZd=>;pm z`19jWim`NA_*YezvK40LD$|xqm5QP`D=cXL-pb84@WZwPPS?@$0ro>EP6}&@8of?| z7Q2LNAsp!*&!hM@Cr={HD!Vt3zd@Ps>b4p0lgB*v47W5pK>1KALDVrnfQ_%Ow_U`h zt6L=AMQSnd2J{dvg1kOb6aNZVNPDh3O&AQ}`iWLLJ!}KSJ+n1B9D7NEQ-qv39dLzt zbuA&1WIUY8vz>=~vj-2ytntD|J_H;veg$rrhI1z-SviiSj8WZ?kK$;mLGUTz`iEiP zcl;8Bd9g{Hq$d za7td3M#|!~o8-|ql1pTIb$C~PM57c|Ls}EP5O7{;Pt{p=g=I#tS4A zMdR@j*@ijB#CE}ZPF+v7VbIC4H}Dz%2+Ha|!wi}Fa76^EwiPXf`HQkgG> zr|HiP@y@dVeAYRY2FhFQZI%RI2Ry;Gwh(QWne76R{=#+ToDnAyBf=d zf;|PpYbk7oqFC5=m$y!$(x-UvUKSj_iE=kl!O>lt!eOO5_P&Nr6JXu$7O7}Mc+F#w zo4lF1I!^d?d>M=dxN*>p*yT;~YY`qJzMJ#xvkKNM__=6caTIekSP5;ByJDqD8fe}B z_=;7kmi{PPh{PQ)4k~&sVrO3~@tXT4iWFzy?keTGsDA9z--zjTC-8hmqlla6GfW;b zkUcE+o?I+67TSN3CM>b%_puIxrvnwlQJtJTtF6>jrUh#;Kd@ib*4w!N>kX%0Xr(m- zTmO*?3QVFhVdz8$fnwI4wx#ss8&>UDlhf-VF(gmH+s z#_hc8^yn_B^uP!#6ryD+B7G(7jEP6_5S<5)kmk!~0CM{J&!d`YfcxT@o7rR&=vCNv z{k2ggnE{9x_YG{=HkoM?=iAf-BQV+*8HqA?^dmJ)>KVbU@kLN}Jwz)JyMUTkv{R*? z>dHPAYVg!r(FyP7IFZHi(+5b1E4!*Qt^C6^HX6rC7ADxIdJk2>G(t^bG{Z`I^zT;= z+qGP%IP=rpcJzEx_^TUflh~5EzIH4CA^ol89FZN^ZWR7*WUGM|`}cVi{(6 zOEytk+{bj*PB^9Mu9N;@1(BLA=;q)^U2K_)xc=(#VOj$ZWUc1Qb|PTYy+fVm;&W56 z?#noj?HVO@!+`CZ-lp7zPDGO){(=Arg2lN%c*qrc4HwM$BIS;_)%X_h{j_xIJ0TLI z23IysQgvuHazE5sT#ldfYqWzgcTK+zRl&9BbblmJ3p2!>0LkNM#t;5(r=ZviNOhll ze9Xj=NL#L-B9;la9g|oXmUBLP5cI2M#or!(a;{k_`Z~$t`Vinl#~J;;Q(8Y=8kq?f z7{eZ~Y?qB&LJ0*k6#c#81GFq*z-HgPwX}hqU3GmGP&!R5J9UeBQS4-m@|-)ATdU7~ z4X8oiZ%10ssb3h!UB@odp+d!zK)G%YgKZ+%CvivKF{TmP;Vj<1mK}%WVu@oY~7nfZglG?W#ybJ2(LaG3e_{Wttgb8Sen}dQed2_)zzNS8V2P zOqflbUtw}+j3686iyy>@-004y-1Ee-!@;1i{t_XQ(1}P#ClR&bUr1h{>v&JAAbTt$ zehscVBUC;9C_%xKI|mS>0i?GMPTmXX(41Iy3lq{R8YGTkc~_GKLQCjiYI!F79)8j{ zj#+O$%;A;HX8=LjPFijfYpoHkb}{-4Gf?8}jm3Q%@_pX1dCE1oYl$rm>6!yhVeLd; zpCb-AQz8)`1^FzTwoTsc`rYH;Id=D%S{5h512Oxt7tGz$9C6eV2hgjL;< zG10y@t7q=m*=!+lImz%_8g=V8s+fuKiKO^PF8K*Qz@u1Pa204c79kRIJK|ZtOyNtz zptv({@Xw9Z&N?ZSiU*iFPDmC|>M{kQusUtuOqZamVn0k^7a_uFl87odqV^DAy#J0= z#vyMJ`ik)azQ58`O5c!&__X#JbH)PapE;J*5I9_W+YPvUN|?$9l`(WEOng9K&77IP zJ~q0*^V_*4p>2uN13qMM3YfszIWQV@;6gu;cBhle%**85opk4NGy`uC>xwktH`T@} zu_ouf8K~+Wq2Y~hMq}a41T*yYvAnO5&FoC*Z?$al#YN&dpBP-7q6kfCU)0)sbIfnA zNNDk8ukeA#WFCRTcu4_d9-?gnVii`XxTfNSzUBgXwI6=c}vap>W+6M_jsq8x#g+S$$Z)ekc~Y^j_9 z8P(FA;dxu8v#r-&`)N)s30)A}dUQ6?8P>ECd650Nv+Q0Wp+>TgND^0?59S_MQ*tq! zTDYr;ipFTr*gS}_kPMrcVH4rL7DMHj%`7HHV}kN_FI|5+PrOb4Pj9pO%?-DgOa7<8 z04fYH8d!3A*it3OO(%sM2Tq>&ID1rA9@35+AG6RAbYZU`V$t+1UaG2h0+ES*Mb3~| z#A|BGIx6E85mC3KtIj3VLi@*IqmTwWQA+RBa@G~Vxw;9O>Ie^%``KZ(An+uZnCu5oO(8`31G5Nq4_<1FQpKAf3()-lES6^@j zsyB){67*mKfyiusEOBbZ6NjQzaV~5OGe4XE5PS?-g{E1((=*R8RDCVL=BGKLn~RJo zmolpS#Z&7orCr1)5|;+u8jnpqRq-k^3`^_cYW%(5cr(JMO!&xpJEtIqKN=c(+ni263QUPwpO~7lejetaSU|@B+!kvNUD*n-}7;IyeB_yf1Cf^SV>Hep_Ll!XFaa#2eg-ZwX z*(%A`XQ*iF@UjT5nzR}G8gdIOy{n_AX8ZEHQXL>*g`p*5;D*{NJcRg%+vNVm z*t)`&E|5pmRMA?a*Um3m3@mbgKJPy%{r?@bIhUfM_-Cio;b)F zDHITv7*x6on5~+;^29u2`43bWb(n2%!e+HGzl)R=%o1h!uG-i^#!<;K zw1b~)|MA_T{fDhT7zOs?*?$HBS{%-W&r>a?GRl{}w!);ys;E~h0LpiXG2%Fbr!!y< zg1g&M_*S;WcCXsB-q)t&V~k^dbIYuH;SMkKpn+>d|kZpx#@ z?t2eiy3?c_CI^rO{jf-A+(ZK#$%@tDoL+$ZBp+iry`w=<{J^Ux=@@B)se8`kdayQx zA{v$Kchoc5YVcIX=k{2gqaNcqqZOKyXyPURY|g;XlKLosgux+Z#ZM1y0)TBgV)ROt z8gjLbm>H()$nwYwSwsP2g^O`PnztW+dh`g@J4c`R&+)2#n1mwF+lyWL*B>7}8d5X{ zwSax3$Ju>Qmhc9-vkvXIM}gdL(k0E#dp?) zHUzwP#&$t^Yvv^jf@6sKDAM%5eT8fXU^YGuGE8HlVb9O0Q(I%ZbJ?})c{hdY2r-zH zG&rAJ_HPFlU&|s&kSw$@0nQNR;iwB_!d!rK2GSvLe@h>*9o;p)gnSuM%zjg*r5V^A z$*xp{HVX7e9GDnya%Y!#&l9hvM| zNnRWhRjpYE;NV9Btg2Ej-t^Zk-xA<79e`HDpXu&MelywxICk!Bj9KlE*kSJlkY!e_DRmPEpj|^gdeAk_4e?wS!XiO9sRw98A5{p`sjc%bO zNSpAeMpP~uF+#rU{75!gE0dft=BL*RM!CWZ7l6L$Rsro=$SLjE27;3|ZdlW_Y{NdI zPal70GmG=GWxg#EjCb&+Q$|OKl;X1JuJSg1*a7>9;Y@`$ z{pX2tYJqoWD>ZU4gn&WO(_Dd&@v@P=V5M*J>2>Olpe4tn|1iLz5*Pi^C&$lv3V5 z_2Nl#E2TRS-91!x&&G$hAYuj5h?t#Q`tsSP&{GRq_0f*rT|fit$XNb`c$Uv`nWU#a zv9xD6vX$q`S07pfIb^JfsG6rFdnc#7s0{4*p3`{Fo#M%_TkW3G?K5eB$1EE?GUvCk z6g)soS8m8Vd!K6&0yTm}VKyufKC^9;%FX8-=aQ10jKdQ~ zosgqe;p&Rf*4!Sf_niGpid|02R~3)F`hY-SK z51m}((Vq+93;gLRj`JN+mo>!Ts#%Y!dl{eAC z6hyNC6%NfC(efBTr6nb>v(vuV5y|kU1Fn?wky@rsW_6dFuV@b=324s>i~aV-obVs*}VAY)p$@ck3cp`iw$V7d_KS}xwwIpG`cAQhMTO!~pJYJ#$vLohs5>r9Fe~;|{U845zbc=`F z=K?G_uQ43vKv>tp|GV(0aRAeSAlT*gO|a6^d->%U0(eH>;|GQAq3yA1=;-es%^80a zJIxok^4h~o)2N0y=2M5fdS{!K>G)}J#CPO2{*V=xmq$K=n9uamyO4;ZOZD-oEXhun zPpx7hPpqU}0#+HehYg=ET#%NEFVmqr=FV0$PY_!!d?!~<|-9Fmfjv;G6bgnQ3aqPO5a`~dq}XuU(!p zA1z9TTErV-j8qL*=PiM7ozXUq*fKPqxG4pecr6@7?k5V{-g01$i<8MM97)S>G7D8d zK)Kk0CSPi(0NlirnM*XL_m}nv_?UPT{zy0u#M;}m?*tfXh9!|32hfe*H+~qDs|CUn z9)n^6zyq?@I4*OHD~N3?3wnX}kK4?iMx8seQ{p^A6>OP zQlJx|I;PYLF1zVNT(U(J19J@NMM1I18G|~MaNULGcm?_$?GxCve_rg;-1y@%cxn6l=U2<S&W z`hl|GW9nSAm*@n6f6Dm9sQ1cxcBkVfahcl{(Z~ep#ZGY-&nZFv1KA!~6YnA^N1dc- z4h+YA%7A0USiW?Uk<+?QiV&OAUz|^};))ls+|Ungf;DOr>KcYr@_bvCYxBHTc47$9uBp__ z_Q(n;a5$n%2&%*el@!M)o$G7N$to16?3l%wE-R5<055HJksIb_-j#{XN8XGon)CzEV-_`BYw6gh13WgLGze zV`-hRIs8HNjmfo$8W-iwSPK@pio{;WwgU8O1raY3m z)5`L1JTyKkI|V;Xf$!Bk-wk{3pGenKFah~$dP=(Ac2b8ci4$yE-7MHp`&%gHO?N!$ zyZQEH4;hiil2e#EJ-}`i}N z`USg4hYah;94O`6a3(cVh`fgxc17owqiXm(C$#hD43~}!#CR5smTQU>DwBijb6JPt ztGkn1+C83}KYD}-%gk`b(P9A9tys9P;O=|QNyia0zJ$a_Buf>Fq|;?go1jAD+e9*I z?ic}YAT17KfCKd`8RT5dN{h26BzxXn>r1rkON^ukJ2WLIkMq%^Ih!tcdqI&aHp|G0 z+lYMryDw+xDpocq$>+^A8PsFuWMf7`g{4+8tuei3?fHe=?A@m+op%-Ebli4*YVEFMWx+FdusW2W2C$; zcFdyt4obpuVRUJ*4yc2=?x`zSHL|y$#3q~?C5}J+t2Q` zIMz!e(sXU7uT|VGx{j3`r4qMmAC?As$V8_eOl@xYysJ|ygqYf{RbIV^KPMmPF2V*L zW+KwtAQoH8HbWe{k$;_kPG8XRaqYelgh&}qZC+^yE>V^h7fAHe?i=8C zjooywg5Byu$d^txv>baaGfIykW`&ECXcz5GR~9Wt40J)AtmhM<6SRG^SLv75wWnEz!C@LEzqJ3 zvg9GFx#Wrp432MtIP2}_i#XWTM#Tm?@1@sJ8W^R2i(@LlI)1cF2k=+|{sqhbF+w{P%%7V?{PN=!H{d|tsg!~y%qozs zI)T!paw`1T*TU}$X@iti^c2$u=0p*_u^V1H?j+5BDq>(t0k-b=s9uY30FT37i#%-t zS}-w_Tj#%BKCiFA79%$>PtF~A=pfE*ha}XAt{jr)4dJvpX%?C22wQdlUWDhFmiXjo5og!C34GqbvG47j61m z+8a_HPT{ zsow0$>yA4QOuO`c1IJ%GBb_)j9;*;X$wAKl5mkwuhWC_xCb>}6y{PrS@*|k9+pT>K zy;edFjnB*Ao_g0Zt;M5U7K;4)W~HW9TO>4p#*8n=Z{GQlPp;KyD zZi|5N!_SW4TcW=}kP-6Qy+7q$`5sv@l!fW*?NNb7lY|99MoJbzco3(V)N1!)pBD2& zKD7x3)Agnq9mt~}>H#{`*k=(Z`Z6}Qm6K{v9Al&5A2IAZdH4Dlyg1*Ufdn64>db?D zA-8JEKOZ4d$5%Z7CAFd;s zhmBcJQd7saOudZn1OTZhuY&wb1=bNuY|e{CIh?MHMh<9-#xDBOU#PsF<%y~D_3)AT zVn&i97ka=uaDSxR*+c%F?1jDcXL_-1HT|J<1##HFRb@q(qa9HL>!;AW^x z&gSk49>wQsNG!x{dE2U-q(aBt|*3$Wi{OI%imURr#4dbJN%4mIvsXJIs?D zdD^DZ`_v61G0Z|JnGfT`d$#z(-bQpF0I(w7F?!;2ca6ImAQSf$F zifKD|(+Mt-ctQOwNy!QiQL*Mdl%yXobeE^hcOuguT9V%WYGcay&uD-&_gp$b4yV^JQ!aWVoPF3JB* zOW8!MdmbGSb(_+iqs_DhzC>x{32Z^u?D%9}h)y7!zwG?_ZbpjL?L&V`+MLK7n4y&dm>Uh&JN`=ug&1@Ne>al?oPvIT5E>}TUPP(Wx?WoK_5Es$Tv~l65j$U*c zeZE+|>KgnVE5=7dOOC&XIM$%y7nn9eckwROxCP5mw~ZlDja*i4Oc#{ie;x)gBQzM4 z?WfXG9T|?qu|s$F-0R@f2w&%f-L`~SFun^?zgkibpnw06x)_%IAGS|j1v&=wUdF4v zu`fQx*>y}8)wC=ksB8ZiLHW_zWmdm3g{3?29vguawlXlx5}LY~Re$fEy0;@)C}gSm0&X#0 z!7l5=9fUh^8%R)`lx54r5J!Fzke(ZX{Jh6(CLb7^8DuKp)QHG>QCdFVmjd5*!XP`{ zP!FEsOfxz;4#moL2F5*ydzeD%WP6rrX6bK&aPw1Mu+-l=mB;hbP>h+3+lDa|)O-|B%Ys?pO>2O4W}< z-%nN6-CZ4_m}=Ze^3>T=6jM z?u0?`7R!>lhuEXW6$z-3P61*$cp2l*>@GqW&h9YEiT=SkIg59~YI*JrIMv5a=SQQ; z)Y_1!|kUG2uPusD(O=nC#9F zO>Ncjqr~|EB-t3Cu#*p}T^;M~mo5et_1^E1S zhD*c^a(TDboK7mdRZ+cpr-z@B!WfNsKMvP(ql89I5<0lPIsO7YV%?*4=BFAF3Rctd z`g92MluO#~?C-f!^3pWLzv41CAK>n1oP+^+_uqeV>{^XY*tTQuJxktG{bM}mE8FSk z&DWxE)dNzb)|7x5nL*S*+;~*_Mt}N_}P8@85*pBO;Nu!|Eo=G zyZNSp1jLfQcSSv1u!kyxKQ$su>xOKnzPKct+92)#A@1;Gc80GvKka*}s$1ps>;Fy#JYcuMJ z*2tZ^hhvdpfuRT+#uLJ?kx)~Iw<>$)Fza|7C@zlEqGi;aBVe|B+or7%A4=K_=k(fqL0kVJ{QmM1_NZ<1}R(NKaLizU%0j&Qd zlNz$(%;&iuZYVwFCS2~~l+Hyj2gZBafB*Fe z%kBh+8E^wPXt9$>U@L{HfP^4pFL!R_Ax-tq-V^StFQlWiFn;9*9PowAT{7=G@ zZE7lhm{1`;O;7rJun3v)#NRSxr0PWC5gA^#n_Rul0ZQGSADG??Y8Tnrf{Lx4XzJz` zsZW*OL^QvwEIttQ4`iRIg;6@oguQrfKAC)p8if{^l?m z6G{>5J%=9XN2}L7mLM;2f-&BW>+riY;N@pcdA*roE_)|(RiCALq&hkkI4=4E5~tT4 z&2adA!?1Zz0lMRFCIZmn$nog274hl|430;3E(WY8fP2%=4z20W*plNH?zxIv<)HBc z6)(qdHM8+~tshGx>6YWi#IrF|uVVO`GoL`D?M%s)XH#aT;=W&g=?n%fm?VNG1r<-- z89+8B6qevj02%u32Fv;8ymaCd+>y0s3j&-Qn;C^X7YK8>vck*<<~7KuU-*8yK@@0K z35=j$zZ`$ba5}ChDL& z2P2pdrCC>`U~;uQ67~Z{&@)RwjA%ldf%DjV`Y=%;j%nHPZ&uDnf?IWU=fQkYSc=b@yXkp_5Bw7%B!20FLen<2+WJ+oSYKHF9 zJG-Bb2?Q_Rw8@EFe(l&XabgbF#98T;a8%CL0`O`;qE~3!mF+zIv7{e==-=?cEn7g0 z4CY4^7j&ktWh0~{oYQt=eBQg<11KRd?NDxuj&B(g<*BRtlB?&tCZ2vr%h0fm=xp(= zOpOaJn;?C$eres&k#_44;R#5-7n%lT?ZEGejhZ?E>f2c(C|V_`Te>~}iymfX#(w~E z3Z_ULSeVt%f4&-TR0%7PZ#L{OoG@twumxg(9P;92@XYapzv9^vnAZo(XdzZ@CUKVr1S|L;^s5xb)I&0`3$)l490xYWMQA1_XvFO*=zm(8!A< zvr3B%N%~kQt?>P?>rJ!J>@4Z6aGK z^a+TWOoYq3exUf_-N~8p4dknSbbgkqzo;-?&NfStQpV7&{HQ$pU(^Y@5BkT*UIYcEF{{<;U4=Dn3Nmc?NmW5@+Zi2 zU2+xfYuY#`{N*N%HE2!sfV_LUzx(PQt9Jepem9nhd-n1(PM9kJtRRd4fksKEaGaF% z@hM_cT}cY#2=RuGu<1#y*<9{ay1qpHjp=AZZy@yEupAk(A>%< z8fCDcJK>g9=E6I=v~CRAv8ID?l4sW{`%&BK9n&y$qhRQz^(@#^%fhFGB-2nU4p+{I zKu*FxcFN2Htyx))DsnopueW&2Z{hIC8rM}EZ~dih-Y?@s&QK<#qros%!l|5^A$*%M zuPROJxEnXLF|(JL6acj3^$fdHv*ux0tluw2nPsQD$^?(k>@G9F)qem1nIz_LaX?|} z^)Qd+_FYG1yMPYY1}2jHNQRi$&?Z*+5@8lB2W5O0ym$SYCJ=O>@cqmV$D~cGsc13M=avkM~ zB%t=+f`T*n-96z)kN~Y=L)x*B4TS8QM2ozRi~y@H7BFhmyS7ineK zo7d=aQ*{>{v7q@?;jxwonwh=W#-_JyqEmz$X^lyPLjed~(Ey6A!=S!br~Mf7UCx)p6QT+R=?mtkY$*bil4d6b7y0SqMuo>+u4xkdO_h7B_^ajUmXB!}MiMKcSb*zw z5XusRA238aqONpFsKvSj4DmS1+N-ITqgv@NxLZbh*ip(ZZ)1eA2#+Nyp)&zER>$WC zhZZIq6WFhncnTgyi?UGS(8fd+9<1OI$&Ka-b5L)QiJh79y9v%1I<%4eU{>bf?TtT^ zp2*Io(iTXYNDEw*;6wWd=ro#C?GdnqPzm-9a;jG?XeC-}N(5%d{^=RAdqR{TXVS{;6Q$`I)aa@>g%B_F zwvNn&&m9vtQ;y%a9p6*Rb>8Xn=}6in1Tu%ObyKa1k9TTn%R*SsZLVmiVz?Tb20j34 zVmJn|9L+P7+MmtF``hOMKiKqIlDv4(J;mzttr__>eoVu?8G#RAM)on3fj zQXvxBw|=;=S>AEVkFW>wdgsVj=vimJ3 zLN1tl%wfLg+gI~D#WG}6%>@(Q5cg2Y?H?Tyh?tTlCJ4>YHLY7J`)RR~xopB0S**Q7 zW?~|Q7SGInpgX+59|$^~IJ%ELwZrE%QF3M3WfI$QvrgX;1{j6*=3fpzzxLY)Z}9Ty zmV%Li$JHrNiR@M@CAF#0$`a-=`E*t0SJ#6LMuDh4D$hR;OxHaKcaywD|+WuoMKXwSHL~SXSfb zc5!zv2@4@sA!+PCiwlH$>a8N|jocyPFcy;d`(sQG zVb!9|@po}=Eqz4vvP4Zv>Oarv?G{*`eeiWImSkFT3C2(Ee-w}?8^tVO*Uo||#f>+xOk{qO$v7D?9U|S< zt^k#0vVo_eSmdAGiS6vV(<=U_U%=A~KI_|+-Mi0d z>?V-pUOyk>*})(Tx}5l=zwpfr`{HNtlC~45T28=i7h5dU`ogqXLhik-R4olho2IJ>r&LLxoQ^)zh ziwAVI^b^(m!PsqB0E1hPcl_)TUo_3p5CVphtu+9CEt4f(z6=md$j&A}?^turTH~IQ z${>XWPm)$zSTf*PcS6+-mW9DZ^lS2yt&h_~@XR2Ds6vk5s)^oywFcP|qxPuI|GKb= zCrg3JKrcd|wWbxxwLfq5XA_n762nw^RD=a2h@Ha6{Wr}dKmP0QNMhZbRwV zh>UWRE83QWL$(ok;#qTA6jjM7XgzeAjbQmYHbN)jekD2VlsUCrTntKYU{4&ZXaPmu@a zp`O0u1(=U8EKOfh4XBnJKfRZmr?B^JFbGT8i6ZcaZzQ+dM16WDrHX?yLd{y#px~D( zv@SR>xd%2%)&jb)0|gF-u}DBcjhV?(xH+Z_qgq)Ad9z#!Emr&mS2{>PK7+<)?(yCE zma==2oQ5=5DVuX$o^P^=G&eb`rOV{GN@C1y41I-KTRJZI6zgWmJqWKq`i0q9Q+lnzUFQS!QI8u>RBSM=x zre=uX_nez7@$JV~J*oh-@Qt{?Zg&`y%O%Yz?1`9=Nd??cv#~I$yODp4uZ~sLIkl*}Lc*}FW5iCaVu_3X3+v4C zvGh~EhlMtiJ^0HxZL^mvBsJ9+(mNXGNW(ZOD)BC$or58@xJ5wpjK>r>Shhv9VVBrO zWgz~r16jwB4Y7FgbLBf@q@eNG$bC6?5pzT zYS!6Mij$79ZL)&=2DAhK;Sm@tmryIWg_emyD{%|a2iz6GZ=-j#i5WapZ$C`1#)X*M zVM$F<0pD5*O#ew=8MU<(O_;-B&rv?fFQ_tx=Eu$dy14-R$T1oPVbYbT2ie)1eSQ$D zoNb-`y!f~A!5wKc;|RjUq%WByg3pYbw9QP(_UrrU~j?sv~qUh3=F#Tg@BbbGY*^eDOJ{?#z=udkIImLHd z>YY~PIK4{`-7I`{nfXi)w|wcQliNl54$c~yOv!KGlG&0Sxk*FqYT|TBMC9k=9Vpg6 z&n8v;*O_*vsF|}8>s$r`H}D+NUzdNAPNziR>Q^?M!z)C@{&ug~ct@|WlDB-%QCdz+ zU$w6xMG(tk30z%V#4{A;#Srz?*)#qh3<=m z<-}i8ZUi%+WOZgp6Xs@ON-cgjeP(|FMlUUY6QyjDjJxEe=JQC~#g6fl%F}H2b_syi z8(}OR5AW=nu5v2lvXTx$e^b}~6(<&V_4tGc#~o0B7;Hyv*b(zU^a55*_kD;DvSm+` z47a+7O-B@OR9`f~Q{(x*cj|st-@BGtV|Aj6{+W!exO*xRmiUbedqK`sp-SSaLo%MG zcT4CHg}f%i>5F%6CD@oA25pHBr@OO|6c=%Q?C?i1hj`>U$m7qe1sj9}TNi|i6##=! zd3AO6=#d$mxq~}WrcOJ9b6A^J3kYcoTns`>-4%6XhA62?Z#yF}$XeKs&*8)qY6oXf zJB#o7VeEyveN$W1P#tgp5`ZniQG^Ww^PBK!L@Vjy+8dZwo`~-pAK1$}A#&Zw-&8U3 z-#4`DD_<&!LLOsB`}h_(Ml96+<(FV(+cDcFIzMV}GImI6T*)_y=CocrA$nMi$AY0I|gI&uBBL6b+9b9Za*<) zaYpVFdjEtObG(%Bw7Ah#IXk|!v0;!zuG|Au?4TQpeN*`6-dZ;X&h>_%G<-BBvLi~5 z6IFqiW6gx#%8^H+l}&}E4VWU6jvs!V2$g>xhw++$R}>l1b*Ri~g-!m%ih~#f&5t`| ztdf$1BTwcm-_DrnOhwg8L8#tccY&B8bcpox?q+Wo@=J;w*$n&|prj+k@fYW+f+r-Wo9q5&f2`}dH#HkGMWku3ZV9qjYH1F ze!d$@3i1=HD(3{7`+ z^$%J`P>`X^TfX+ivx8rEkAB--d9QhV!=h(>V=fsOyR9Y#swdBh^B8J0fd*7t;JeXw zr+`Gi+j9{e(+DXR>*V~K#iDWHC>7M)*`vIqfASQMOH0}ef8+S_pu?9T2Uq_{4 z0y10pYDg$voX;E&d8U|`C}DMs_=QQnAjt3==!mTGK+6e%t7iX3>&XaIY+p24FeCH43g9~;w9zZJ53i!B-0RD~lRU@_i8jpvYv#J&&ym2MN` z*J&txI=+cDioeRMmiuzAZ4Fmm;*jC`fcEW|3}#rc`Mdo0KBaG%9{hWM{`u>n-adq8 z-h!}|L4jPvyS4YQDixM4V1$q`l2CBF zR9d-dVVe#!_Vk(50z&)t%Qea(|ApM5XTf+)s8mVw;)CMK;{I@U%aZ%v9NUY_@e?>d z7-khBy46CnufEzJB25!bZMyLzfVP-f77hd3nA-ZShmU8V)JEQv7Fo≤U8)4@j-AyQRa>~7{Ns%9Rfw;hd2p;VmC7KFmrtKG4ALV3kt9< z#?PPu5oEI@CB)dG9^W3}@2}F?k|t{_fD`61i3X%;L=KK$_lzo7tXZqDQzS(Dl4{bW zK6K6*6XQA~>JN1hVU@I(RivhMo=$xu?~gOfn45x@_(D0>RsiP(oQaL$mWdz;_xSo#9by<*+F~R zE7^|THt`hk0(NOuh;XS#JMOx%iC5PPkv(`1;f6;KL`AZTE zrwcnrudwuM91Fw@i;CrD4aSTR|hUMW>R4^$MkWV$YfF!m!oIu z*?rKI+K!;Hd{;E>Zlf(q_ueg48gHWCrSn>O-8x;<9l|fd5Zta+lbi&Z`tewX0Dn4b z4mZEu-j?sH^M!E+mR~$?`QQSQflj%^JGS4^XIO&+Hd1AK0K4f&PCjp{vAXvvC&4(V z;v91pCjTcA4=4)39VHI(=FRpF)20(XfTacJ5V!WLsjzZ)ylA49iAerdm5Dc5+=P3% z^eBUmG;ugsNRmz0zW#*6O9XONk``&VZcW==+@`($f}Vr*B~gPb-7a4#-7Ksa>_Q?rOkz%v2oT;0zQRl^8uP~7EbqQPKFpR|I%ob93Z^R<<3^K3cJl?p6@_s;bl;#(x&C#` z$^Wjla=e>KduaVzGDv^eA)4VgKmh0xpu$Z%gpLrXV<5cV+GlM?1nVjBm5D2yB8V+_ zD}YIwhR3mR;LF7qM2!;S!{A|oC8)M<_8B5t40gOnt~@c$GmKZo3*(&{=D=lYrW2G( zrBh$*`%1tr4QEhW&V9T6_~mDF3y?gi9U{S<#m>JrSHMfe?jE3{`CrZ3O+e^z>zaFu z6hLR5ffsN9mozxxCdI)Q@+M{CWe$rj5D~hcHkMU^b zDEufa0WZ*<1+$8`I2Wm-`&~}?5+naIJlQOH(qjDh>_dj|Zuh#iOANk(BP`553CZA)2pSmB0mR;pbQpq6O`C@^9g9a}v6!II(F!j#PSZSr@c8XC;t6l(<|S`k%E z4UoYJcnTGTpcwNH(T(}Ihuc%CeE1zIRCc3I!W$x#8LcV%jAi?jHpC@ z(dX^|-c^ko7 zGJaJ&frA#(JR<2CM_4dXP?&s{$M0@`j_-;FM}DfVPz0T-s)(r3&IcSi5m-xMZXWC; zxzu=aqBsLYxuDT}7wKO);gQFbka8@gZH9l?Wwu+aZZf@|u5+5=C=6l~!o-Svu=9&c zAQlL=lEO~y&OUt5rU&cf0(6&=!bz!*>yVSZ*?qPBRhHZR71?2Yo1n!dVsxJB} zQ%z7Te!ezD-!%1>ObHojeUJVBwfp4g|1coLfBf%if5gA;{CggAByT9?tb;G@$CHCI^9)RnY-}Zm~b(b*B!HeC; z&tAOXUtd1sZLf9@zyI!+-4`!@Km2~@*WV7F{Qlw@AF4x3m_*bs*TYCWYNxqDY?*ts zdiVVmpPgA5uxp|=Psz&#*}tUJ*ElbKR=s>Xk@Pr2ZE&04=x~8BBKPO5xwu<92zBD< zh(EqQu7Px7PX5aFINPcF`9ZBLe#m%fc>>d~8vqFj=eGs(tQt(i z)>aNMh)wGW`xNxcBW@kMW?1eYS#=QP6;nzzLI9D>WBEF_Ke*; zQVz%EqXz9oRehe*maP+Td%zsL7#FMRM26fTz7#jq{ukr#9Wma6iEu~aBHjarC< zjTcIPs2P5ic%h_Hd)e-+*K#bC-W9kA{x9j@ozUAK8cg{EKzH8D zaa;(Zou|fH``P|y54+ll*8=-KwA556wP@R2<9)oI>Ka{ESYt+JoYgSAUqLxg)5}*ADd++@!KsGbTesIw2SiN$$iMzqfT3P6x2NR4W%D70g zvteG7M>{xrfq2jj+kJNJ@eqJug2vqVHcUL^$p$9vqY}>SU?Ezxd%)9a5+FQnfFpVfHE=f`ik1r_@R7AQ^{BOT5FPcc795#k1CMl z&yGP3a95Kr6@ZpTg!)@4EsSnK0tqfm&c_I*iwZ$nHAT`$hr4J2v0||9lM!IlKT9I=3qVes!%vnmc-3kc@fPi`6 z1J;`-O#Lrfrd#B5+(%>!Iz~6tkr1ZxlxihEk-WlacW2Nf3dgzaiB3oHNemXa^{4Xtm6e&gY!^bhrb;A?TP@(mUsJ#ZOiEE_ zBS^{`wfg(?>OMz8XS*dn#vfBpAYc<_{eXF+sDW~4R|^YZ4hWYE1co1^k;KnsH>Vn3 zsaKe^o@b?+fm&E$2zSWyRXW&)!H}qt7#jI7tJ2WI6S`0x9-`J!2raEzVG?*7mt}&ryyjdGMk#h`R3f;1->K4*eRcjGzU~&8yXM~SQh=yxe?&bEkD2PNprCiJ> zp?1Ap|75ZdbOch3v4y_@3_MX?R|+NY&mw{iE*U|$+W6qL3H3J?0Ob;a;K;$Ukx|S6 z?=`3b5|RV$!@P2{pfxr{*p^=p5PW5F-5`cmC9k=0fTn3ISx#8;ispns7yzZD)kpfn z_@a#bpi$2N{*X#{sM&VU@r8Y-4SmMXRH+j8D{WlyVt%EVZ8@B5b=Gm2HfPK;#EGVr zp=@!A>dcP)lQWSG6+jaYOQA|m1VW5{&T%_tTR?Uz_T}J3Z@{VkSz`JHDJFlCkP$6l zi_FC?7E<4$%7PqM<$V+%GG_q`j_(G>BS7G&yalyxC%|_nQjz9gS>XJeIQTxZ4`4mo zhWZHei=;Gsx^o7+pFrv0+~d7OQYm0DH!f#%Qnqb%lXs9@O3a}`E71B9V%RhERuB?8 zMty*x2HixbwYLxvpZH{DEfm~j8e{ceDAkJn(rHpx30TKSF%G@t9lHYkKZVNCUP;Ov ze`UWE8q3o0)}@nYh836Lv$C|;`el{9pvfxJsq7a7w78iO*?!%Cf|W@kyExs|mO>dk zs9nCgW>&#GSz>vbh)45HwJ$F%CMO%9#@iZNL%+QB2pp@~3pE;QR02fsyq(mtF$IMV zQ~n;3Dc0{UGMGzX^yigMrlkS(83t^B!hfaZc`u1Y4rab7opmUP=QlLSt9hayt>f;fP?c zz@K;z3&k-dTUO&3>lZnNz>1#45kfR`=qN%ARy(69s*^0pAH~@pvrF0-iQR^ku+o!8 z#o@xvcT=|?RuE-o1;QOfW zREZ5CwoV}A%O8&a#Ky5D(`?61&n|Puh;}?_5Zv+i$k~FI^8xNOSzAJVXR{AKT6+M` ztO=B~ZU}hXX($S45;D$K(J%Ow-O~18Kh8?XiN9qj+9mhs}~4snUxBcV3L zth*?n>N?*~GX5%!Y@ocyp@T)LLC1lAIB-qK%-%F8Jq{C8JXwnnK@jZ}y^FmlMmsm@ zCCb6;B0JJNx*P0VIw0fRGGUgc|89t5nrQ35EDDi0BdR{g5v2r|+f)xD)oo&ogS`R| z9+Yy=&&1Gv9FQFxO!P`S!tn{SNzCFT57WmSF~t&TO0Ga>?af%x_8mXAJ6t=4RK6G7 z@!Gc1s78}@G_myTCh80dLV>dUO)D=}y;V!hZJpWm%>S#)VD+4sSvrWn@!}KZl&ScB zYwY`pbfaTu?`SSWT(3T+9*PJE7Q=ryc!FkJ@UuBFH{5CZPmMpei>^JzKM`NTE2dW# zSk*8?w(|Gil0(%lR(U)(#&@w>swtbi5=!+Zq< zAcwyg+DR@ObZuoV)bjmzh-!5@Mxt*q7|ywAu^jN5u88APfCM;pJTE{=(fS5dv{~W#*|J;O7LliJ0a`zhc6q>}FoHRX~@v)ICS@6pzfSJ_ee9 z`_|yhoX)<|EZoDeBvym?vFH)@!{8q!S=E@aOH zToXqRtL!M|BK%jSLx5N-$}u9=QVkI_bjj%6%ff zZNOnvx<_2vRc20(fdXoLvU|wKB>6>OkllHS$3%$I3$f8pL}_vc#?Qs*DeuV0Or(jL z>fN_rh|+M#DvI#`xOB`RhbIcD0e1A$mDx9x(WxJkA=7NrZskSL_$7AXCm$%Xt$epF zlZqFzdWCDrJO7yWuj%(2w%JnB3^I7-m4_FY)GbCL3W>Fj^90pKNowLf zL%B_m#|?z^fWd#*`L86#vczU0?svE1;@r#UV~j_ploK7pT4%eVz;Z$$e^V(wcDppU zW7rpdJ&4l0y_A%Hi#?Okfa5P>0i<@MO-#0VmSGg0z;4d8Xo##In!F2X1zzIsu#NM2 zjw22;4Gv(1evTQRLFJcAcoy@XGEa3W0MW6rvV0J zbp97$W_Kb-8TL32gCfA)8)U;M>6|8Z;FXkCzQp)co@*k-Ms%@7+f!0***-}c?Q#z2 zkkC1$gO`6dwka1_LwAU(E$)&9r))Rl#8lJ8Cj$d&3{Qp&bPI4%(>qQ7hBF`p)Bdf@ zm+ph%_LgGK@8cGxx5?B=tSaCQ*LFxf%^XPYze*&yHPcGSQkpS?leB<2d$1O`X|asC zRdTekCfdKPgpwv&3A-F4Bt6(UAv9Q_UsZ4!ODPL^cU7)p+uWM<*2tj=jI5`H+V3}m zMX!56P(3> z*X8ZS4V-o(R*7@{3c;Po=^%_3<*H2tb+j~`b`SsvaM@B3SS%D}(WSL9Jc84dLOl2^ z#UADG!*b%IL}xzq2==~tv7QS%f!F8({(d_ zMZ#<|1KE5GdycT(0yr0rP>u!*sj`UJp=vOO#$XfxwCS|3n5m|TjGn@L+fHF@I1`g2 z+Xsgd<5Aa!O@z3fX0)g?bR~yT8A#9yt-6RF$24Wvj(q?w?}shzvPqCB?#Kek!3KX% z(U&3`deQu0!0%W=`TKw!J;-r{vaeZG5fp8-fxrzeqLat@= z?TE2q+ojPJLFl?DmF!R?)N`pM8Rq6#4QAx0WG9fB?ZfCTpjFR&c5~8Q*k0olc7$|E z(=4hZWKhQZ!8h);{Jic#F0^qOP5^Gc=~3kKrt*B}`0nE!!@M4TWXI$4i4O}Z=vW#S zYZQwl7)}#A+g2WYj-lRiUiaW@KX*2}yapzEbjkRr)CUV}WvWruNt~@JoceDVdH20re?l=TT8wCk7 zSa-&qU-Pau%EW#!v@zKd<6BSsbBBF9@v~k|fOB4@(}kB(w+awJyd@IQhFmS#5>dCZ z$GBq)={tM#tqS6oWFnPTaLDs^>UjO`q5$c8potF)Np~ttogl1l|I>V>#aE&lU@7u+ z^4F3mPjen=MCC>EU*OXoO2-xZhWbbZijc$$uC|6zt)cIht?f)hA{HO%g*`q`Oul-r|GD{5S4PHEfhA%iV?&9LZF+ug7;-JS;bjWKjvOedMobtKz@U7kr0TJKT zt(+V_2k?0+$P3`wTa53EZTkeTC(=ER8llvBNOh`yFi_0XC8h@RBOx_41dMn@alQzy zwC1Mc6H{)c3%V%6e zo7pT1#u%VkGLo7VQ<%x$BCT&faC)5(!%0=pg#M^)0nM8c51Fs&@2M*1uM^+%+{!kb zzc0rt+7UxE7GrZX)DqYONo;88H{Z&(i4tATbmqJU;X76|*r|0D{sD2T&)>S`wCGLr zY`*EGY~D@BxSmi=1;9w=VsYrIgdyq1TfTU`{MpOsR6<>tQt$z!gE@ubX-RnBdhRc) zQ3C50Rr?8_Krdk3;JLy1f9?#$c`>{h)tC?fY+z_R4-<+tYnHV#GWWZ}rAYoO>T$mJ z+_hbS8oJc7?^}Z0IB8_4g!q2v@Y#!Jzdsq*=_;31=tp#zx?27gAN%_9KCxq7Cd-a9 zNWS&t_p2g^_XcKlfMzG(7L>SjOmj++70zvjvN+7YvB6zqd6no;0EO;a-rRXRrn(J* zUZ25?$&*3+zLg_`412`t(y;&vJ|Bl2AIH+$pB7Psm{7oThV{o!wQ>*u)1_GJ;yk~e z5RPI5S3)NyEA1{c>Z=-|JE7nBVH~C?nAgG@VuFL#} zWD)D{(CI;sh8)WAvnVlHPj>@841|VNY+~59QS3QUtz!08umE;%>KrwPKk!ceWLyFo zv(uCqx95LtN4cF;S~;b;E&YAOXLq-xiYH_dK=;nl zng)FbtUbTNPCM;8tP${#R*tW)3ka$>BdXDLx3Y)+Gui#ebmJ;YGYpx6y#nBu*?cO= zgrThGZ@3S&ivqNz7KW-YW?c_>Mn6{BGOZ-b0UjRX@Z8ZK3h)mK1Z9OP#}`osCN^aQ z0P3wFZh{-43;s!VJAUvWeZ>+zZh803xwY$gJpng}c8YlZgD2^q!M;6+JF_J0e=oWV_gbq*tij(^`A( z%Qp-l{fKicWzI4;VZ6ydLe0!-riqKi)w-5B6DztB=dHXk%LgJ!#*gnA#o*Q>tCD|_ z?G~0~{gXb6VGtV!C~UXe@O@mvs+(LpKBJ+Q=!2^e$PmGwGu3uqTYtPq5_8kx>3yPH zCimo=6OWHwC`oz(^>R6ODODbn9NyxC&j>{4Ht0!m>V$f_VPpBP#PNvyk(sOs6uEqS zhB*(glai}6nmq(kxI3j$&-^AfX{W)=SrI(#YW@ibbT}pMnJTSgL1^X#8fr7YC_#k~ z!9)(-@7FMhhYU7#L0(rFuA$4Vj;GVlO8=VGE_lir5w%R%LL&e^wd#C0qV2AK?GcrQ zJ3r>g2G@2rNMXwIF2l-nfX&)5ViB~QLv3gL=&(#gN~s&R%^tQ}3 zdIB8vjl}Vc;xBv>_JYJTo{S#yGf0N|6!zwM6xI^zzljX*W9T?mC&Y4@t7Jp)IS1=k zpvvgW+9~u1uFn^@B*2#1G0As%5cQ_ztUt{|TZuGxL&UiH26RHxf{tqnSQQOJDe{A| zU81h_d~xQKqdjDbz*dkZx)tN>;teVN1SRm zTt&Fxc24pbalP~Z4BWjeCZr_mBj5CZkOSHWIOz@NU2tg9p2rk>WJuWIhv7?9Z#qZV z>C4=Uw|q&%8kz{n)xWAFy3}P#ZTvyPdQ5r=PE545?6&B{)8lG6t@URkgYLYWi3^9D z&ugJr3S!zGZaK`PM$-}G632~%nc3R~G?`;NfMwOeEaY~wDDdm;`d?IONpq`K#ki>e zh$2T`{NmdF9vzZgm(XkrxV5qC?7OVcQc4%A2gTRDk?(gt*&u+{@jjvYC7=TkaNLdc{TL<0pST#$LVT<${=1 z&~0zn+awp`gmf8QyJw__SR?-bsC&2eIF9UG)7SkgAYTk`nzGyWwr5M8)&okU$gxOj z1jroiy8>7QA>zOd5Te+>e%|+6k(HHERbWy?UyL8O2ox$aB37*PkqwFD%ZuMWe6g1Z zaTJOMqUjnStpUo^x0Hpqa=H$- z69kN_5IJYh{Y*f&zP%i$3tS>9Q=9dl$uCW8*(?IviZ_85?!BPKxT7EAAwSUXW|0@R zWSvMF(lT!PmIZJKj4lk7)09GC^x9uJa}jPZ7|u5OIleGe-wRzAzBKq@4lnPDVa?>3 z!Li7jr*K$rZfXvGJIzP{D=!0hvnFc?k5JRLf5d);DWvu>#pRu5^xR6ovSkgu1cw;s zXhhe0LgrR8ydGDZUM$MdDg`vCFPDu=b1PzEybB(-fJgR;Tr!?w$AB<0mM+T2&1M-- zcpHg-O!9HL`j>B4TGqTcwnN(r#w{+K-D2)R%;$vd8~r*_O(4bh4P!^M)dr;S9vBc7 zn_jvo*@z@qlQ=d3r^_I{ObfJzJVc0&)UavGalTg%k-dfPM_8hHE-W(Y9@&?D$;43Z z!^n`0mC=4Tcd*hnx^hwB~v{59!Njx+yv710@jj1%>Bp0 zbSz7&XNd6Z$Xvt{Wac`-JTbwE71X#)Ujq$K_xwvU`5S9!9d+~(ToB74mW)nVMp}|A zI)~eHjeqf6P7Ov;6_bLV)I4R@32S!njeKE?HhTtT`2yayG&X4rpW`hk9{Fq);OUUd zR{Mv;X+VvN&F63(G5h+GHnzJqy2fz^m;IM})zfpYV`%}N;+f##N8RSTS<++BlJi-1 zh7hn@x7D?4pEdE+lxMJPiV3i^O4t(BVlZkVb>jOok8jIsbFujMSLE@S&*ABn15TpRH5_T8P;ast8Ti#S! zZvijiU4gK-Fc{m>K~q4|(h)BTO6<^GN2eavIrbMBtv$J;*y}^%JHVWm18R~`X72Ry z{!dSz>_6LZOtiRCr6g{9*V;w>w7iefMh3`wcH8Tq2_%IYr;$bPI;qqmvln&uF?&>!&XQI z$9Z@YW}+g-93>KIRNHxnag^>6;>JUeR@3#XAqjPGVIF*-WABuX~DVu3+GpCtZG zx2ix|$?330fCK$F2V@%_dq7s$OU#^CUeS_v^H6fu6u6;D_DnF3GTM|d6@|0Rd#{m`rw zBarivnP#uncdvU_!B3`cfYT>eX275Zw3DfiVrU-|r?*IQx}dTNq_@hF6;pE~w_vH? z7H>h6dH1|kTpmJ?>EFcr>8CJ{o9)J!I27Y=PZ;YMGcT?oZy&{baf1~{+f+zo<2tyt z@<*X{+l(inLPJ2L^qPd{tz6U31vl{P58llbMh`G~@KJFo<~BG5g|y0QbxUFs_hGdb z#`#6}+(U{soZ!ggbbXOS;T5CxeLSf+^~jh87D=K4WunXI4=l5lu*22yx2^rh+p&F-dYEBI0j8o7q6S}I9baL&GsT#%K8`%VD5$C zUX9myxAj$RXwWd!<}8U1YQ2&PRR^gd!c@4KRADd3kwv)fr|0AX2}k1Ulx^6t`Wjt` zA)S?Ms*#(KOj0=PWxSqjKAJB)z)nVLIG9oBDx)}Dw=;(;i=>s0V?x&Vqi0Rczp*6# zVU+YmmD67*n|Fm7+seUG6~AE_Jl2iP{uanb*d$Fo&ZySwAfdaa1z{|aCMa}4=YvvX zEYC}kZ}HNN)QQLB;0dzIfp~Q~w~_L0()*zWLFM^S#|Zh=@dL@6w}(sVEXG=@KPD`p zx>V|@o}T(CcTWP$qFKNq+84l5o&h%O#6GrfWS5ZeIXM zt{wq04T_D(xy>wG!4)FY%l$3*f<+@#_7 zK1IFXr&^-{KQxZHv(~D5wZD&H%;QP}$64mSKy}jzFy#_&Nx`gtfU!*Bd?=#5fq_&K z$XnE<#HY72f<7HGklmozLloJR^Tl`0ajKc|P~NAf5E9KciF$NWNr|P7?dVi;aA4r^ z0D@ibjBN>>YI))<9HmR%^h90`aUDBb4s&SoPACO^S6sq#!7n%pjUx9wj!TN0k4NtG z=icp4t%sq~D6GE5Z*#(chW%(R-J3Z8Vmq||+Z1o`5QxJfqR+KzyGvYs&@*uc6HyZE zx$G(cxP)`a5g+5IfuMWfTtSr3sIDcDqleY&~gTly7ia|R+(W@eRXUdW`;i12p4?DTn5 zbI&w9`0}y{CE^$~j*ew4L#^l*lh*ZYVEWW*4G7m0=ZOd+9+FK8V zuc!PS;=Dd)0`d?LlS(*ai>nRCi&8SZHCT_V%kEn+k;OWUQz4xSa!1Wrnp<~8ws?$ByQF0pwfa9YAj{$q8s!o0Y*oIc17#Y zg-oSiBSmCx#(MCNC{`dQ4+Dzh=9sz2RwM}5b=P0+eXowr+ndg$)i6f?T06rqQc9i zQ+g28h*+_KUHKlj)#QODnekJe^~34oO~+JtU`D-;$tILE`-dtyR0G!_eHS+tYR&Ww ztcDq6Oa0--cnNA=3=2H-fcVVS}g^=wW^J8N?$) z*D)TX9~MIsrd9RlCjh^QSM+4y)+flzyWs-A=zs)g=^+2^wYRk5-QnBi*V#>=gEFnM zx~^M~HsQ)6pc%>Ap#Tg!;sx|D(pvrV-v7bK$FGmqpZ=uCNT`u}#$`~i&6H{UDpWYQ zfZl4AuPMfu36x&wDnhWa?cLVC1iO_M%b7TZ||sqv@?4V%ZGSp`!J{ zMbA#(c9Di(V&Z?uf1`Zpq5Rq!$m2H*OO86ln1<5crVE#o52jY-SNfYEH4bZUqDFMn zC-f5#q~*^5q5KsjBL@(cOZv290K11c-x+oKGW0Pi&9Kivi_!MfIlBk#L_?->G<;~L zCS|_2gaL)_=K2VKggjE?xPuA~6{SrGC|zg0G#t!2M3W5a0Z4&7TTAAPGb7CDm9|2h z9khe;o5?g8YOy=T?@8@m*+y<46`@XeqY2>GrI{lL;S3jR{*bKmtp+{k2TfflZD9n* znZ_3_z|YO8N?HTR!!3Akj$7GiP1E?nxXAqZp3qeZMMDX3F7Rq3rLtsLM0+ zY>}LL{tc?s`An72C`ITIAHvw#%o7xrkw0EE0@Rkg$S8Wiz8$@zf6UWxsNNYmGTkDG z-w`17D;i2r63F2c8um)o58YGI{EG2(S6}6(3Oa5MD>RlS0=dphJ_VoTyl;TYlu^{} zvn6n{`h-aQ@@7U}L1=Z6(`Oo&evf-&9h+8|532)N!FG*7nU%+}D9Um#-wvr88MDvw z>Y+w13CWoO3;({Nc%S3T?ARcxiY6*4r5D$yM2R|4@SL`Yun+UK(v6MQM5nmNGAWxZ z?8|cpRqJY>p5=ku>V!sgLH)sF%+Olf`QiSE^Z=6q;-a68eArMZStVNdUX%^z5Im*0 z6wd|}9n^omcza0*49~+%blQdH&u#XKqriSHi>sM0+o`+$So6&}>T%E9M@HZ^DDOZR z8W<8d1xfia_u1iP07M!uLR1uPjp>sOHgb_$$!O=f@mR3pn=eu*ke^ofdg}{!o_(L} zv-Eg|VII-y1B5UijTo;S)W(7Q+e-4*>chjXR}<6j-%s$f5@=SE32B8FXtlo;$}^hw z?1Sdo0+>n^kjy|I=`yqW=0F@GT=U`(E_}E6u(p%&*lVWycb!byrp4`o@sn0jh>qX5 z_czF9G*c9&!gRyzO1P|l%`*4a(Y^+!m zGWF{6!{Y3VJ?yO(t8ZJGvyBsiz}5I~E8=dxaxG*z zP4POs#0ti2#&9{u{9Nie9e24R97#;6Q}Sq2+=R^|hDrTrXzMCs_ywAQDi-MgHUkN^ zk8dCEcyq(I!GO(NT`mxrxKzJjL^`TbY;wtHNX6nKn!(;cax*<62Q^S=JBl1~f5sM9 z*-PeKp>dKE)$EC{OI%@oeE#_oFT*Zv6~8N?eyo*r&@K;&m{$dse%}+}@fW zwT0@}zc)qu$6&n0UwW8?@H{bkD~^DNa_O1qwtH)XeX8=N^1s>7>oBn+*dPY!^3^Mv z4#dJ2Elkp>(o(2StCX9Va&44KwT@NlpERlKiE2n9>+~-uRnwKH4<7Jfb~DU-(GR-D zE7q}ZxJ$KA?6kdm`-z6Il*s`2&9D~4tvc(jrYI}-%C>B6l|$(u8QUNqRRTHdie+t! zf$oO-bJHWLM?l1~Y|-JsGQ67;%Yi#77M(1XwG${*TG@>BIFJ6jBf(Lp2>QTw!6?lr zDqfH_d8ZT}__4WC$oFXLCT-SGzA`{j?K0>oV!3-5w*MwV$xsBhkNXgNewiRo4lqk4@ip!d0E+>P@+) z?0-n*K}}4|4o5eh>aSx-Qkg^6t5z=QJ#TvV)e{KR7F|LITk%{@k~VO%i&MBo*S+z1 z4Q7-BrsPUzHr@g9d716b7sNjS zX>_0x4qHh2ejH()c@#(D_1P)fYRaSdHb!cSWFdCZu>6bLo=vX@zj7>h77`?*+!4or zst*&z7H7a&=aVV>ABn`uC{QmeXyb*wMcU*401mnc^@L54^nfQ~D-2!w$Xhz5M*)_j zIs>KEblRIf@*K0yL~f)4;7Fbc(m^AJCXhjrZ7G1r8olpqDPg5jjfK=O=BDGWZfq67 zJffdkG79p7B7pAuq)>tGxPXT{W0OW4>mAH7PY-$nH6iO}=rfiXmxJ2rtA&m$+PUCl zGS+1i$-3_A?)(REI}VEi6tBN-84oX-RN^#0`A+h#t$&&>$Tm~QkJ6J`JOoXJ`cEi) z1E#w1rcXR*nasXlaAR!ObaX`O-g9I+1QRb_pRWN=K%b`XBrwo^t9k|WE!=aYYh(I% z8jFmURX~-?j4&7+lPjYJD(G{j11?J1xL1ypmzPJbmZscr8!@jrPO9E+Vle*G!M6{e zJ^bsR{_~&qzdiJyAAi{Y<*$GGhy0&jfPeY&_w%#*_3!fEzx?w*|Hps)kG+4`+uQq( zfB(11zk7TC_LqPCm%sdlzx?Gt{_S7?{onufKmK`d?=S!IFaP$>|FHMZf0x~KFC{Sn zLfZR>Unx(eMyUJi`0L01K7RB6S3mvdH{896TC%AB>mUE}!1Gf(No^} z<6jSHO4bra3%#+6ZctZ_QRGt@i(B^TCI9(h`8%%ivpv*iYdZh(KmX^yfcHP z{XzTV4eXJx{@=*C2bb44x%0LAeD~j0-~E62YCGHJcmLh*{-3MwM#jYp(1wq-VSdM# zbfhi44L3W?3YCN3`R3Wn_f$Bx7yOfF6f6&aI(YtUUwW(KGg^vn_;86I0rKU=0`|9p zOZ)O)51zwCZ7liS4XjTH(oy?b{>iE9g`U0rwEyt?htHpB*A4a7K0*!L!pN!n86M%8 z?|*v!a{xCy{Os__kB^`K@bkgm^GACRe|-G#;OB$2N5)kqUp)Bbr+r#qK3xCi_dosk z-NS=#fBNe?{G6)ya68M6N>4%H!T9!DlRm{jDC98MO@xx_+h-D9!vg`$ZqviUvd#~}_+-!ifbP#T2Yy4Mwu9_;^c_;~++?f`FX*YR#k3wydM!ml#%KytOOUAMj3Rr8ySZ{Z>+p*iPl zC(E;ppd>wJsP^03#E(y&ef!wGULU`P0&6_V*6r>;eEQ^vC;R3~6T@2Fw>Ut72ZaC?a?28{O4&QsItgR+9Zjv&z#f6x- z`Q?517%}DL-;$iY`SFgr6$jamP9Q?9zj;^$mVluC_F2(Pg!0}R zXjXq|y!GJc2Zw(>eEe|l+viUoe)mhL;HRIS|0J=(l=El4s!VU3h9CDB5VhhiCHLDO zYrQ95?ooY_6cb4HyPyB-;a=lC_YTnA&T158d&B4ntU$o}*Z;gXaG_fO@~3rx-0CQ* zmu+}ueyoA#bUGpIXxWV@hR#k+zr4 z`~d4m-q_thU+eT}lRr)JmiE)Bgh@3q5UG>?6amL^unH_LfbFAvo-N*Q`b8|Rgkcb} z$quKZC(u$wJ372h5lCa*Vkqm7x8ns+ zm8R7}Z56fb9fGzxBemya2KQB|4}ZtNBJm>$!Z8AJaJO!aH_%C^_&5ll`1Qpf*$4TW z+>)3Z=4p}lA1GUuMynM%9R0fJt2OT(zCT5eNwA-)Ou9`0*%2^#expJRw}0$HsBW+# zK3tHKi2b1lX302^eh$e4n96|5O}JYEn)h?MO7H0W99&=8KZ<``wLD536Bx9;gk8y` z-}92hdreI1UO*B@M*LaEKXPuZBy5$7DfsqrmlA^*_oiB=od=pa$9$uaUD`-Es-S!9 zh}EXAu3V{PSK1Hado*|q@|XC?)!k)X#;sN87j1{E5c+g2#szO3CesAC6Ack9LOXCRQe->;x^8~W0ZFUUi5K&kyk6?l@lN<_@*4RXb zCh#0D!KpFs@HYOpgD?M*dQ4@-?-2p06o&1VosuZq-k7R8Lq!n zBr*4?xCu$tpn2!^#iwQ0IChbgAa5=cAWU4yC%VBpClJ@%eD_{WEFQ4Vw1#)dw~t$& z7jcyx1ab8+q1ITM9IR3S9gvb=lbCTyAWdSJKA2pE8`P)8~M*)H(?6DpvV7(0^a1P<#`11Ov0I?QHp~ zbL@TR7$krnDo6$@aR%(oOe%_F)|;ZA!*!2CkJLh9eDDowzPq6UKR#N@GNwD+7l&Tmm>JIfRm>1$)tfWagL}D?1l;tH#wbPLxFBB;EcW(^ zn(|0R3i7+J>?m%34}Q5u!BFnl>j<17MRba9Zk&w<=$mJh#nMc+GF(bpPw%*nDG!; z8ql~t?R_G=3|lwA?sU8EyOB2Hs8|_+GK4KB#4qH1pDuG^&czUY&Ph=>f>sYEB(*!Bzw= z0I_&8efVB8xUQpzM*uI9`SR#m^RW&83tm>ZZpk7b|P-cK9L|ohkz%lh&}j0+uIqhXD^F;TzxW z;a(b<4FfWvn)OWj2IK>C4i-*9|DwJDzX7wjxDF|#c0b?y9bvr zF|D?Yx891!Gb6mijc^83*(+yMyV@(-tAg1iS)I@uQy}Gr&`|_!_cRx16CsdQ9fmH# z0FmHDkp(N$q=S8}PcJPb=EKBMP}yOGRsLN(v?fc*9WUUB9g-ae2&aUROs5U$yJh)2 z3O~G041j9-G!cQL{49p|`jljHHiy+mxhHVCe|7p+E__Loo^HhztI#ZiEK%e^9~Tv_ znE7Z$_oLTLuTkLzBvBD!L!S?J0?i@L&dfF7ZMyI}cpg{~uVh#*$c&XZtpJYzg!AOA zs1X}uR?#`E`B8*`(eA8(8TK2tol#@mV&L`}I>eEI;eOL!>|pO z1Lc|-hj@eK#_#CGKj~I{bo#qz@|Kb{i=2)Y;s?=ty!Th9zi)7`M*}GNW-FmUpw&e8 z7%iSzcy{FL#lLc>n9C||O|Qp=asiBL~pgl-`v48ZriQ*T{WxJ7);o;T5%bT=cv)l%}!(d(G=4> zBAD+|!}u0%PPdJfegjMG*75?uF}||^*c>xC>b!|Vag}M6(2Mv(1tYY>4XHkYpFmvl z!4#IshkUwqo>OX8Aj>g;ykQiy;~TP;`G{n_>1h-rYGWJ+D;6n;(8ojX5DJ)#hGKK# zbtx?0!@AK^gu?Zb_Eu*F6Cn5AdoFG|-qUS1-mS)KTr|)j0t8$~KL!RA08!5qy%99B zzRq0G?wTIS$i-1q4cvw@pvV*-gBxIzyZO)a001aSSs@o3hgDQ7^DIyk+bT zohbCC@VW?{NuDOq9>3|pS8}*=`F=aa+kas`wT!p2A$q)np^&WqRz4=qh6i4gz74-# zX};&DROEE)kl-o3wITzE=;q>A`>ww{{Y_CP{%*Jtfrq`}0^^c0WH6}l=F6q`c0iY; z0c{&qi-Mq8P@k1)Ew3NZHA3?-_^s;$Z#K%X@{c+LWNwnuAmviLd#a7)>s6}7FNJT^ zCUGJDo#9WHy+X0h{_@R8K8I-*}bM01HUcd~e zKzV?bEXTT1-*LN;`{ZI9@)vO+r~6&P?zj0BB(SGw~7&>CxPp+$47@) z(Tq_|aTr98T%M_m`Q>`j#i?aGcJwb=`ZbLPr>7J+r9eh=M4u?Fq>diNu-=5u)^}We z+=#foaAZH^h;8gK%uNdz0Gk%TmNj{%IU%)FWM9#&`4}?N?N={_9ucq|<{c{Gvt4t2 zEl!KdcWq&eTBG6su}{(f>5kLmM!kf#X`2d0OD4*k_$X6hhiaqMup#gd0VsY#PQyK- z@>qAl;aJ6FUEanOt(kRyn4d>MXxbe43WNS+aKu8rxVdx`viS}ZzB4Ez)R2S>=|J}A z!Zq-t4;kb*n?mA+^v(Y=o(?7OUXYyUOGoAh-d3FzX3jU=;Ri-#<(lv-ux)JEGWMDK zEAXga@?&+a?l}9ZQxL0a;0JD4Uwf?3(+fQ7*F;qM3j2K=V2}ztvSk_2T}aBe`DKkD z@BJk~1|KNv9NXgleOw-Xa*X3Gh@q39+uBdk2$#RMRQYAfx{QL2xq!0JI|!Z@zvDPO ze6CmgtyXRGe4r)(i@dnEW{I5XAZz;Z8J75sn9g?>Sh>hGxdSC>(<@fY{te=Q>4+DZ zNz5bRFE}1~ZY@JRJpGX}k`XEh+(u!P;t{|||D;!hyc2}@9?rSpmHp{*5zc>%KS13f zp53e)T|eZqe|@^>_6WjTjf$!)>l=6hAL()6R4umOk?~_}_dy?Ua3DnjAk=DFZ`{9# z%CgC(KzXiK2I@m(!XDc+k-ZSZm2fp-b1kkAxTprou*%$R?5U+20#Pto?$K>hS}xCV zq|6iVnQy)$%N1QA)_n(zZ!57GKI8XHtt5SP@C_7oyVBhD9=B?aj~4`?(Wv$xntY?Y z2gO8_SFntB`3AZAM$~Y^q?P(rQnkn>H|h)!UfLa@IAN!tQ8$TxX?q_)%Sr+g`KHWI zouXI|!!oYj0c783ppUiQ8Zqab$($eB{G9HYQq33(p_hZz0#kMi=TX~rRZI$kW;gu^ zk*7LI(a`D<254*g5)!>koDh~Pe0pI7YOIv!Flhs6i!{& z<=TfFG#vJaD;1e7Q||rU;2}gIv}&dj9{h6CYw%K9dJ<=Q2NC@~t!#IY4RUwTF-l&h zrEZAWCY}M>D$gi^7m~Cqu&J>^mDfz-pWy;S7E8Kv$iVrQrzE59-Q~-zS+|=-l=YI0JdwJepta*T#_PCEmDnM1 zSKM(S2g2+Kp}*7wC~AiU;hEEGlf`i!6rf|eg}{x~9K2xfFfZVlZQGicSZr348OGC0 zBU6nIxu&NO4e#EOQ?x&38Nz3V97Yl$EA?;vvp;#{X@1;#o-@1HUd8jYA^FC=u)C(G zCbnqrZIw>_WvsjMcXz=2K}jAllUR4f&Ib27)S#QVMcVGRWXOO;Ebi#FF^Jn|<2eefjI9O@hZi0kj z;;s2I3WXV0@&2vVY#f=1h4Qr_dpRWMv-!JGq$tyAZxKVCL!j;{MEbjb1UD{I-hM-+ z#Q+rhl)pR06(xlRCB~a9?h10S9zxgJ#}x|RcFWt+db=Bbs*;B%2c5KO0!oLFa2qFO zB1(t8-tUQ7cXmA1iS?VxDW-vh?AuUrx9xDk3a1Uqws4*`wssqi?w9JsWM8lVb%^{F zt75}l!YGLRcl)P%f_WT({3Ui@-A4S`!XE9D3?q`4dvpGW*jvO9(Y9XS7=x)N`!xlIN) zIR3P7H=sDZ_>FiJ*ID4BYh!ig_iS`TCn2|k;`5cMWT-HE6n7FlY_9E&yF&V@hYHg# zFY1@1r5gUn1lP8@yhoQ0W|pqdJ23nag9Rexu}0wWTm}NdbmgWub#D5kKurI$cu*jpSUc(*=F(~e=Q{QC z0ACdKHMuo1r@ss7KbCQWIyeg4k7O7$ls?w>Glkc;(=P*OULyaBp0Dtyt>bvHLBQmE z`OV7Tf^nomlmY=R zoVZvVyB-`6BiV$G^<@M|Ph3%$;mQ=S35PM3Vs?eu_MAmH@- zMK2zbGGRvcljTpxgw;Am?;h9xj22nX$v3^+DA{_mx(9#h|2}y#K5YVun3Ly#cgwTg zHwM)NWG<~H`4W8)Nz)ERHmoE~abxypijl;jku6QwQ+P^s`=Bb`Kg1>;K>Z)#gfEw` z7B^>V%=aA!vU+4>hMRnMrieIr2fKl%WF12}k=8U&(f1w)n60p@{c(4X=sfIp6%|;e zS4`}P6BCJe`CU=SbHc&oCjui=wgccYR_O8mVrj!^tvK6fte_NnKE%*2^TCM1ONxa8 zUQl$v-+QPM4hFj#w^~n1H0=?CO1A4lp|RI$<~)~Q?M^Qx+LLmFi=o_0ce<)fQxV}x z-Ax?UWtF4aP!fKwr8^s8*s@E_j38j34?6ahOpjreWmH2!@g+_n3a>9m z*dpU?D3xhEW+f}jNi!~a8@hg+H?Ftf9y;iOSlk7~q=!*kt^*A1J6dRJxaHy%rD6zP z2U>MhPqD?5W`c?NtNMJu$|JE*rl8OOGd(LhxrIPMc{zhrJMcoL2ldA3sw;5F?|gwf zEoy*<9$1FGN`3)`YW}=UkKKohaCc_y2n+dN^mwfRt)`7$X1CqJOqPMVvu1(Idn0U+ z)v;E}Q^WcQSTjs6FFN@CgdThaI)jur^fY7KscAO)Sh0biYW;(!n}&}Jr=hb2o7_(L z3-{=RB~Xxy<+U>#S$!Pm;2NF|a{mVR1paJ+oF$PN*#Qf#C%4@Zc#)x5l{TeDpo_^S znIo8B<52>jX8U*Al=v{PZfSUQ>gukx{(z4Z%R*}&&-yr&Py!nG4eTCifSM~vJJ-N# zrFv`{Q;)@0zMNBW_oM=Qf9bq8QbY_oqU_SnIf@m%Y2EESxBQ^-lOzg7ewC#U+fc=XXaq)=xO%AOZw4-! zw|PQWG$AHVwQ@S3lDF&a7+}fhMlvG9S}9e-4V%)c3*Q;hD32j;N64TPTC_DBYQhRT z*kmrD`&gIA_4vR*iy!Jxo3dZEEv4m_Vlh-$6j?9Ge4G&x$_A6sHzuocAmFvfl0O1; z;An}Kb<5%4YxaQe4bOM=Ax3wOqCvA@)04*a?tm{?tu{!pdj(jT5{NN;3b~v$cEO!` zy>}&he_4YJ0kF!QU%M#<@EYBA3}1ymn4W-0wBN9O!57s%Vrxa^9k^v_D#~gxEVF|7 zYg&l=%k<$p1^LR;R?sw&6jX~=Og>C9&H)0IJ$)X)gS4DOe`f&dH^y7^Stp3)jy=a3 zd!M)EkF#NS%jS@sERy$s6CL;aOG@QXVM(h1SV&F&jHH#B^kHMi9LY)D<$&0Y3C9hd zBXDFy?Ae-h9I+bB!&R~zytgV0m-leN7luGT6Jz4^Ry6JK{|*6JATq?A3~3 zU8NNq5h3kYJiWG72rYmobTF}TXe*f@sb*a=%Hd7VU!_2NCPtKwU*C^{T2lyHP8R;m zPVAsUqm`Srt{hN{4mP35I@ctp(rvQdII}piu@)GC$6m#S_#JO$i*R!pVRHR~Ws6|= zqzR?0N%`yD$(t3{<3-OxSNurTplRsPwAs{QMpuVJ4idda>{-(=_-*W=u`38$Jvc&; zt*OSkb8B0Cqn_9ZsXl9Y@=|V1>i{arttenNYiBbDPmY3eooXmk11z756Pr89g(Onp zAJ*F0<|Yp)=K1s94>xvYTi`{EU&qR-=8xZY8=Fpy@z~Sw0wUP-zouifXaXz^?^O#r z=%DP7Uo@WRqOdU;Bg_O%3d^E!b5J(N9p*sa8;C0Hva-=r^8hl-FRMd1t%=@sc)2g( zh=tXyMJ%Y0Z-#hwUp}-OAKwN0NkOeA!jEIgYZ8IcfISIK-QhsA4-yZQ!p$mGzcSrI z;)Cj%Mj|V3{aA7)P!i=;Z_>s_aTF1EmiuD9LY&(4ag$6)~VShLJtB~>ZM0A3y)ZF+d8|B^t^J8eQDh@ zPVrNmQQ}3UvTSnR60Jke2Cah#Y0_=+RBSObo>bWY8rM?m>A0S^%&wDU6g4sg>%mdi z+($6lW8r+$Pn}>IJq5;ff^*~kDn^3)<_1ZGHf6ktH_~bVszh2&Wvohm2weI}f@4zj z$t5s1O(wOJ&+zY?4?SH0ZnSeEsbyUz)0&^9T#$>MB5F8ko;3;(QUy}k!4 zJo+`QrhT}yUTD=Y6?x2kEz45_D9M_qdLS+xmn*`KRiq-qM;qNQen_hZkfOFn?)t}`V|nSfB1&mJL6YM_ z&jKq??E=qmoD$mr;}t_=fYl1Vd}M-E51d1-SIc@SO`viyQ zt+VSa5CLpTSrrX^Nuyvhan;tFtB};1XvcK%ooRmVqVR{TLe4S-#x9|bRCHQK8)J6}J{24i@EUg* zc1lZmv<|d+SvvW(F!)mN!vLzgkkw{WCZXTo^68~bb0EQZlFYnnsaeMx<4R<|K|H=S z)EO0%i9OH?i$Hj1^>(Z6qyAXdieeHTQxNAx>IyXui+f4Ua>u!eF$g!=HoP{;sX+9` znq5s2#0FSs!<4eIdIRf)87((g84_Dh<{=mW4z)w{GWuJP!W}%H@Ik9(DmE zpmk|eo!wrL;Ns;xUhr{`NoqM`1SrWdPT~|PwASxm3Icz_aPrSkN9UN8VxcrW@!P;O zUrOcNbUJ8`J8M8FBn=UxiGGvOsx|u#Tok?Z3fvVOYUpt3x;lu4_cwp70X>f1(dL8Y zH3fa|^JU?dSQ$BQV0jgTMx%?<8uM=+bau(216%XUzeySxS~xgWD3qDifSZPX=Idnp z_`y)??g~N;Q;fAaK$C5RUAYI7mkiUHY@B4PVGmPgVq>10r|BlLUs4%tk-F}aO@zK7a?O0xIl>%DtV=~%`yUksnU z?lqfxzvymG?oRywrYC*m4gCgQC@s};S7-cYSZ@0kMqCKM?g23b9;X1)!!*{yU@eAR zG|=w-n|o7qzjw%UBvXz1W-11+^DOsXetmBn82Up1s5LY`y;?uPW1PJ(TedcSX~eyp zp-tBoh1S1i12dc;sugRGj@!QtZ+CIq|0kDAdD>lZ&w3$3w5Ya+OdV|n+t(1lJ=DDT zO>NN>ev%&s0Dd_hFu;kPSaO;~f(Rq5U=&42LD`Lm;-=T`oT7`fTk??&QcMl=g6pk< zl&f%KN2ibMegLT$u&p5yAz0fKVM4U{5coDC;cb_&iy?hap~M1F)9jQ;x)$D3hE#Z6 z(s$4EmijoK>K1h&l++8PyeEU}L@Z2c190rEhnVm)Xo~>IvRXbh31x>5`vJhfe^~)e zmK;I~EsjopwdTNrf#7phv5$jepOZlNrL1Gnw-`&-h!k%}6gUXY4zC|9$LRAA_K!hF zvruXQG`7|_`yauP$CU*)L~u|U;cM@{Xvh(c5a0Zr;>i9vut#^n$pLtxp~{-c{P3Ur zzF|onYQDv4T4)!&V}z$HUPC1c zg#!Z!l-Z?3)eYoJ+-fz_QfFat=o$URmu#b@b#v$UkCvJ2;;WBN742GCRH2(9fT~PH z8T;L^MHyZOlSj_~hmeAEYEB!l%MpQbQ|QyLF7uK)&0y6CifalFRQ15#EiIuERBY8e z!X|9@u>lvq4>sf9;s&9>voq${AkH9)8|su%i@P!Ok6KDznwTU(_#2;e_ELFx$s$UV zmW^Nv92#y;stX!7o&%H;;nZW0bhTB{?mxt~d2di6W9Y^tIMCX9l?TvRHBMkL3d+M&a98nT_i3X@QTS3U?Iur)`S zq@otx(bqu&ip5Zq>CR0N_eW6!A>gY?4nFNuKwp%_tW*Gi=DOw|ues3UW!gq|dqm4y zeIcHE5e`!XcQK)(NFL9OmwkkN)14DF<>6nghvFN^1QI!5TujEDqHJu9RG!qkWMW^Z z*+pnV_Xw_2cu(Sx-3}w?0%H)>`J^STs#9=2@J7`qx<&Q7#`Fr8iq}|`UUn>xU8K25 zw-^*mZO8SJVIYK}AOI4N$zYem)SZKg@Vj;Gu$1N2TA!Cg%Z*KgU$TNVBYrbqCH{XK zSO!brOiB5-c(jh$dRu~)YZ!T(2-`(tMg@{^@{0E8jQ}bti!nSB-3o@|8WTaigmJHD zOz`uN(y^;Fo>6Hv`Y4&5(ABQojIr7ozC0AjuP?UE=Q%!fTlwiBc$(be!7{yU<<1pc z#mwj%XC0)?v4-I)ZDuv34U(yDj!~UgAS=4SRRiKw52w_CDLe*;P7*v+<8QiJ3J^~P z;|5VAu8Dev4ABIx6=867-nGG#%1b7at@$>qj+?Xubs-S)l@rRlO&YnBebaGJ&A~$= zf8SB(sn;r&Js;^BSo6W|EEC&ySFKJAtkcO%w)Yn+7j)@9t@47VU&5+mfrK|yC({+k zXgKjj43*1d!w>p!nQU^i5txD-zSjY#b0w=oaKT94bTE)sT-9(}L@Y z{(4mA+wIzgtMj5Cnw|2NalRAT-HojAUFiS zAD0_xa?=SuT3$Q*1VpE)*8~pzC;9&2NB(Q8ysaM+NS zpM-<>ij*s?3xV6VI}=R*jRZ?!{2C6>^IM##mBR)=;RC{$MhH{5q~U%} zV7%=&vVA+-f0c`?!paw|jKQ65^gMayq=m#X=M+`DH4mF+Ue7P4!o0LxDfja$Ds;H? zDt|fq2YHs5o+JH4CJ?*4c(aN$J!w0C%$2cMqcC};s!0ibf7-VBHA(!S+#|B^k4VTF z_fb26BUI1m(eq@m{k^wyRh(Dtm)r#a-`!2piKFLYINI(CP~}$BZw94lE*qP98)F_D z43~;Dy0$TJ?+Om8m(1oPT~HAl(hedbng%#Znl?bGc&dbYl+K8a#|(??CO~ema}73#33QcDCXEAChvV1(1lv^ceui$@m#lkqg-WV9_~-qZThfLn=1P9L`m zr_41zBN`U)aH~mbNa(AAgqWP!>-2e+!)zxW(JaZq@21Rap9N?ZF)$Ao94ByA9u_4E|Sz+tM{!zA8~?~8zkz=(3YSz&zXiTXPCL+85n_P z?Fu;O1Qok86s8VA5_#cvh4F@p*^U{7>W398wJb&?Nd{p)?OZUD}psGSVDOcMN@-= zxKO-*ywh{WITESoYIt3&)x4lzCu$Q`oY`akp)Kt;rgXB}@|Syx$RuY3Vq7iwxfXuE zGtfOFy`3fOWFS1uIW!ichGBCievQJSFu7Y$nPz|>mC)J_LszjD zyWETHW$zxH?%}%lMqjQc*?lHXqvvqtM+42Qn+@JhQu|TV=4MsRlq0{W)bwoPGf(n^ z`X^^u5r%EhP%e+dp^5xa=T-r@*2HK^6aW>4=kI!rXjOzCA5Y=mA8s7L6=Lp6zZBeLL09&IQ=WT8#^*=?A1Nth% zn8UIk%ucaiOrobRPBZUAc{YQ#a zosZi)L~hz*GXa8vsah30Ywkv}p-&urz^bk;O+x(mL8>~euc?7T@kTKe(LoWYdknLq zc{d;28AT1Q1+I&&(qhdd6}A)v@;a*{1(hfk7QjgR=?taSBUAV;I7fJqK#;+AwyCui zAP;|2iBV~#Z3}J9mJ+0O*S~I`!K+ah)F~|INOL8GtzCq*(=m zs&yb!KC5bNPtQnUJOA^Upy;hV5z(GiDbN6xzg|+K{%)=O;5P9%lQ174c}qfdCe3Z{ zo^f{HOO}5bj}+6u_XtfRlrCr1O3vJWa>?bhSwol0;1!;SPcx2U^>sF%+uPbX_Yd>M zar;SC&UMF}N;G=5!ad^@-?nzKbqlE5^WZc48Z`4~XVTBuuf&sN2Ka=NgF-pghajeu zbVR-ecH!y8RG`t&=iG!6#s&Kh{)2GIGqC0|{rt<|I#WtdUy!b$@@vEZ(N~I!=eBp6 z2`!9+mYUI&Ug#nknCDERhd5k7J9cn!Io;i;@n9Zc3$s48GaS$)tAR6&SQF zE@Y)+!MCToi#GB1QL41$vHAzIDpA)^eodW!$9M)kREeA20=jYg^C(~zuHt&KECINk z!qcPXGHrJX|B1Ctlv$81zg$G`ER#WbE$v z#4)_lden+bTskDW3-@pbU9|?hyHp zR%Dhh!FT=jiGB1M-l~`6{!5CGFUTZP%<<5)NTmu{T#tVw$mD67?wvr(MlznfUzHJQOU>H?tGhTk6os;UY&UnB^G`rv)I-;XYHw^ zRaNB`xm~7#)N(qbHyY=xJr_q;Ajj})X8TbWlLaA0L$lpBf4)fmH$e?c(J}zVdn0kw zfJgSwekOg!)QuvLHOE;XY-Uq3RkW~@XF$-Z6+ZuYkMK}AB1w?i|o^mXPKd-meuFFQxw}3#C>E!0@Revt|pMgy3x%mpYf*w%k zEJ&8OZR-#BqUq)IjAdw=F;#SFX81Vup11UCAV^&zxlQ73-C=l*a0xoEDWEGf62iog zix0*)F(+og{#15vw7oyTU^rXVn3jnH*okyXt7oLf!vRlF> z;=+LG-MJnx#q;LXD_G7RTn?CJX>h?u-#NLAXS+Pj7wfx=oTOQ`X-b?*##rh8`F(ST zuWznYE^uLZJDTXN)e%%}2%W_G&4+?@K161)Nzil;;p@(Cl`aAB<-7Z|dhSN1?iZ8J z%)_K$hSgBoCy3yl^kv|p87h5ovbkgsPV(1&LwY%U{eDds<@@s0j(%G@Z_pa+oPMZ0eq2h7it9`6`s zH30TrJ7FtzQZY|$zbV{YHu`YeEea%-Wy@LZkOi-&HH(>8=n`))PYG6eiAt(hu-?8s zeRbHg9QXX~QilZ!oZk&{oj`RR82-zQ173XrBG8Xyxu77dHby~jtb5a@c!&Mq$sq31 zkEkOtg2RBj!V|S~4kUjBu`f{ZX1Dg3mjwDFTU?DpFietM=a-2fm0<6|JJJ ziqq9yuTcXh?)i{-5|${8-@~u6Hv<(YTgiCm{x_~ooH%vVr>PXd#c;5tppoOiaF z8ZnHv78%*bZ4jGb8}P>2DNsA~TwVddSAlpyUJNP?4k&oCj}oMIIqR z&}@AbV%{DD;7o*&)$|=+eOJ*E8MmE;<51?iC14(@uG9I=Q{2;m)1iE#o=Zpgg2{)?F{? z;Gp7W7&G!+1K>^(beb$`<2EFiBr>Gf*7UU)4O89(a0h+PLD{BGL|27nh!ZG$>l)=} zV*Lst={1oH6fliRfmOC2y;H!%7(hR=Cs5RR&Sd&MH(_3g6XLF-&GeYwjFIqJzenZQ z+f!uuMC{!Lj$K8^?np#f(Aojd=b)qn6R8`dg;qDO;*cODo>DZ4SEQfRDF{_vU#pp_ z-{jG%+MV9*zE*)k!o+@V_1$X*Dn-)+;z0Wx^`m3U4{$E=4c?rVbCR9)ZiD8${C#=8 zk#DRAroJ8{Lz1gsvpanL##B`re0IFuXxZOlOX{ADW5c$^+)gkLqIE7C3PCg!Ow>*Y zAF3R99By}cMt!+Vsnk(lxS%&JMNd?sz6*~kTR^%%asyg13VXSoZS)iF2^6rO_R359US zVisp&D7$7rRIyw{^u1SkYP`E!5r3Zc1zH14=`yt{^lkBHdwK=R9wJN=?{N>(7oZp? zLV3cwHeO+4CE&qJ49@eBp9j;PbFCoMTiG%s=pprdyy5o*qJ^ByxqHmD_DDfzJ4g*T zC&!?0i&!A9z(C{&vg`Zje#Xi>P><|JX;`KG&eZxt$b;qKqURPd0noQZrS>G0UzA2L+wIz3ciflj=O; z+HEV&I9$SX!+9KjpeEyTS*>I&)uC@^-u=uo9xC|Krw<&(uCsr-z`eU;VRh!-kgT&e zzYB38t{|0?dN2vW-H1eb(9q^m@##vR-cw{}Wm~!_>1HUQLa45Uij;zz!DhO0CDg6; zMZO-)L2Y>!ViTs`m!f9xIJ=XqNS5GovXcX}%(IhBqcQJzi;XHbX@qvArYog3@7B zNoz>kREgPVe{{>!h2&R|5NoqaqBFriX}mu|KKFfaN;4xEHy*VN)xCa`$3>6m)s4KT zYtc#2>?V_xSW>!DIqp^%);WoKY+gWr1e_V-&_0Q}Sbw}-}V{pUZc z%%30ko;;Z`*5N_r;tJUEH9^s9xXy1MpYAYPBIDN?Z2r^pr%#^CA=(@M{HOhA-#`2T zT~**e*#F`1@&5nX|LG%zl+-xynlYL|1_!4n7pKt5KD_n}kt$b8;TJiI`p(c0L8Q)z zd#X+UDj6PW{b{vd0>oGnvP>;(z;V^fV?U42V$C7dc#5W36|U*w*uor8I1*6T|IwTgDkE#131J^rac_CHK~ooL z%C0eCaxV!`0qntcg;EB5S30?6E2O;~m__0@v8aEkwIv zJNz-Fbk{YtBq|b?A(8Y8V&E{+c#7`j)E}A;*@QtKK#5R5oSM?Y`ChLWRZj-j4e1AO z_?hOJK0kbv3+6v>(S0|e-+*1Me_9hU{8i0cW+8ND9#SW1Xw|7Q15D6+s*Mi>(>SFwqRZkbH_WY^9=Xc1sb(hQ+- zM-u5HbgKHjmtWE#b#QE+#}GrmI$bkLZ-d~+h3wPC4SeZAl74|1%$%&7Kkt1@IuG*p zd!%lD{~l-NXxGeJa;F#LKUWQwUO>g>Gp|0_+j|=0AB1{$pYGh6S(!|4*3j<|(osj@ zr!P-0**9jx@#}Gef$GCz5w;_MSG3TC@>~GNJaqiCn%STnKpw|y3?OeAYyFAz{Hm*4 zN_tOE9gINLp~xM?zp*#`lvAeYcT)hwa97#^P5+qEY%>_RZ2!?Rv2f)Zt{i{%)_u<- z)sNqM-2Z`^t7H8l{ZLWL@cYKjjLj!P$cXWo8%gJ?MTCi#YpDG9#0m=Q66|}g6FfmX zjR$mV_Hj&NOEV?=VlU_!m-VsY?9$0l^hKPx;d$Ja&{X{LRZtU?9Rd6e3<&%pU0=it zx{av$_k_|G@z?e^FpcoB*aujbz3%8=g>sf8|_P71Xw^as6h@@+H1_v1F<%+H^pS zTa#>Cc{zXNKZNx#FHjo^bZuDtgWsQyIb6_@Bcki114h9GSR23R#$?}E(lPpYGQ|$7 zzyVRPxC#VvHSAQQC2&0jYtmEmoll4>;3#{Dh@C3=x60rM2irOk>Gi^;V%$r%X1#Q7 zG};z&5Olrnu068wQyhjPe(t(>IY01h(&!W}#$eEyHqNJ?{1MHSahhLH$phJMJqP*est9LFXkwHiLFUKSPXTf)wZRLNYO1TF z`}c>&K9pJNN(O=m@>j3s`DQ*HZkt=t$XL=Jq->7+_Ynns5*v=Vl{}iau@PPV7&{B` z7Qa%{72F;sFzCZnT z1ddYN=u4FTIIhqu<^k)|Z&6Q?rVygYc~)ZA^&1Y&KIWbGbMHr*5}U6VcqcDDImEGx zmJU$Iq-8t4p1u7lK`7N%k{lE#aJMAg=!A-P?+^11Jnw?T_>UGe0AT!gt`osN*M|G_ zYy5VNuF6_TdYDt(=V8uxjk_Q^#Ec00k_rL{%WAc${XD>Z`l$vJpe<>tA?DixGi=_9 z^RDWbA~}W_26FndN$m(dCYIXV@~abTKF(YJ(@z{Re38!d`d8$n<<(eV?^Xi~G@#=* znrFEKQ}5^9@U8&-N7Mp*ys880t1+tU)>kTP0zXs6DFC5l;;HjCHlp4ijDIW*P-COo^UOn| zKXh|oH9ia0kKRtQ2E6?X)ZaK6+|kOym_HADys2GG0%_~J@s&ge0cX+PN=s7Eawqq) zlY7lM&wj4)lFWodBAXj}?PHm_DAydsamzJp#p>1OlvD=Ps#8oLC(jrSlCPehx0l_X z(I+&BUD9?l;T~HMm!QG610QgqB;>ABu@tzr>v^kw`~nV>ZeT^sI=v2e@b&0c99*?E z3TNK(^Y|%Ss0&&3s2i9Sle1B8!#jY4)jn8bI=$u!#*d#Lv~Q?o{)Nd;1NM;>18O>H zh2%XmMH9Ev895$4iXX2qXq?`~?IeVXD(cE;iXeDuj>?nJ7_lCwK9k!R;^C) zMW5izn&%_2lbFIf`h(BY?WJO$ORN{6t$-duJ8gYll$j)}KZ^fHOHgp;g^x%1V_<(m zVQmI<#^y1x-Za~6hb#Cho~N6)eNH{7!Ped7XpPQ8j$HtkW#gNT_uDe$d%C<_STtNR z9I`dJ$;wjzYZ-llD*d?-^FHGR%sl&&6QspC&-^+4D9vN0FEBi@ z0ksJm)?9q06hEqr6(E7rhTN5T3}b}GuAAA<+k*OWL)JNo4z!H1WPO0pVDhNlg>62H zA1kmh_~p}S)h5HQ>o%41{n_f>{914Cye#Mtb*7kZgBeWrWs)1;FNMY1Gqcs(*WO(s zLs(-T4CcWdrn!p7oZvpB1kJhWq8_Z`l+5!_0Q+ub;jwxLtSr;r!e_p=t%FDo?F zs7JahFA(r7zJhLn_fW?*bR0gzIF&YOWlA^N*)Ojs-}7=-GK;xy%q$~)?D{6{K&HV;TB+80)~gL$!u;035X39GGLNCzNl((GcH z`e$l^&=j+6?qh`n)8*V~a|WR+xYi+c@7y`tnU>|l~zx{PuvUA19zfqO5q}ZudMULqHzi&>rWn|)T(R^A} zMxDb6u<9dqL?+WTXyDzWU?DGRn=yJZ7s`cz#Ss#GYW!mdnI9=DXWk^B^x_=a2Hj(w z_yPB>YL4+x<=7qbI1CUQn@BA4%e_Nw+mvk=!r3cp(ci|?%DcROA6BDd7uUU2Q2Gf< zU~+P5=K9w2k8Nj}p>Q%Bulxu?dWc{Yc9o zZjHcRrxb|De4*4vz8HMA_6)0e3KkKN>&r$r>7`wckz;%HvS?OTahS}j9FI3QZGYF~zC&=xzD$5_$IL#Q zw@>z;?S~<(9SP>LZ6@57rf%&88vvL4D=$Du_*pPGR3`t2Em9H+o)V{?V#WSY5suwd zF>mI*=u7%D^gDEi8xbMu*h3re4q}#iQP7_4R~IZRr2Xl>g``dgg%@=ieFYTj>%8UY z1_sGj;ff~9W6C-SmK5~vF%$yNW41Z1jS`2f9w?Gu*b*IS;p)@#aUrSwTl)qlHGqKy zVA7B53_h6jKKsR5_TI6U=jkTo;u#*jMm=eDRC-Dl@3_lTQj-b(^CUT?4#^B?YQbP4 z=F0Z=B zen|g6Tlkpe(&3UwjWyA7ER=Gzyr$l@iROg-E;dsu@lcSR z@$P=^0jI4T-iSzaa}K;XW`t|j>+mkIlWT@$bCdA+CAW380I|6?g|mQV7?O{652jA? zCDs5BrrSe_qpKVIBZUJwE^VHs5R#D9&Be{N>n_T=Bu2x|k1jWdK0{ZsRi(P!DDfrI z&4@W)SUgb*aYg~CUiEt-%dg0gx?#3kkqs@<Zz=$-0Vz&-=b$(1y}6zybr@dd0W zBozU8M4J@U!;TcISr!OoD;dEzX3KCQU{Ta@(|2t!<_T^j3qZVrpLz56PO-2gAR^j+ zP|YwpbbEnpYov01nLc>w`V)1G6=j$_^ieWN!CD{5j?kMeHQt+^(XZW~c~kVS`?iTCe^o`TQGEP^m(lPY7;+TchX zHkdhwYM1BeZKdTQVBx5Q4#W<19PIrmZTz6NG+d}}#tdMfbi2;fE5nI=Q80sLr_UeS z8B%s^?(}QeR4t7uV1_$Fvz33s@@$UM<3w1G1eL~;LlJBB%jCvX0jr%s8g_KZ@dLP4 zshn1N)($@0PoEkLN7&$oop}Jtf-lt*G#xe|Ix}0z{6}5z;bm1X4xS4O#43<)q&M?W9Xlo!~5%9N+8y_ zdPfN0)8$!T1-s9iy{H9hFJs>;q~Qkvs}>pjx$i=FIP^*I&%g!}_3nXv4)>OHMjA<~ z#(r?g?tBkW&ZFtVZ&JQbA1Yr`{IO7Vdaz$Su2tVEIZXtE+c;!Z>{nM!>usEPm!hvi0@^q7!4wvWn*;^JkOOwDTz913w#rO?|4c2^r8IEy zSD{CM)o`CQ%>>E9=SO-_$3gIfJdfqQToB zI`b+I*I8Ndw0ZB}bqMqY>)ciwp7*wyyM1=qX$VCCjj5uO!1iNbY<95jxa3p630<(& zLLYwS`LHpKW0W2Kq|yT4&9D6nOj5f97kNev_J;S#UOjj;J%?TMip53|`BQZTG*s76 zq2lej2in8+Teiic^PsouF7}b?aP76*fL)~eN>x*Z4{Cz_&Rb>X9eu0LK8U%(s75p; z0jqlxI{vt7d?aXKI#GGpZaNqk0;CCL{*u1B0M~AgowU^4LFVAEQgLYAqYW9DbYy2X zUb7J)``)H5QHiKoWT|mCw^7wDNTXa>LkMR&y|Qm@)$n3QaGq|r5y>rvIj9lHqY86s z>xA71i0xY%#uq&N{S;Mwqs%vO*Zp>;Zmi(qo)pq365w`tDsW>rCv%R zZUn!S%@;=-|2Dvm0r8y)72j1HG1ie51zOcN>Akh8j!65jh$AfbaowO-{#NZ`#=qRQ zqnY3c@|i?dB^qiLoZSyVR<^PBK_iOo*ZSkbhu=Q_YrL2H|Ml>vzs>;2s-Ko^C+in| z3k$35_-_aZFMKtsirRT+OAn0)YM9#k5^X0hX~zvto4~_Xm@eH1I-`IcpTXnUEVpT8 zP(FLTpwCbDS=d7S81sGXhfjWd{QQTX5B6>|%HyiaF0j<7wN&3^+vwbPmopn!iLIcl zIX1N%s)J&_j5zzNv#hAQ67Ip;l;O6?R&1@l-2gFY`Ow%`8Z7}I;Ge#slxy5u)_${t z{w;)y48W*N@wZOM4G*VoK5xL?G8*v#SsjPn|2?h-*~PBr5$p_#*{_zxghSz0_!OOq z0CZt!xtT?=ETz*bwCc$H>{7ih1j#Wsxv2Bz!ktIXGa1=q{DLnH`h{#C>`97LADk)>lV6Kx+jn>d`h;4?QWtBYDknIyXL`5( zk0kQ&2nGd``m4dvwLdPf@FAj<=L8k|#0#OddZk{LH>gT!y9WW6Z<7i^z2}!#>%N>j zkaNW;aYdgkeiB{In#iI?)M!|HD3~3qb-h(J$H_1&W8LF-2T(jL>u6%JzsEPg zcD;7u0y7oS8Srdl!dpX3aFWax(*vGeE;KvJZ`GPl!q{+x-0@*W85WXCbaCGq|9D15 z30La`^Z`BMLN6+$F#^$BPzgW2%9neXQa@$?M9km+$WM zRBO8*c8<7PUKr)afoX)46D#cAF3!&5_vYvM7W5~b?h$hK`0f}77!rCWbRga%5^Nt` z8h|PxK3e=a)w2*`G^P0=m((}BUf%f-VK4I`v&!xgLv^HFT0~rt2V(z7&NpcAI8k1~ z9Oa%9g6#)@z9+J{^yUYZra0~shOXs9)M2vof)Rq{80p+w5cVZhL&iUS^?G^Zusdj! zd&MGohX!-8SC^byxN`v%60!L0&Zz!Tbt#}T9x2?AIKpB8%Bv-Z^rK%hvfIZMIKLIG>O3WK22_yBh!=e5TH&YbxV0m_O+OFqWiXx=! zzr!67O-j8!hGp`R6z}gz=W~2W4I&puPCFHdo}y#{#1&j|dz=&aVi?D&M@HD|lQ9@U zD7{ZQEl(G1(6KT=zLZsxGVUz3Xhmr~p$ioCMTtDUAF|wj(=&F(y=Ws3De^98ii`;D zU%kKc&Hyiubq!hOTUt608pguKGDUnm+WOz=fWu1ZUvRc$MpN=&!iIF35n;jrgda71 z2#e5=8i;-Tx>J7kt(t5tJyVg+%Hk$Cg^`J}ep0VurRg%dw6!=-?*)?hWtTe~$-T#Mrevld zhny~7jtY=;w0CD%TBNX}KvreM)#aiu!Q5v@RZd{7UV4sl0J2$dodob)?M9Qf79ba{ zCdZjs5Zwb8Nh-_fBbBgQcTMCyrl3(I(jQ=W29}n@ytYFStWU2gzS@>dud_B=3ol)p zd=|18Sw=i`ET-x}q6!X7ebX6I>@}0`9bfeYf9Y7afHm$wlM1zn%c&R_2xhbKm2S5OIP&rPp&9zDtTV(@&%sy0=V^dt+~s;y zqKj8-Nz|Q*fro5CuaSU_4ij31<7>iRyj~nBgZWAqD};K*+BSSMQzBjyv>l3v&`P*` z6$Y%Z^)5%bW@$2~PM6(8X`!^$Y$#A5e8?oADLsCN&m#JlEQJJ^#Sw`9<=%spU&DZT zGF3aM|5!C4L(lEzpsYFwOhY}GGy~VD-)MIA+%htpXcl4Ms8ZMtiKF2fyGk)0Clm6F zIZ%52;9)WooCf5+O4=1Y>4=SkSSIHamDvJ9oRFYQNe7P@hrM5^oKvtQ3WlhBTQS?% zyMu-u2~cS{+%l;dC))^M9kmkhs($OyNWAB&Mp%!KZN<2)srXf7xt7npo>n(v_aDv$ zw3tU%OffJ1GHp8aZgCGxHZlP{ z_}Ewo(p~)F0Ah01u=Y_06;lmsFu7Iy@mK3?7+1Xm*u@danG)xVzov=ur1!}K)Ja)< zZa4J8n~my#tb?}`_jA?~l>=HW{0hN%Sb(l|#cuBOluy^*-GZIYxgwnR7K^SK9&=aR ziZsuVs^(DIaSif*PeOWGgji)4Qg2F}!~_pJE)~R43D7$gPy+(x-5RWl)%nLoKa4;w ze0_;(27Z}cw!FHZ>di0&W`FQ9)|@hn#JCSCfuW(@F(&J zOy-_K`cN-I(_E^SE9nLZQ#6`k#$Dh(0->=yZ=^Jzu*Yihq>h>SggCtz_x|c#pH_)_ zIF|6F`>vjC;J z8P0K)sIi3|^;-va{a_0YO!)&33$R4 zr`9HCAH+&4Obp{_wYWbq($ zI&eOCrtid|`Z02D>h3DTg+sTgLwQ$o@r|GUDTbSbUb+SNgBDQ0(SoX#d0VHB+E6a4N=A4Icm*fuDD@Iv3*yKzBYWFF?nG9&n_}HOskj^OJE&!fCEy zF?xk}^`ikwfuT!g*3sJpxfbaSqDnl1c-yt#B8|2%`>7*f*OvAN4I#yPn94r5 zb|@RrDY0EN8Lp+j#3yYBCQYTDXV^=lyb|GPm{>!Toi!aI!XZ0Ad~JjNHlo|g+7Fl& z4*mu%rCPqeIcL9RN+{!{=R`*&QkTa~eUd>H> zZy=ER3+L0JqpGU7lzM7qDCs~dMm@a>`ST#t$L9)(6pc=nZhUz9;CH9jBv)acXC2fK ztV(~3JbxR`d9}QJ)!iaW81Gg`Y`$I`Ry>8+!tY;}X`Bml?_w(*9cnE|^N}#cmfc#N z3Jj2sNGy64=EWxG@kA;lO!dT3J}OGhKWC&DCM8P5;$G4+;ACy&%4j;wW*09;cT)lL z%e}|w0*%`Z$E_Afoc1mcx}Si;sI@n*7a!O}QP7LM9d4r-r}Du+tGqP7zSyU{_Cm@; z|E$Z;2xnm}4KW}#N>bb3$*Sz2L8rsc2fB+BB8gT=?slxgvJhe^ueehuDkLMTGf8}O zbOIdPo1p1-HBJwP4^n3C@#gTE(7pZgzX;Tmhp7tu64jddMbH^r+Sb797olzV36iJqu+7dZ1~K|&5-kva!_r{RoO-D(3&!Aqc*ra)e2(<#3Lc7UJwsnQ9^0l z?nH|N;L)0GtK(#oC_ZMA!UW`Z`1dIdW!jx{$TkoHv5`?g<0=0D9Rf;j8mhOrpH)}J zt9kTa-@m_NJ@e`)=dqd;f&`!8H9&4%SFZY&JEA9g1c1&(P~KR~d@KR+yWEB8ug3MS zbzcmwJS>HcRDpzk8v6xP@(IjPZ;wpz9ZuD?_OrKN!5pf1M18Fr3W^z93+5DtbY0wA)BuBH z*Wg%0ny#*4Xd+6ORN3)3oLW#S%+ar6PRph$=|DXFL=VNB%Rm*nAPYF#vs4KKp)v^o z?f93u(rK5#Z_eG=AEK$Q5Uxtbw4%Qeh@X#QDU1((B@miu^^=J+Blt6S$*sqdv1}o? zBw(XJ7kATtHg}ad*X5!iZlLGUzRDpJMHHYhG~hSYS9VF_C;@n_o0fxyG#75MC-XqRvO!+jwe~?-FRG* z>BWV=h0%m3>2j2;`oyH?mV17_({m1viUn2>t+oY24&_sCYCZ_s*IJY29B99;PgzE` zwfS?zimVq&Dp6cHFNxmFpU}A##E-RPgffNQG(IuMb-)_Zy~@?zi5ygS(iSRz$$6dS znXCprg$%s!m^f(voCP!kC#AU~Sx4L{aTv1wc*_NOLF1j?#_w*!&SC!`W{9(xL6jZ2 z>U-+bqlqNF z5@YTXEQe@k^wOcD6uu~W729gZC_|!Y@HQoA#Y8$UNpM^TEVt+`OJk<7m7SY&i&3HP zvR3UE*=7UFxG&5gEw|>E>-4boN7$oC4KrzUS?`bu+$BUC`^;1sSIP| z&z7g=K~ISy@Qj;7?6>bn+~Q~eL4%4P`ZVBPEB`AZrYrDE=Ry-Urn4Z~*|45f{_T(i z(19ksEkf^Z@y!ybNP8KV$j9mJEWGG)+=WEti$ML@^g@_Z(*3i)!<(x&lF_?`!%?sL z!qh`v8?}S+SaHchQ00678ki&52^L$xSvH)) zQ;7^UQJd7Vl0kt&b~6z%A4Z=?Ao?5t4th}3x2sdTKSczf&dd9^;mo>N`+aGkq0`=N z|9Y}C>#nIY0s5cDAh%Ag7N-4tk%xe8c;OU1(2t9*;44;VruuI1I&bJZxch?YjT8-3 zu0M=OY7t#pI%{`BroQraeTPS(6UFw)S7#_@(Y*y8?CRa} zyn-6)*rT%|uzqSPP!cj=q~z39njGU7U`|VNx0V0gAx(~^z>XfoyIsQky$~J@idfG( z=A{v);x;b&)qcP`1b5&${%X|+|4YG3BGG`K@UdO4b2rX0|;e@o@vymfXR224ior$&dB!X)_j zJ<=5a_YjqnZ_it1*Ks1-*=}D1h3XCy2rF%KJVO*^-8Zs!GLRIUVr{Rhg8_gxo+xY zeBo&s1}8|zuudO;iX(y45C`!z%Hk107k-GklPtCh_o)V;CXOT#@$fhj{H0lh6g(44 z(ZBBxLz)E)dBeMNc1G*(W4{Ffb9dCz!xOXI+a(nrh!YE+qIGmT{_oS{Q)K9$9onKF zh@Cf=7m1Q*C{O;y-tpaY2k%Z#9PL7VjhI%Y*u(mMz$4?)ZezZ?=<@}YqpQNDUkS^x z%|b^|qp-(O-wTjnUh0D5zw!=mA`5N+LPVSu5lB=3J;W=fWd{Ipr z6QWDWKjKR-#R#N7f?PTx(uYtoEo9LRTXDV-rEYum3~$#1@)qc`XK=#)dh@ot3+rBp zG=sl*aMV;f`Zs$t= zk^0GbyV^yy`^39mF`ko?=Z+cy*_Zh}i(b6MAvSipw13Ct#popdOgLm&81`bFRG4Rj72!liEtr&%F8*Y~rI$wbN$1ItEVD8>TX}HgNBQFa$KJa&$8n@- zntffr0_q#yh?=gRp6;DeZIyusiI^q{4G9%jdzS=~6iPS^K#*en`hA}F^T^2X%mhQE z#@5zs+qy&mnGx>(`FB2o$mx2`*~WX8V~OEd51nuKNp!TzK`yv^)buL}M#QxM;_!ln zb4SNbjJL98JiJ)m$GU_8HEXXFxfX(0=fuCk^ZGqRKQy@E5UV+dNtK8BHO{32O~*&i z^#H_>?QN4AAZ(%#n0U&h)RR-{hSrlJ0@{asJV+M{Y3NjKe*O)*X-dkXof0p0{7gUH zhcp!bj7TK&sEtw*(fIZB;_s-eBLZ!O;Nyfc)V#!6D11OJ{6-r(^&s>dyJd!7pCnY=p|Njnn9zdC>-}+@8FsO1_?D}U z!y}sbqzwqP1Hh+k9CEcUy?HvQnJ!kP-;wULP6|pe(K#FJR?1Vc;w$mQcUT@+=Jqgh z4`w^%p7mJ9{qh9Zz^rD>6*vxyl8w@dsmE4vnxP6S!JUCCb=B@k`ZHW$7Rq#ueQ=Av zegHRFgRwL=*OIF!aB!4Y>Ox$eAob?6p(qg=2M?_47v55RHg$l=&@KW&%c4d+LWPjfr{|73`$ek~QSbb>< zuJ*+8${kzvWTZDXT0VKZ*6*w7%l~w%ZHR zSlEWw;*<_47(sM1Tzu@uQL20SmbG;>f#ZcaxuqQL*ldM{H8=K;xb4`i%r{w^aBdu_ z(HNP*dv@ttq7V3ubG-B7C#y}5!1qD*`&Sqf^R9DEP)`}&G(0*OTW}%i+z=3pw`7Xn zjq3NonNW*lHrNi{y}xVN%0NALEB5L5zpdX?2LN5m?Z?n;-)8O~#xkJuH5p2gPo~h* zn${lR!{Yp`A*2{BsQ*RLpOmsyQE!txkTG2gyY{VX)7m5~?y(K7eJKT|44OY$m+P8jy`c|9m-~JgUSa z`ac7jCp=&5fak-6g%#N3`;&y0H+1dgPpIR@KJ`n#D@`4-iAn=kup-egERJDc!7)(y zeD9#s^Y+0$H3%Sti51qk0K#6wnkr{+Z~Wc=lg_3kz8aX5<8B&wqy9BG>Yyk=@W zF5mCoL+&^YBtD_VuYa#&A}HIi4>|Per&5g88iI@<&dsGcLgNa-0uA2kvMmyfW%%f` z-T6ZZT&~}XizwIJiVdU1f_pnm%}u#ClTp50AXB-+_m`)XMH2O`;tKYlE}J|a)Mu>q z@tk+Dh8eUusFvW6%Gutt<|5t8R_5{MnTNx_l~v4=Atdv=9U;yQ6RLT8i=4FIyt}ML z?hk5iF$qav#o2--e!CnS(1XvE%8%r5DzI4!=o>V34Q=tc#o^O8)YAj=s;?vMtu&I; zySOzD-`ZO>iIR&>_^j6^<%zWH;jSk;a5{`YoG$+1n#sk>a%9u3gs2ZrQbdmGL{l75 z7^6kxG##SX34tA)c!?EQ6Qika1?6NW{*>GROsMONj;G+E!|$;MBu-tW>wNbcsJ_q@ z1A{#Qxp|kqn+g0B_aHlUXHs;kP8QeiVF^a$lb#-eA6Lp9uc86;fr%nSiuT^COho}s zc-Rm@q(7R)?*-38{NF-f848S2_BA!lR|1)ae3uvZ(TC^*j^wdJA8JOglG)p%zB(T0 zE|Q-~ixJ5!T)V=iyWTtNs7TT1SjoPOa}o4i9U!=0nVB3qx3F3|ceEnc-}R{bGE>PK z3Q1J9RdO0KTFsmTO(A#t3Qi&m+13&I33#S#t=U`R98pnz9~=cYz4Wwaf_^mh&B}$Z zdQ~YJY-<_D^Uk@O8j`QgD+Qb(5zmlOmL=;@u5f7usXI&O5SvnE1=k6_MLbL>r@mxU z&~*>)v-yY4N*6WF1(TYSp%&Nc(BqKJfx!=i3}i6h6oQYyxg!`6sj>b7mFDJqDny%++v2QMrZ05%7!cE>p4xRu5b-b&sU z%?S8fyDy=S>rp`k3k-LgTi>}Gw>BCMfGYw#xw>hFu|k+mpscmhzy=j9H2RcC02GV_ zVSndgwEOR!{U-_oM$Uiy^H&i)gGU|-ML-GIyxrO&ZBt#FftL5aBN~Kf=w$izy%=>N zAXxY@y}bUk7HVFZ2*&(JY#5pCy0%;u0@K|8!T%l3Q9N4=9{&8<3iWdkbbeJNMTKDxE! zG=oKEJ3a6naHrKrwC_e<9qy$K3gjACl`A~7LsquRfqv`p3L1CU6wV;;+?4$xz--k3 z70OK@JZO_Kqb?5(k*eJwZWS&jP8h5BrYz&2Tmy(AGTwdryOc`Bh}g}j>va^wp*onl zUXvAXeun=HZLeJqe^~rO-Qk!fVc!zJVLLjQ+A&psPLFG?ll#T_-t-}FT4B7}CX}Qju8nuUy)dz}+u#!} z4bDzDfy1PQ2&Av4-**`wAoU6CIqo!l_7^!>MVCjO{us@V2T2N7i^el`{PQJQUmd^y z4q>^DtRB(AmO*bP!E>w2Y(m?=six#_>6D69Pb?fL@2VUbo$z4ZdNW6_Y^;yID*!Q7 z8Fw|!`%Vo$6vWlwC$pqC(ntnxrXs2;6aOz;;uUZVyNs2^-l3A{w>n5H85n!|0EVMe z(Ti_~Y?W?nI(}9yg^=deJ|GA=+(WleWhp7@^=+3VPp%wB{>{&=*apS}pw~*G#%!VW z@Y334N8S%TDv4|3S6n^QdXGL4>}8`O#oOh@T`3<*+&ON)iZyl+ZB??-^hky9=G0ai zvy9)%-?@t4YBcRaHMfgCS6l!rYbX%F1#H#OVTl`lcGllWCwKG;RjW%{dR8X{j3n8b zOK_LQulBxhHW#!N z#cd}U9bT>C*6VY>{9Yq5$Auc88$ZT7Dyd7P+d#Y~kshFpYc-;{t@8}N(U%*Jf@JEG zZM_?@tl)rE_41otle_38ZoR|+MN|%ioF7+F`~Y8h{0Z)L+W)WVB6mUfuA{WH{?yc% zfL|pE+n~4!qS0iy^FUy}$|dRv#aA@x1NI&)ORA0#l{;DE{&$^Z9uJ83f-GOcJtn)t zuTj#GX#*eF=D+`((}8_y2E)Hfo7FlEIh(ksqAyhjdd!?nMJfo38_Yfu+aSKJ3ii`5 zui|i!#H3$CKuf}RiYvP~Vot!Ru(&G^M^DvsN0G!JT7+KsS`vvX`XL`n} zB}YdLf*nK`FoQ{_evg6p9bY9Lz%&h-M#?M!s#$HGxpUx|SYGA^DFF zt(q*?;L1}}IQ+xG^M_A9AvHctoco!?J z-9gGs7CxqOReM*`#63tSDqj403AX2N-Q#KMuEPAy=LFh!arsx&ma_p^OiD7f_O})j ztbu*L99~@lke^CowRFQx@-}Sm)L_YG1HRUuL1)H14QN+}^pmi%V|d${+zlT$(V=s# zeQdgoMh?*q9$g+g2J8o#{o#mpdvTxptp4%qj(t3aPcj^Yjv-1$#Jk)RYph9>a(Z-z zmFiGae#Tisr4YJaOa^>ZknuMB-=w)+HK6;Hj=?oe20u09JiJ`EW58X>O`W89Gf$xI zzR71MJGclh#QHxs20s8|c?@`NKF(WWAkitJ3^9X343FUdT6<81a-qx>g;l?=ijLY- z9yKaKS+dxO>%zsyb>Y?9E@_QgURvxhPi@@}fiA4xV(#%DmE1}c7_`yn zM4pBY0Mg+olfT{Q#w*0on=-QyVV*eo0WUW4gj{9e{NeN;cu=Te^si zP1)1Vdj$n{$EA+`PX{EuDQ;7d|TVG77_?BI>% z>X=#Z_dKJF%KPLXWSBU=_dN9a#U9A?t_i`x46y^yFIwAB@)dutqaIvXsHYZO?2p63W|_-{wib&M z3sn){F|?!bj!ce=8-Ntfa*4h_eXIWiBT-`yUA#PiSs#$EO9(GvgcaexcV@hYPuaSQDcuyWUc*nCusf zH&ov9R+=Qg!6m+^93s zD>ziClEV$M%!U0<6Eg2Zfo|NPG)RvUO;`~D-<$`!Pv5U*PM_DRI#^lRWMPtl+h$zd>nkb8an+T? zPX(sk1M@UJvMtY!1>V_EM>Zi&N+RKd8l@#r(B;yg??!i`1{ol_Jf8>pdCcZzdbvo- z05KQa6h^j589U4J_b7YQMDog;VT+VOp&ktnIawfA`MRh~ruAjG4rZ6CfbS3rGcD!z zrYdE!L-3~UaUNJcjgk8r{%x&+b*yGjsI`O@clLwqCJB{ql&3?D4?4Jo3@WcpZ$v|# zx6fU)|L7Eh6Nig3X{GIl-y}Cz4)lf=+Tq9I?#$I@vAFfM@h6>W31{i(FT<)>{~d6E zsL84E#kC2O#Z)YWKi4R`qGv)!(;N23P!dO>y}mbDKn4?$CQMc70*);ee?(C^^q5Sz4ac~d_d)> zZsYnC15O&0sHrKFm)dcb-%u*P`2!3c=Zd9!pKw?2!V26J!`sH6zKY9CtwouXfNOHx z=V}c*SYu_A(iQjOqgJQxaAVrSCrwjwwQ2*LYz5rAf`mH-5iAi6Bfb{>T=6)rxGCj9 z$k*bg^H}{nj#!0&qi%6~oeFnS@uQf%U08)yc;Bf;@j4lktzz8e^`8L6%F|ARpoq|A z->!d!=%{qZw=?(NZaIcW-(tyN?a2Xf93}{m z-22!S+V}J1Hy_;+K^|x>7B1<%1IQh~(3UeTAbmUXhi)?ctgc@mG_0%&Lx=$xpVr=`{9Z zf@}ARhS>JZI_O=JP9uo+fLBe}rDb4=yVGjuBs!y56Q`Yb+q>vdrYvoV*|^-8ZMOH4 zYDxtfk$HtcsBN9>SBaX{Pba}0jjYw^hE*^3#+qbte^`$lt`d8zX&fd8fF`eh?CYmY zfL_b10m3b2x^Z_g!$Z8#!^DC`)b$Kg{}Yx<4w=@5z+fIW?E+@A_2e{g{OWAtN&JIj zy!c1PI&YC(M>(R^+K`!8>5((ZZ->t<2A`qZ@I^yCCts>_JR>S zgNK7lW{8_9>XYbi(z#N?y#Bt+l`JIr{&?Y-T9tU3at_a=?FS*0{)v|@47wx~{v_?_Cf-Fno&Z4P-_&62Q$Cw^LN)n?QYi;#xlFsgb$AnWk>}ap?kf!B0Yj^uf>Op zum9jhHIcFQ?N69uckuEmT+|XEM0<>t6z**Z7vB$dX$n-fonuRQYzG;m@03NC`@Tr# zo5Kj!6EmaRJY8n!jmVoiK*fwoOH_RJMI-Jc`2U7I?P68?pn08Di`$+;)U$0n=K6J2 zHJRIsP{Ay?!tACFZtW(bG?&`{hqXU{;)2$2;I76mdcPA<<6;0N3Op%yT-5?21=3=@ z9e`~DVwkzMn>=}^yZhhM#Y*IY%}YY4F<2q^X@1-B{vN#W($Nq1ygc~rxk z6#=(4`prFP4sEp8)Plhw5l2&@Ags@U4Q=S`CZtxm_l9gKT~u(ij43jUho(kahkerq zMVoF#M_3xGQUGagpu?5nxdKgP@^Ka2`ed0FTF<+c*+dh_>nq6Z3_Wg258zHb_d7}g zJ5S)zO0icaVsPV3#Xud=Z#M>QFpYKc=jWlb(e?}+I|V?TQo zv^Gc}NlWgo_COVt?!fxMVsNXWFD(yD*0joymr&*MVqTxOqC1O;Rk2tx&};4qbi_mH z02yEYx`@Av?{$#iKBgx#_p_zC3+PX5IB#OQ*P(0_GHo zPSNa0G>K^)Hg03EK@NLp)55|yT4Rciv{Cdd`O#CU*AafJ6u+URJk#qvo7R@V>KtOP z41*k>ULW6{!=aJpStFqoUg3k%cfV6MR4}hr_NpVB86MMof45>gQ&XX#XjJbo(KR z6(mU-fqJiRHd{1XnW4YK9(SbliNf9o(4Z$Z{}|-#X!l6OE1(Dp`D(c7x5VJnnJ$3D zke8IEsN(Z_3-km@Qwo_{`afZ@D$RWU}VeeAr z+=7NrT_U~*tg;GLq3JX4D~Se!=cyfcWb*6E%AkpX2$%`5`-$o}Iyg((mBoE`X_2XF zCeg2+-B$Xy%dzsL3gT!s7(Xn`y&ML3fhp>?FRqzRs!Nmf^^!eue^qA10oRLk0+|hz z%cf$fZtHRIK--}O&LHIiW7H51lIqS?5M#w?*CE*JuQQCd7q=HSVJPP^Q+6;lA@1>v zZQHaxGRb^&Yv-6@x*o{mB0$u__YQa=uOOpB` zf@B*Nn?FD@K&2`F>v#O;)pZZS{pH@%#mUFL{nz*_S`%0nE4JOagIBoH^h9+q4`cYja2P!PschCkOn)-uqXQj?YfLgFZF=?s|FHyt0`2m8jjgLf(}h_Tnp%wwLNiTC~E zr-y)d3OF6{%M>G-5~i{g${!*GR;s*u;hv?m z6~>nCIm=(IGc%PLO8pI)$Gc!GXgRjHlEH^LQnSa%+(XctE`;xIxfr1w5_oV+KC%uk zu$YWK^p#9h`0L&i-{}j}dCY6u7)h+_{v#H=A1(mjaok5B>+Qwa;)9t*nw{N(`}Q4H z$i7EqRnN^{`4i0G-OoORZr!QYs#ilx8&VK6E>WJ=TCIDsw~K%9vQc2>aAi*MDPKls z648mCx9}Y4$jQHYw>C5RP6XeDc+Gwuma*OBvAyrnGl@22pRkY)hn{0jsA2^0nL-=p zL*Ec6pTGy!ZegO;J>#+N+~5xE_z5klYOuqnvQGTv_!F*|#D-2}lk0ZjovZ8sLp58u zM7Kx({IU((R$TYF*q7$jgzk0j%zW#P4erC9eG4`;Y!ykt&<$0=mLF$k$(KU<3BxQ@ z8^!O__wzfh?AkHRiFd(Kk|K#?X1?9B0~gjo`+ESDsG34bBtO0+M+72hMr*j~rq`X^ z3$YDiJd;MeVudHyaeK|Akhf4xM=AS|(WRyp553{pIgcQ4G}gz>4aCm=Sc>g#l*}D4 z@!&FPfS@qchwOMu=^ImZH@C#_TYM_oS*^pA6x&uqbuZ_{>)FMcyY7nXXQv4lR!st8 zNAHW!RyYayO`ZCPPpM8N;(|LKXMNd;i@jll?uSdBoyw;l@FL)1f+EmFTV6LSfqK3l zhsImmGg;9xl3*At*H5^Ka&$#NdQ(_S!b?`(-Q~RjK!gN zBf=Z#-&D8bZs{0@RP1T-y1H$BD~En@>!oQ?@Fwu;3c-6yDc{@PR{ZfI3aXFVk|WYN z_xcvTfDrHHK9J322!DjO*qT<&Ak~{hk!i}-XV7~@s}}(vlS%7aur_|?pkm|V#G0i} zNcM+IhVBHkt*t)3T@1jp{$wvvgy5Y^9KsdTc}+;>9^Q}?QpO)nSDM?da8|v>u5DFk ze{u^f5K_>9rPkfm{U-*qIEZ!e_8RP8a853eizKO?GHY1tmGP)4q6rPI2{`gUJ7i-o z-4ZCmV;$`QWgNd@Kg#IvO;sJ>*0&@3k~S2?$at8yy>T)On>eEdT?a%cnjs4u8Xg(| z<^jKWKQ=y1UbsOkkEWZ%nm}&5*zNV}04<(QDM3!fq4@{>|K?ridyw*Q*WHkDsIk!^ z1V#fsuBgw}kJIIz*xwBgs5y#$b1+zj&?Od%k+k2S_3w)18>+ns3U-n|()RrHjO?uL z!jd||W7Tjpx?Wn2mMCSa=PezTyaxhkJl)YKI)Q*nnjL3T2fJRrzI7~%&$OW*F!b@} zK1cQNTry*3aHEidvmyj+o`4aEW)f%=zl@XCJV_=qeXmr3mQ=y(=$=qySg}!0(9SNCDd- zg~I?7AL3$V_FBMO8gr3MtV(oxh2{Ie#-lQ{fTjL$zBBI)%c9`TN< z^r_TYKnrsbaAOOnFI95!>$JXei398R2|vqD9h{(})aR|;+*8eGU>fm3oS-;i@F5C4wY$D zCG8T`nl*Q%<`lhU_8sl89MxVz`1$WbdK)jRmz|Lq1I09SYEM$6#y^d_3 ziZ6hC<6*FNM86f&@o4^TBG3qi&*42FT1lBf=KSZCp$rqi5X#qcngiA&h3e~%ah3}7%p#_6x*(l$}Y;iuE`q@gjJqk z8+7mKlol_p$#zHWRgVwQkh!sl93i*0RZf%-ny=-pZ<%Kl*{LS3n6hVLKI6C@LAzcY z^*SYO`@F^uM;F>6p5$$p^4ootlxhoUgVX$b@r&+Sjfj;iCS#l)vkC|r1i!>diDLdmz^y;=FyYYR{8Qn=_0={)96zBbI1jnRu>38lgNmKu(z;7! zs<@=h19TTki~C)5p}cMm;5VOUiG(*MS3zU6P`!rHK3xWG2(ZStpGEc<`Zyh;VG zQgvFZcsP^z@0CTaL?5 zy{qg?#bB~Wx<~e#^+Q$wRt_vjB~&x(>-+b^SK_JL;4O7ot%^|?PTesO9X1ZA(dpPn zy5Dqd=z}mzS18_+?HY}?eAV*t%Dn8h8dWxW9@l3D8hR)=!okXq=RZF$`}&ASnP-VN zV!g)FHIA9g_z%1|Q>lfNbWB66o#Y`BFNvv%;M5fN1D=NVrq8|L1HSV2C9PoiHyM0+ zi@YpFB3jt?`W!JNI_;rkT&_?9`JQj=_>g`UIkI}qbj8+kP4=eeE&6gewTBBXY7%7b z?DWB0fUA|$6IyfCa6tOjezui{%$#YbF_y8S*7V!Ohq2|^B7V4E3SN$%Z z*scGP{WW)t#%1{@S;(Ef#p(^BZeoi%QaGu*(|uGtS0t64^1zl%x5o=ik7SU^gQ;9K zB-Wye`0xB<>O5K=e+SfOT!9=x#*;-@7BqcrU=t;Pfk{(XKBUPFxa%V*Ii#UOE_iho zrv&L*V;SL0BGvS{SKwbV{jv~u*ovc<=ilrd>6QRAaBIdePoF!m@HKQ9xurvr6X6bo zA=^~uHiWc1@A#LH%smV7QYzu>x9EKSf$=eF1Rig?T3rAH=560p$vQ`63zaS)X2Juy zvpq*!0C96cA|w#cjkCq7Ai|*)oSy0uQwy(&1G(R+{6+EIu1_Z)8aCA50q5yJKfyUd zRf&x>9n#)O;CUR2k#(UvEMiX7Y_j{VPfJdn6#q4y(@KP?&SW735#X-#v1XdzPug@0$aO^Wn zYAqT}hb@3$)0MwBECN%tp9(L&x;;5@VEnmrH@V-q?=(EO4WgY#!p2j@JoV|OJ!@I8m`{dSKs)Kg=60f!e0mI``kEb5P+PH*i9#~Q7H&<6O7 zyd1p>K`dC|>CKzdw}uq>XuP#mKHozWAvJ~5i*-g)_3BUPyG6wW|Fy9&24933;;QkB zZ;;yNU)ZAWVl9zzWLujMzf;1LODnmv)%eaTJyr#xDwyBweIE^~6s5)4YdV$AG?R8& zWc$W|68@LiRzmHoy;s#BGNPaZwt67=D1AqtbV&qBH`z?7D9~VrWJ}tkL}<1!Yi@|F_By1I4wW3@P1$8Z2-Dxg@!+eV4oQ zF8kwdyR%J-DWA)RU*yN)>>ejH!qt@*G0ymCO8}N%dbm{CFSxG1cC@tvk4luDxv$@b z0yoJsD2)#pymEIyOi45GVy@SiNVuX73?e#;B3Q12xugKqD*p@P;TV~nxaA(=vRJ%M z2uWrLC#(#ALeu?Bm0(&!4t1f6w@sD)d6`Z}0QXeEx7O7r`+Kmo#*8`xQ#>kt0=5to z(@xy7E1fA(`KY&nwX#ggg(9U*btVqYwXCa+BRy~8#+}^s5xs}327>L-b0qRWDC*|Y z`^~-^<^7)beWU@)*(7c?r!r@;FzbvR1CV4i^|`n|r^|CU*9ae@V%DcgN-qg$(E<94;HB#rji2h5&--8AM_W zj8P!mMFl%z8HWa?)K+n=n8=o%eOPFNodR>VY-A&}tBhz=j7vljt9ZgSV)WJvkih49 zf@d+wX9^gS9aJ#MvS(--fjM@qreSxKK2@TnSjf^$Jo2kji+pF9K%vuxz>u+eg#FRSTvyUt1OPR>0f>Ms{nqQ}09zFDT~Tnt+0P zR&oZuGQfDsK<$f-E>lOIeQcT{)&rhO7_97dJ7>bmi%($JO(}pDH)QAVVQ}iS6&lG` zF#qz5G6YfG>F|j=(PLl$ir=#DX>$X${xaQSKV;ZU-fPTaE7759~J74~HTyI*oHZ--NzIk{(K#Sw}}IhRZ;_q-y)@ix%~ zI4__*LEcRl%e^OT8!;jxYwMDqxFlX%wlZ{h+DGyb9F8!meqyO_#GN7SP~m*}?oTVN z4Lkd|8x8iPoYJe#;)NcNJ&pzYAa=*AE4n1yTCk7f8rq=LkklsNhyQwdrtcs--jhxk z)yXQ!QNhC0)7MP2d!3IDIHGOhRe<$GkG;~lEK0SFO;-{y=vJC1O+3|!Z6*HN92S#+r<*vW5rGmMKLH8Y34Pprv$r38JsV=d z&cnhnapNT*qT?BYscBT^dtSW7JS9SOabKGjdl;Yd=H2hJcPMFr26lPwt*|k&8kRIm z{-q;=eNK#Ca$*D%w*K}lEloo~$@qD9d$GLlwCfFavQNgu@`QJj*}@RS`_z?KFpNOB zy07$;?5aj~V?mRoSKonJOh`eza7(yy;_zi|%KM%?!~$|dxNQprE+SwhxanSmDe!9>E=fLpZaW_fk8yQyWpNyM(fL_^by#7(7<($heN z?Dama6^OjA7K|`gx5qz~wusG5nCLHOT!YL;P$ppKsR`6W6 zFa=AQm_FpD7iaTeawln~RxpN8hy+*OF>G_y*&%k?>6#NbyK8q9gj5J6T0|c3MJih# z(i=vv>hbMbRJ5Mc(azO+mN*&WtJO;9kKkhUt;^f2818V4yd%Hkcd>tA`yLFRi9tQ00NBShXc?r!Ji%L?o5j9fOz2l#;0CRSY zu6c{CIl<|Uxa*wh%VdnFc(&HLmjS-Tyi0yjjFWZy;4aF)6U$#N(vBt0CiT3Oe+F%k=YKZ~u$xcK5m(}<)uZn)Z? z8C;CdgC;6T&cJ7Q{Bv4B!t1JG7G3c5>`kdNRlWe&kRKKX3bSHg(VECYuHj8b(X!U{ ze8r%*ezrE~2=+PxYdrVnY3{t9>XfEZY6x%4$8B6An8-%g8phwX)Q%X%LG`2vUR|Uq zR{D;K0nqIjdl>P0C0sxn*=7!(=T}*96{8p*xocma=4Y=4>B@qz{3xf;goNH!#Psc* zOVJmbPJsELhoWo*PNk@(b2V2>ME0bwwtQ~7gNci+CC!4%b}hY(w->QQ=3VYCi2HVF zJTffi4SE*<#Dj89X*hgyHu}6ce0FfScL$He6-8E>T>x<(z(a;5P#+wYiEhNgosiW2 zIE&%qLlKrab&O8;scJ>AY)VHztv+~H?g7Mit1RNdI&D^+hZZ}4XANRp+3usUEY(n& z?+1iw&#ZET;rd0ll+fqt5`Dn{5Vl70SY#!cJ(%TjT=55nTK73;@&`*mP4*LGn>*7c zia>?-P*g6#C#{s50s6R3XXE+|;Crksv48?|e06H!vy6Y8jLm%zZGmgEIMhh$- zNmjsHHuC~zxy75C`-?zbbY}cjCFS?igAD1Q9w-=K1i^QgIQXK?Ru@#MH~d#H&wJme z`JCMxiYmFxah=G1F7eyAB~%!1E@MS3rnn=2B6@_XSz~V-m9Bgzn2IPARX{jz%R6ZB zd5=(EFa0)&k1kJv9=#t8mrr5xpRlQtQoJ4?n;y|$@MXeq@jl{8EIQp*uuu0{69S?hp>V~` zA>sLKvI4QZe?IN#*NrZ&?pro~8$UlTDME%4{q^ZT{-IHO!&^Jx$Swwri$_mxt%r3P zoL4YA%Hppt|3MYlRrj2A7hz9)F;P4pG7SPizy}Z}v?lAGd4|UrhNZWVeiB#ceSY=L z0-NS7Ei&|a!Ts^WZ9bP40=P=wRJ%fkpK)^llt`~33zLsJuC!ljB%fP{jU{5FXL)n~ zKKY$hz_7f&?o;HC8XF6J*-DzzP4ggTV(%;lwr*Ur&>K_2hMC`ja^_mwwNklqA#PB% zWJmmMO`BWj-|VmG2p$-YZ8HocuoHNf1f*Z{HeAw-?*aJ>DBUIH$?_K_wgs7iZD1Wm z+T1dF4|!qBG>wiH_lM}wj~_mYxW5oI_CQlCZ@TGVUX`7<^n(oW%bB#4=HAqu8h6VbWXA}Mb%h&G`-PK=Wjs?3 z6Nzo;2kwNrLrRIXFzU_S4LzbkmvM-629{^)PuM(GM@MCy=p|ZB=~5p-uSkH`{UY$+ zZAI2KW9!B8ep#)p6=2I?U$eI1AMMz=&p(VgXTzUAX`XFwAGm3swcOGH`UD=9+v}==?@-O> z8lLgJo2ygQ%D?g#`;#6s$U^iHbo11~QGMCn7PzS|m^Y=;(S^7QBecF!k}?yBpEPR@ zHkYy>x+V)kRCZmSx7Hf+-!+J1|MA-;q^&rLTtXlFtC+HCr}vih_nD1PW`Gjw0|u|q ztSo+~jv-++MJf~~KvjL_T^{tlwW%kW6wxGC?_7lyP&>fU!Nl%(o|si7Nvr+<7?OUu zweTB3A2Ab0286|N3XX_C$pA6;8iPoX>W{sca@M1$cNDDJbtMBxQe!%JCLYt_=Co6R z^9UKg;R$dspKK0q4SoalsuI1$541P;b7Mi&u-*8Nw$NAkl~z3j+B4bN%)F@WiUWS? z#7rP1FillA)-{98>e#(}XwVcrN!#N4O@|^f0x-fV0LF%T9C*EPhAoO|9L3xOT&RRL zx)6-znvR%XhGi5n%IsiZkFQRhsaxa6)E5q>S1w|S3-3^ySMC@1eS8om*5iS>>e!+uQSCd0MvHpN@`lcWz|h~S8LG&=Tx>j6+td(dQT1rlpQJO^9*)Um zB`~=)Pib#)d`zH~**IqymcGKos~rhnBM=&?ddu*wpix7>3ZKrTq>1LeQ{d|0Xl@N0 z1~%dkc4gH;UmH>JvvK4l!jy$TP zRO=W6NoGz`7}oIR&SQR3v5*5L_kqOb*{uX7x3%ju4_f-vK-RXpkYk0cbEG%+cLa{k zTiTc*`$!Y(+45q+1hR`bcu-TTq6!mFN5<{N{1EpJb+|N3C`&dY6|d5o!{}svNq7ET zu^3%$7-&BfEv<~muOg@CfTg$AX`i$0UDx@Xc`(WQMFZy&<<*Osp1>*g z@1$-~5iW|;9@56Ao69gMJJC7r;^wkb1AHR>swg@I)RrkPtE}|(sEr+4encYZ@K)0N z;ErnVstzq8PML*{7vw?Ob&C*>WkicWOB0WT=+Bm?iG%>ZShA+y zccKPWC}+am=8ck8MslE{GXXnJtlw!a+u7QoJz7vRrz&r?xM~?sp2jR0sA8@WIP-#s z6bL*XuggI#N+EO7b%VpsJ=L6i6>)>3G=YImQBbWUs>-W{hs;0WR!9yf;uLLI%jcbF z2Yd3)$=(f7qcRNca*zk);IH)YqzJe}J5;^F8r6X|Y!Nyd5?X=(Y+#SJ+iu2SBm;#8 z-aa3k>sPpwBOGaB*;gUjW~C`}pjF~^1izQO;-9NnZGoV~=K}d5W3McDh@FD{HTOoI zDe}Vkh^rU{+^?UTf1X(_VG#JVG1;~(Rf(qW7WdG1#j`|S7a`RAPB3P3f3lp)6)V*Y zzp$0m^6>@j($=7Sm65oF zsL$}IkGaSr-iHyubFSZgr1=tQDCPey{KDl8H?^odhr3?9hxJakc-Xphludn8C=zxO zfxc>S07$0~J~PXXK_7$fru^eAw`QXc5Qx}Penw5Nu+eg`vheVlZMOOZ7K#!P2jz#f zpO!nzAMg`dEBcgHb6J!ZWD&o-LTR|>5bvF%wbXncbl*5&;M zD>19g^A_F7ZC+VlCjJQqP}rP*hXvM7D^m>&OP%I;$s;YU`PkcJB46e$kLImf&|q1v zcL8(qajQ!IdT^(#eKp`|UY7Bj>ssWN@Y5)2cM8O3mYg~XCin~&Uy^QEn!wt?U^=E; zW#+)|ss9fGwGucpotTp)a6l8L<|QVZhcvi*C-WAbap0e1KeE)8PHK;9IF-KVrrEWC zTN|Og?VdM1L^Ji(Com3&cgBN)#(=y*5s%(gUh3iMJG0~=5$5ow#)T=ASc;g*pKkfO?=*t$3f2NNAnsi%(yRj6B-Ou zM`j_1Y}tkUv<5Z!JKo)mlAnl$Uo!Li0uGP8r$i0Is*+>gQbmRx*j~cVkd=)n(2V83 zsa#euO*nmi7vJ%WyYv8hl#%-mQnsx+U5p?3`$ zEbLW;K!6fJiQxv3*Y9w{&K)>8uC|kHz6$weCKxSjOI1?}VqM{QqWDb^^9S3}r#^&Z zr}$$YI~jMd3um6|P;>$RyD{|fmGU?a?l1?a{PG&Ng6?E(HlC|`!1iTK_b#%hSjKmPas{;x2fAM%oy-#>eb zQ|0=X+bg|C|ARQ}gG)xP*4OUy-G5ts_y6Ln?Yx`c{dd3nf2_WX4M$kvG6fl^Qg%PJ zrT`o9%2v1XN840fpQ7yZd)Vyd&09zNfH%wFsGkFFOKeDLlcQqI1h zN@4|<_T|4ke7gVRqXS!1rvQ6|;Wv96yWZ&G+fPsSpFMqigf3foRPg4cqBmV)N!AJ} znW~JymxO5Dfxg$BZoKP5POv$QcHbGS{ew>W*r+J`Upj}Ax%Xlfek6b5X4`_ui$ zFOD8_uC~=5F0t-N<=Ot_A0HmFcKrBq+lMa>Up{{R@CDla|Gn+w?|(da`sndtArCXY?0WD;Qrn{Pg1SPtW(CKH__} zJ6nK&;K`kr{{#G{fHk~h1ke#dwuf{pxPTWB(tN+X&)^9&`?jwA-QkO)gXa%k?1OgI zrc0(oSt*tGF9gUm+SV|d=IX_`0>%>uhmd1 zucEQSqRjKs%0`OWO5BG(JTVOk@AJa`c$#yc0NqpTbC`Bbda~&Sht=(apI+7t`4fI8 z_Wd=%QQU3GimW}=miE&apKIC<(eE{#M_!M{_+4@fr(hV+0M>_A`0#m=K{UXBZTWy}- zaObJMjh}%U8@e!)B2Asyx0(MAL`@X{L9w{Lqpp=7^pMWm(e9gqU&kk4e7J@hrGNZ9-^1>M z>T*bf90Gl~14sv%TR6RQtFK5f(+5W}Nh(FglUKN}Qa;EvNR;o%B?A!{Jg;W>EHkst zHhRq;K{bxWt)|QPm_AtRO+7=JN*Pl$Dx)v-i}87g*|r9^VAnYn^yVW`!GB33*t^-! zx*;E~_oNiPI^L*S+{ydeGaLNsyA%w!q%u(OEonX%TXWbqX=_cb>A0Zin(A+p7$GEC z)T7147en%U`JG;@ZJ02I0)ER@)t7PmS#arX5Ee7hH2PEL?c$UO0?wO}v#}mrZx$2|+?;k%pc(JuQX7}Yiq5st4tG!`o^Tf(d%C+C+ zO45}F6d;nZe6u%@DcCVr@a{rmw2D(o9n^$&xYIXrV&|4&Hj1CGhYdPWsr@B*w_+(H z7y3NhGwTSX!T-# z&AgOf1lttPZ_rm4H~qcK6w;W0^>uF|ULyw+{-)_!a+J8eDYhKh2SK<5L?%x;OzXy^ zUc9d#jH2XF-28?o79QUw9Jb?Q%-J%Q7pOKajAu?)OAwOqGdwHuAe@V+*ksCA?xSziR4az?O;~Wmcu-vxoH_?suWsF0wdc_zqCY ze`&X-UtHi9ip{(Kof!3!yd#?{QW#@L-3}g+0NgRKTpO*Wm3Er&2c;(#!A_wcAR2=q z)@+i(>6`d>qyw!SFe@qD(}$9}N06b?MzBY7J)|Ol>hq4c3wot=+;C1_?@_z$uAMPC z7X!y^_XkFbv-9?_6U?X)OW(GRd z5T?|BCV5#^fVKvOBYf9W;75dCmu%61V)lNvfNQfMn2)}2bg&g8Z{B>Hc&@!kJQ1f+ z1VF?h-xrF&gQ`c#{nVrc3cZX5c_ncxwhOb5<2-cPExl7iYatD8AV;Myu9-7u*gMw> z&unaa-a5O{l9rL=Bh~1t$zlGw7auU6{l>L+k>N9494`Xj9o#|zo z%nh9#(%gBh186;Meep6grQJK4Y4`2Ek^k9(qQ4(R!t%W1)K!H@Z^=Wxs#-KePbza^ z7vUQn!#HYPC<}afIQiCrAV|Cp9- zdA_?b+mh)af7RJZ8xmBKaLMMlMc0>X*Kea-=GJ+w2T^* zgQuQ>?9L>cx4&JOjLrA)t2*d#-q}!C=Yckz`Z5+6?}yoM<(e8bgA4K(zSOnygzz<< zl@)ieo;#pE>u1H$zOss>UB@>qz%!%e_4`7glI_subj=J#5px|?|II_^? z#l4vof4lDKK{+Nd$7uGPywr?nP<4tH+B%U;DMZz=I*+T7USv>94r`0Iy+!(YO`kYy z30Epn>VETB@*T*(gCVb0y^9KV!D3(IBzYr=6p#Syn85DdUZ(y%cFfYSo)u>l1KA25 zhHWb7Y7gzkpnpD-nGn@dNk&nuK!AexwK{sIQNBbcba(}p5l^X9Mp$1!q7E12|Lx?A z%DpiM`3V<-hrgC%Zor8a1R)RFOQ@Y@7uXRa__N25$k0`8wf(z5xcg>~>`Pn5N;$oY z?EZdxezi($RQSiS+41X`Brwl+&Xx`<0;Hy1*GwcCs*sMkG9KZseeSyI@c5?iogUw! zI^^md1*AP>0Cp^#>u%=(rXTmHT2DywmNzh414&lrgE(N1-nQRTI_TXY-MV760YXwQkjydI)yxrY_H1e3tr2x45=?=GG4CX{Y&U;ll{D}au_p7E;g;`dc*ST&ip zC)iCyyfp@Rg2Y0Gt@0~VbbFBc81kE`GW4ZTN#GZonbJoe^EZuoFbcZs^K@Bkm^~K`VSe8$9OnEw7BUf8eMXKa>o;rfw3h#&`aEi*tesfIEyh5H z1#A{Ky5oQ7wXWKvlf}5j%1;D|Y3RQrbWPKV^M>)H>1tkQpu*aokZg{*9({i( zZGCzLG?xQsX2^TU8-OgRvhJN(t_RM%VQLA)Vd}b^%sxKAJ%;VjfI`adiaK7U0}7ac zJA;VE&9dqx$f{of@Pa@!B}5(O?l)IR!y@Aq-p%O}GVxrD{JiK!EPx3#yKsnQ6BHVp zLd7O5%H|!DA?hYiFF5}xp^EaYj&SYpMmGZINgE-O3J50I!hn(q(2#ZAZ}_IuqAtLK z?s0h*alY$QvAl6aB_0*Q6cpm@L+W>rJOSKk?+?gYnhOdVXc>g3n02$ zRA~=0BABEBRRJ^!1yWk#8pJZ-!H!xGpS)P|YUORtz57#x_$kX0UsDu7iPI$yO425O z4gJs9QR6DeI21)=ggC6G55WNe^{ve{f`}|DA1ZIK7a}g>k+!ua8r$??@GAVr9EJOl zTiGT}-bL(&Kpt_PDzn>7;WC&zs4+;+;btQmXfZ}7ioUcu>`i~O<}6}>=Lae4KvU2v zBg!(yb9$tQ*Kz+ZIDLm8 z`_OUtg=Fu6qQRI3w)a17&uoSzTk0MXtCAmuo*SMI$YFzF0|QdQG2!1({03S@xB)ku zcz*fqDb;^z^5^nw5e){qj6Loaxip?U~_z=p^KF z4T#GCl2!!gsW0VbzI}S-KRRutv_-V4v)haaf4;o3;?Qe2r;(R}kArk{uB5t2dP5Dg z0F=OC^@lEip?ctamgrDJn>d3z>qkoPfo;ZEpsNpu5d z)K91Xap1rln>^Jt(Dem)=?PU<4&bVuK+|N3ySV)Mb*o+S}}NFBJ=1hD|(YTqL0cKK|ZL=RD(3EvjrzunD)LM z7QP&O*PC-CVmX0Kus~mz*sOf9W~3Ri!M`=d&M+kTpiDx~KPe+9=0yzlWXYXw(EQd- ztN{1WDeeH~1E{k4;~yH(+MzYXklPf6W@Xj2+#)0)l6BjRf@oy;>I$=u3c2Uef^j5ZsxxRU|PXiB7v&iv|2`9##QG!`|jzFZ` z`<6iqF^+o`E%KW3P6*s~@AQ3E9$6flfVzS0-t;;Gs%}}bmxcOS!qmmuaZV#t&odO}-)(-q*YJ_~nA6NBJtcpEGBiUEk~HXKVxv+c4^ z*e9#o#a#yhJ2FT^8y+)z*UF3TgBa6BpoPH=?iJe1f?so);nvB8Aq=vGd7j~vH_qX$?4?+3ca;;_-pLt^m{#Co-Oe{ zftFjq&9~aUeyHYsY|;sMa_n;OnXS|NLmI}f3m~kal0TpbfK{90iYc~S^brMbIj@E^ zQ6o@V(Z@MeMk1LeWj5XgT6Gf`-L9WkM^4}bYQk^GZ&#Sat-{8yE{UW`4MsWp@pIU; z%A(5;_y$4(wD}l1!VUC9)O5U=&*Ao21f#d|PMbRc*QcGHQAdV;`s+zXK>&<<5)=sV z2drrNP_j=ki{J(V$K0@O`tcbPsHWKbDsj?Fn_P?b7#E@$$th3&h=@sd(|Gj~q$bg;tb*+@ce4vAR4GISoJeFO%HE!17T>EXox^;_R-PJuHs@*DcnV ztcw9~`VTkU7nmnD-H|IdnmWVT!kTZeKSRQh*~#s@kMlH^-7EqqLZ&B|_qg^0xKZP* z{2XV-&`whYbPAQq{#l(V1K&MQOlppd(Zx6-=2~yV<4Hh!i84G2OpSVa zA#$`Do1js|xXmS$>()w8CmoLfq-6%3=L+dX8rF5zQRc(+0BCx>#`o#wsNAR$XTVtI1Bnlm(FYRDIWwhU4Mkdy?Q-jDq#I2|A0 zn*q`tNVp3a6Yg{3wh2xljZl?ZkOM{--7Lu4JsOw62O|l?l-xQi%>y$_2l)P-u?xNR z|0*4Hh=kw5sY-wK?rk}Ei6M#5#|hRgm1meQDFXuXIBD$0?qh-O$tmOL{e(zjyz|}T z2V^e5$j}>YK}3)bKJP*{JMs>F(Fs_yvDYJHCIoA1t2Zc1%FgJUvhV0g^uoI)3wjFE zp9J=6-1SHpE*>a)H05Js+ zQc=xPA)_&-_K7IN6Uzty2JOFf-QC8`L%@HK27?E ze6<)MK!y0BryyOU-tdw)6g>9`@Q|}D7dW%pjYn`JjKf)#dI(+wKK|?O>Ytd}0orKd z0#ZY3tbApVjwZ*=<%o)Sv-#-3xlU)#=xVqdXktw6o`9)vz?LY$QzMX*IDo>J^;HmS zMlz2RTIsfVR8fqGq#*>^cqslV4A1nj68%okw_bgCqB$>tB#MGJ1RcuR!3ad*ub+5%Kurn15nkIpM6rg zcAqdU4kF@eDBjgmtMg?Z3UAKh|EjPXu|k7J&{T@$I>lMp)^v|g`2HJes5yd-^ig6$ zvI+QxM7-9s<$H@>r%xXF0gmH}2&8!%mQJw2P)zi2FH{?7`s5*|GoWy*Ep?(yti~|J zcRJ7YW2<2pt!#`A%oi{vbvBt2ApLspZW5B=f-;z!wW!Wm^9* z*pA=`rg#@#xLoh8OoN}f((>}KH}%_j8@XGldZpv(_H{gda4c7&Ns)?kDQaUjnoEqPpu)g%7H?#;v-(kQGf$$~(4%U^usU zg?3m%06mU|wjd7&ZPioB1ZyXp?op(D9X+24T~3k>t~@6r#PA2O^mv_jUC`Z(Bf2=t z)KZm!fRxNqf2DL+3a|ifXrVC^DGpye+8b=l4b*s2sulbnXN1^+o?t#plIc)8`11|i zy*4og&;v7g)0ax>OX5qx$;MYzFABolu-r%QcOb|DnIv89>cD{|h$$!`-eO`_1 z>A(@m&{u$jpLyglfDw{6u_ZVS00_~cw>1mAKq%yi@UH1!cVX7hJ7W-sO~o6VTkPME z>c;gCGDNRN%Pts2Km2nL*xHYBGyXI)h2(DNnyJs#1Ca=0Ipp!#jw9PkgqkItB)b5T$;f-eY|YfteILb=qu@Px5@rZ}N4e%zD}J^r9ir zT|s7wnCjSB1R^Xg*rchfc>HCEptPn+0Hblwk+}qMaP%q3v=8DkZ<-u4Yq5=3_UyFJ zcc8rFmwZ3FyEI>=B{+d}!sCRlXdU6{8%j>3r9P>-Rc*U8FC$8nBn~AK!m^V#xntcK zf1m;rX~63%lc(G(a#B)Hh@Z?=wszd}J`r40_XV@6!6`7Sb~l3RE^7J2tujb~cQR>U z%DBL7>82AxRtyan?6V*jUjHgc!R|vHDXZWIs1W3x!bOi|+ zm{M2j%)g^-=(Ry!V1X+%Ubj+n{LAVpafi;MiM!c%5G)O6qb4>B&KUPfc`E{^6svjx zcA!N{E4SGjCZq*B3B94YVY#o=t71wWBT9@&wJ3~6LP+X|wg0>g2 zY`Fk+&1}o>PM2R5+(p9Mv*ELCK&;PWdw_=S7I!O3_u7+&loHO39H{Re+#cd=Ie;P zPku!A?7bt^{Nd(qF2A|AhgZaKU#alw1nEwxS7alw#G|(t_ySdDygnm3xqewAAr~{i zr{LlC`Rl*~{uBz;=j8UQpu@nq^0UW*pWrL@m1i2>VsON|naeFV4%DPj}HUm20RAgQ-cUx6`rXR=33|LaQ(8SASG2v$a1Z%c=T za3;oCY-^b{4-)ty?mRut6Wq6n@C)z^JIT$0!txcN=b1LOfkXb9RCvKAN3^RZDtGLr z!VVt*_8yv0W%qz>m%fn(Q7dtj6@Crgfo(l4S6`au5F9oq9L!l@0R#R=wUJe8utwa1 zk+A2>v(6m^Q>?l-Sm7lN5wW=v`^w}Y=5)Tdx16 z684d5I+O*~ljbC&WDvK`lNXg-F`ykBUYqG7Xfx(@Uy!6?y?~kx zjoNUp>Z=78xbchWHgUy9XJ)T&z!s-sxGnN}DsruWsEb;2C&rhgpyP^5)Vc(BPMX$TK@A@|AsX;B8ZS03&YJ7T?dUo{tTri-Zb4bnh4Ax85KgMG9L)- zbBR}%SOKLR!67Ny?-k6aCe=SbMbaVrr870CKi$6%j0N%6Ri?vF{chn_p?9X8n&;CCE$N?lACGyE&LuOjvena>w%e{{3XF0Q+c#93eAiyT_jm!5m2FGrQzLNhsm;xWbhVfb4nDnKHmP4Z#N)NW{&QmCP~x0U-&u#%l@|P z9iyrf13`moJU||_CBVotQ%7;5zq^fvYqeU>eh#9pinltJK>QY!95|LfuMQw`taRdH zb4^7^Iwnw!sbU$EpSJEU~H;0zl|<_x03S3{Q@$+4|v7hwmWn%E0^IoZ*`Maj8Mt=LRD z+*LJj6il@J1yHVNQ`nT6dq{rhY@v%}=2AO6sa9gJNIakAPsfYuC@oT?A=o1OFt!jZ z(CCn5AO0Dhn%)mWD@lcH zPfa%oEF;X?=|V$i1-~Rs{RujYr^|KqMIa3dG$Q~eccY!26MpG3}ZsR;bVH@V)Cc!Q3L~G4$WK%$gu?IiKUPJ`1tqMnX2hMz|&K z*U~Ap7&LSk+U1A)$PKjRcb(5>=#9-)St122+#%>tDkC!a1O1wibu;(3gLfY!NU+A@ zSZ}44wKd z)AyqF+a$Oh1Y+Fu8LFlK7_acn3I+X_<;uE@3w4_kec0lZ^&CKNzA*iU0}~yETM!p? zDjf)QlZEvA-9RSo!s?@$Alz|HDJWRo(nvR-gObE|z#{Aja7fqHaHucaA;-l)HqpTj zWZ%NYscy0C8 z%uVgM7_NTcB)NC5y}<#cGofQ*X*1W@e(;xz51L~$$4`8cw@aIm()nEGqB5~L zOmdHqK%YtiGGdTlv0uxfM?9;0;z+E)VtBtCbloqy)bBuBOgl(W|2u)cvTT+8GvB)d zdJZ0HwR@{p3NpaEPxovX1`$hq=+}&SoP-OUrfKuif#jfNW0r)7QjWCx&6YX?(zL}T)_;!vacDA83Ne00yth=-Cx|l z>O&^`=^Aeu9h^c)bbPonMA&wUT2t?K@YSQ;8S3DkB@LnmYmalcEg|zy`tzt6}Vv7ma?4gPqDM zolWV$uf~?N+ttA$R}Rb=dt)jOJ<5#Pz?+HfNGKd)aRTD^4I7OQ)5PE8_`xIl zsGP?m%1hRKF%i}m7A}DCw5DKBZ~Mr|c*;J?x`k1$mw$f-st=YGigw(IcrS<* zyymVYSlD|yaiLSJ0V;@RbzF$Vo2~Y0L%H@Bkok~myXNbgM#e~aI3e*3B~DH-^YXS+ zIU-tytm6V=|KGXkw$ksh>B=CXvkv6@?s8^oqhhLB(Q3X9U&@XHK^BiV=6 z$G~C5Gb2IsZeSOC#XGM5e87w{?UZA_p11@w4~zUFgn1Poo)#MP4b}sfG`l#PS;D_- z>-(l$m~oPF<_KRnz^Wvg5Cy-=tL7@*+wuJeOMFr2>Gb+#pw5uQ@KAfee-TLKg9$O` znv8%{8;98hxeKPxjFedYfYP|5gTWRb`*y$$ZqM?yq)94Fk7<}I=6omP-61-V+$C&N zIGtkC#W{VU!&PU#di#F=ri)tj?dxjh9!+i!_4y-si;OY6_GK9G%8^0nC2)Rq>5D%QQN^sl{A{DE(n>CAnlsnlaYZanZvmqm9N2x#eBF+ts6vreMdz$Iu1Pus zN@dey(^lngr<^9}wPqOTHC8uAC;*j~a?`Rs-jW=@0(CSed7fh`S#OArV#+eW0Gu1r zIk$NUYe;6e#m`V3<*76$(l%J6c;9=kxoXqp%tqjZ)>$E8)|1OYg|zpKf!^I-6jbEh z&82a}z2zB3VnuSu_DA!msdgP)GU-rgc1g;sfP6)PyQpA8rfbu?;UTbo^UtY*vB01B zgI1}!!j{PD%0V{Rfoj|zPJhKxp?)1IUI6PF4cl!sS4dJayXD>UFlhhmY$LjO*z|Pu zvFyAIfO#+65m)|c1p$0OsNPh^wbzq{3p518xWKCV?D4NDb;iFO{eAWFRj|A&oKfpm z*xfp9sa{agL9UeOTeARMq{9g<9c!?Yr%#s`%Hb`(Xktt|pu8hVO)+Pn3qTynGx!?1 z4PXz}MPI`%JTt$T7ixsGT1LH3-KAMR1(mr=SHnew@UdSN9Q!O@8oXv)1b)$q8mz@y zMiIT9aL^6fe8i28z940z(z^MBPlB!zv{F4ncCjsmrc6Wfhqt`H z#8bvHRGmeUT15vL!?L{UZbt{fnuEBk_SwWW!ewgkk*m!+8s-@_o~OC&0-zr4){`l` z0a(yY^i9%Sv8H9BRj%C~F6OSRjgLtj!Q<@X3$QJf*4r8|?^lz{2xVc;F+|rlF!@tV z!L2n3Y|j>JmEwo&A&Nw}fa@b?k$(~)!eydwaC%`!EWr2QD8<~Foj)AdQ2mioR6?uz zZ|Qpx@7~BhM!MLyDFEE8U z0WPd+y59H682#Yn3pL`!**AOWKZr^lawW}z#ui#lpWJ7IRfh;)B}2)~tK$WC1vAuD zO!hg!rQYxFCg0Qci6+(!YgL4*Bua8g?$s6_x*BQns}oL2zbAblqJ1s zbj$_Zp3~s$l)gOE>-*6z-y%mEaAe(&r^{*_m6h>*nrk_I>XoZ`aiChl1a6k-5+}ge z&)?IPfX|}|i8~$M9GIJpzn4#jW?CSS`hgN~Yz{eH7=NW}mX)wpYira+m?6-QJ5qYa z?X~s*T=?RnL0QH0bT&onlrvo|>6HQ}GaL#+oO&n*$N-&2F8zbRA=t^a0YU0S+DS{j z3QAEc`;N$X4;Q-CC2m8;JPgB}6%~b0UnY2nsFbw4?Q<;;Q}x*NPp;z{goPFZUkn-j z)T{2=%*DFsc3c!-m7s!FLpxovm{YP|lpO=$JNIK}?87VcmeYkfHtjFSh4cz*;(EG< zc>0S;Mp8I;C}c^gP4#ZJN$wK$hPw2p54Pq@ejlr(Zu%q+C8?w8kVc7+J%Wj*^{HhL z4|22zU?W5Sgo4d8LZT{S-6~Xgn{ylx`~sS#36q^j8ikx+l0!Y+*x!nUHH+yU&91B@ zmIKj`?)IFCW0rg3gr-Al_h>dqMt;kmEh=JU0d`@l4B)Ux+D%CbV%XVy47CygML1xa z=qSV|e?y5kkgC>=c{D@Lubbih!RU@F9u(i?=yO+U#3_T54NTc~8$( z?l0E73;{lMKFFU$XK4+P`lH6zn7B=th5}lyy#vTkQ7?dXgXZ-TCScxwDb0qH%8a|< z+3~hNNB$29D$IlM8Y5k3r9~*Abf%ZBh%rDf!11}2@$xtZ&RQ#+ zl~vRFX+?8K@Zxx=-WBi>3Xy8SOA5(45F8%U16Bpw>a)QgOlbN*hF7<6dy01U`k_0~{JkN7s;jPYL$W*&1X5lVrif zC^(zyG&Pp^c^%pY*;w)me_O)QJI%^x?YOIv1|M3`hZe`?lKy@HS7s}0=r<4G-oxxe z@h&y>tfZ3tI*Sx9UaoPBTuSa-XC7!G_XRk-s@4*h{6kc1ITDgv4vQW*&ly0drbOvfIw13-2H?V6 zDw)aa-d6E^G@v0tfP<-~ptB^SsFFHdxJom&oUxXO{tk{8G(=PP&xG68Jbb-Y`NT0PraoL|s2$ z0z34^>wcC*;6&VChhg6}pXi?^9JI|jNE$=wZEL*rPPwuF_{GuVmm99$gcN@!{S0h| zSG2KpFzv{sRpfcx+?|WuDV&agPP(Hp_ynJLY#(dgzb$j5mGp3fV9Ct&b{uMe-O3oL zru{>OWt{F)gf|M#;S-0*%s=i0p*C+DyF5xs%i)%QW@~_cM*RW2bh`FF=&b-%zy|v& zBoU)XjQRy42ASvP#O%_R2B2&^(;eR9ZGPN4uQ{M}le! zRLe}*L(UxmC3E8*&+yeu5kMiv{0rJ#rJINi1PlENZP#&Q`rs3raq>zrhYstsSyAdH zmLJA#>+!H{8y3B@jI-PA(7y}-hLz!KNpjZ}q6%eoBbifFO_38=t)ao%y7e^9p`I|t zM996A=~jON!J95Jw)$B^dHfpO=k-~q=i%Cc<2pDl4dmcatsXze%DMa4R5*M?LazxY zb2SSrhY!FdsZhFt!@v7^9$xH1!zXpWD>jL9r7jqTj@J~fj*bwa*Ln30%PYU~ZFu49 z*=V27W%mbn$yZiN{`{3NZuy?g?8^uSfN|Yx@;_eORMsW8n>t(EnWtnwBJImpn3=mH z!~$sySXUT)b4u2B;$W1Y;r2*u>-@`TJ~@dhP}SK>83lbqkHx!NXj(R~Ha+O`0okZM zG1XOJoZ07QHi~)S)#x~j*S)@mi?H$;BTwEf9DfSNDeemy;t5v6Kp*ebM2vJ)Z>=fM zs^85x4)#c?WaMKl2vbb!wSsPOIbDSyf|t95)pY zF4gba0Rw7JS?DW0z?g5UE<7VRGq-n5@K$~g+d;&4vnSd`V zL)yY*eQfWV&lOAnbKYugnX5KkEs+Y_0-hs?5Zq}*dbXq^i3Zo2a@3vf0sebzYPQND zxp&}0)Jmr-Y>nIFXzaKj+9!I)v4EYopc4QlD>@SYNKQOurqq`Gm5CqO95J03l zK*L<~&fdk}cd^{Tr*EUP6opmv=U#Yw`XaDKnM5MAdI40m5gX znf3J0m7jTw)wN1}nx>MtvaQu{LcyykLRo&GV4JZg;a@R4tl{2v=uP`)SK{$OF+)CAQG*9c;|IC|!|7 zd>}d}S;}w6Q?Fth)Oy-$Je^C?25UenJi;r5hAUk^0ad$^hmvj0xO%?zi_fJcV+Dv? zNwdHNYb>y6H$Cf#6>iAZIKO&5M? z(v%#LTF0~&E>BR^vk9Y6!StEvoq0yz{QLBUX1#J#C5?_sT*JNc3cHMN!QQKw7@X1D zMrrqu?&2NzN%*D*shP}rwb)Q)#e;G+$n~_k-J?5ftit-LFo|}Ph?X>hbF-LOtdioI zLf*Gd-{&-S=@ED6Ft%|6v+9qf_^BJ#VCin*m-Y!878qK95kA8xm*Otzbar~N)=DW# zN)?kQ8`q?~igst`DAf?@uX?_w*Ki0t(mPCMlfp-_(wZWy2_UQT6-ENyZZ`2fgbs}F zVFg!QDLVIRD1~Xk_v4wr=p0Jmo3Bo(HHrvBLLj@MtRMkUXCEAP^Ge7~yEXwz7apll zq6{4K-QxTz_fJ^F36IA;zC1`MuWrmbc@~sGaJ3$hp^f~Jw_L9eCJ9j4Di>IZel#K= z3UdFx0hj3Rne~Q7V@4lYTODvwILU^AZmh5Uf2ezx>8xL;#r)?!J88ul9tne}HjZ z!b%r+{q4OOdfs}VM6Ss4X_3C_Vq!D%r^H^Tm)LALfVPxQmX_QR(+Y$@nq<6=uVtS2 zV4YXl1^I-)LF@y=D@&l`Gsk8ABT*cUcY8*r<+z`}!}`^&M`;|MnoNg9)P3>NWyLq< z=r6>p+1^6l@2-j-P^m-&PAo?fBsV*lo|_C3@yS!)pAVZjQE{>%N|OOfFz94C(nipB z`m{Qj-Q2DOPJW0O4>RS4qXuIhY3s=5v8BPqBq)u4UFlMj#f_2T;DRh$7eby(n%KI& ztrg9KK#>}3EPhgh-24tRsA?o5kE|A_v(U3thh)9mSUSw#&n~OE-{b|8s6Pr zS-8g>OMNeZ6pv@J#c(>Vw;yX6js<<6!kA-Cn^NgJ;VJX`r|VNJl#Z$gZI;?`OK5YK zq@c}>uwUKcLRz``7i@qe0k#uui5!}WN(c3357qxZA==94J`;OG*;P^x4-A3AkdGwQo?9kQH6yL>PPh!jj1$ z?6!*Lq5P<@@+FQTyzr$103;o-A*^D)(*=o)UBcC_?AnEbMJy5TltO%|d*1Yk_!Ub{ zfCR9VO-XW}(X0?#O$*Rzx76(-IQB+);r${)1)trLxOgPM+#i8kbIaQoU~N+U5(5hB zVFRb{%vv0E(XB@DrgAzxMD-q>8UO+ZuI;YNHL-54xt(;`uA-OgAyYPv51ScOF6k6# z;F2#+mi}(cdfCpOn()Q0!^Z+Ld&?$T*9OE)S3a~((r8#am`NJ=rJy!E=4twiIPA1h zRrD5p0}fef*KqPCQ(K#ODfwwP7>sv11%OAgj7nBjT{^@+%T@E|YKq70?6}AVoG8!X zC>M8p*?KD#HrWDC4Zll)Nj~Dj{0)hWVJjx|#6o7cv|NMy%PUil&gZ?a_n_27zbwtQ zh2Y2{DAx!!s}^=Fqx3O~cj9s+2^TF4F@c&?xwHMY1~dZSRoDnE$j_QQ`a&R&m!JsC z;*<)Cwc!+U7+evw{2BZP-GR`Bb5c!l*lrpfJ&{}7e7K}pQ4xGTC62Ia0^r!-@GZ3E z2#6CC1=20vMRBNfD+zEufr?tmY`8#~cVVUq6t(N_xeC=)WjMtf&Z#QTF#BuRU!9pATsrJ=qucFeLBg~9TMcYN=9?)Eab4S}T|6V&MtO5C-X zE!La97h29+orKH+nc!^EqgLyKFy8EH$~Q|5dWC)nPrF8Kh7IBZ`g7^pO1nriK};=s z#a>eo{_WmBZv`8AEg@M{_Kd$;oHb8jw3q&mL91@jor_OLU0Eh#dsXu0$mw>#p)-_d1W7(1h+Pdy|Q&hPqZm=cc5A_TvHz&5Ph zjP8`vD~XcR*hYhi_tatmDYaI>;H(%xx7$*iw1$M4X0-M)K^#rQYq+9Kfk)!1f;rl% z3t{#54)lh1G(qU#6#G*aX39m(vH}lU#nxvYs4UIw{*3sqk>(xsl+D+XUNm6mcqf|x zzk+kihC4rRsrxo+b-*R-!c^N8BK)#L6zi84M{kZ}v>F!K5xBDEAvey>+c5z zFrH7ZiDe2ZS5kk(IB-(`xn85)tmA@~|M|h|tA&lF<)P{fk6&)})=bK6v3iDhr z%56ls{^-9st)aXTm1J)&I0%P|>!=oZjJgA7n;vT55knx?K~FmeGuNN$jV(6YwBHPJ z6x1&MIX8>dEe)w>`8p#DIK$N3xy<}k;jHI<*L5}^p&IpW%yIoa)R1Gq-nGl{6-+g%I1V)!R;gsTl_hv=1u2t*voV^~te{wOR^2f5 zSd6sGIxLAokB9ylce;`%heELm0Ql|RulS*@c=W$GOZ{QN z$4cQ-@Ikk@dbl)AqQE#M%dJq*WW=TD!Z!A|fo^Mh*a*~ZZ&**bI1*(kv1dafo-w(A zq-@UKKRp1a8)kieWs^~0U}Aq3}3) z!rF4vAT>n{eba}S{0}t392`b6-jrBCdKd^ih`RpJk46kj8&=V1z)E-4C@Wo&DEjJ- zOuj0?s2D;=>#3Khgjih^-Uf`!Mm*{{#rl~SuHo3hmfmmkFc4NG8IjA$jJMjS@nl%S zlA1rg%tIXA7@|rlWrf9bxL*9aqDHU9^!u+V%EzHTJ3$mMz0TN4WuXVS73aeQUkQsG zN2t^`gK7Gc)9Qw!E(u|K+f7#la9omH%@D^a0%NH{Y#UW>!2Kgn#yJrQFz`4QzutS< z#*cjty_+18qb&w;wLg$EagC*?xm7dNrYC$`6h~$;S?X0A<53~RMvY`$T%q);99}?< zk1LVnBfgxmVLTHyQD3X=RPH_SzrUk**K}#rH3iq)q>iu2A^^kr-PKXkZsxig8E4ip z0Piw(USUZ%6)g%0bzVG6G~1052%Pr~A&y1@h`npKCF@-O%qXT8cMx`y5VUTN zrc}UO^4*we5HLL}?7}fUS7;bmd_b<^Y=Id-LHSqdA!uuO3mLQ12OZc@LwvOdB|CJK zE7JQ6FGhDeo8xZnI})W;9rMAF=;nt<{TYN&?t-15n^|u#UGG>Iw#t&Q&WGRA6Ax7N5>D2!-}yZKg5lyf`&Gvu+i7fy^@N817m4J954RZ0E z-Ga7rrhG0Bw6RAN+w<(zpd9V17oxc9C_k#O?+GcxZw5Z<#c$)K@PSGb!!9-JRzSH{ zJTKWg4A~@*OKiSfdz>f=1rxGUM2=fKp*9@{0 z7NK8o`RU9`fj_S801UQB z_y7%DHYOTSd}lO3xN}I!wnIQyY&GMgPTHi{8URE9MN$v;m!{@=#hLt4w=4=i^EupS z4L^+XUt#4%-v2TLpIup&H`B9GPxHrupy zgIPQxKQy_L?{S|j9`EC4I7%l9C@*Lu8-42bc=FL1e*;) z<|rzQ{2arCApm=?+W_fYZg-!hq~qjB>F0&VRkM`Iw$=ph6tqXA&pL*2 zShk!@35}dyJ0zK06As$G;_pP+M_&|PGDGOa`quUHiMRj56)IYibi5YnNFD0jbR-y^ z1x6ht%f#rYKm1^OD#VGUmf@c5 zgYNw=NiFk2wa9KrwvdrS=Q!!>=e4yFVKj0O#Wje>1p$MlA7Lr8VJAJz4c00~4)@_k zs+=kyyoqPfNC0dT4hz;Y@`JZj$8e-&4pCja%wDhI+jY-j1Te=qEt&+?h1>1rFlXJL za;;@BxJU`c*CuGs+J;`ZL$= zBrI^-a?8b~J1+EhPW7UquY(zKb+bU!ZP4A|BEbp-pRSht!&_gnAtz%SSoU5(XrSF? z`(L|@&)RkO%9A_N77|vpb1lNhpVuMXvII@$ce{X)%UM|d-ZWbJ0{>NYf#Hzl zAQSBtV)XC8*60^{hk~+!ZpZhn4+SEtasA*j$KkW%9c;MtjLIG;D{0l>(K@KFCLwJ= z##hxB$pjX?#t6clU$|k9uW-&{1&zBLrN?gj+`nt9r;!cI5iM*+_Q8~QyX~2XFPZRvzEjGrEJKBIc*%BLajsXc7O@ACQ zNfxuv^(gn~wI6Je#8|P#wXSyD%zC0t>Mhk;m-uwJ=yXcC+E;|rMbE!Z0iwEQcO!Ku zBMeEFUhZe31q3)d)eEVOLJWGfBq+XQc|x-3br+P^#z5&pMJ{LVe@f|R&Bq``c$k940gvCuRmLM3SW5>r^9bE zjG@zN_uq8YD^BvlJ``7dt9cPVZ9CeA&}8pSwgX9I9`SBilY*5aqv>#>v|CZnvL9+G z%T$cZp?Q{v4qqMOi@8DihUa)K43U)ujdd`z;(F?>xFgtHH5f-dZlt4f&s<`T!Yk~) zb2qer?4LfULI^RgdPD-4N4nh8gQRGbIm37sR(gTZG)CJOSyJ%{q-=_~cD%L?_z(qU zRC}WE$Ty5z?D*pK`0y&>k7Tk6R6;pwLVz0T6~L`31{2=X5nOIdwHOUC2(sHQ{_h~h3$h__TRojH_JH(bjSg70_5&l4 zOiMtIqxheqy2U9L_g_QnmWmNNP$^XQ?9qiKmn%3>pVS#kH!TWC@Y@@gTdIegst$__ z{EK#A7CTBltD1b}y+&?@Akc2~BP8V9vetHVC#!Mh`SF!H))Uf#5EkN=I$mIH=Hbb-96y~qD}XgXl9z3 zF5g)%FH|U1;c5;Ev%S1o2achqRB5w8?t4>1Bxg_5G7WmiA4 z1?FjgVS-N1SZj70tWYb#2$BkzwZvo%ocR{iq)kRphkH4TRoOjygz=bsEr!zEo~pRR zeG>-HxI#0O);91&9p0C)FnHU4D^YDutT4$lfjVZ?FfU`nK18PX=MB^-q{K}6w4wL{%w(*~ zw8tA6-b!y$1QQ2(e|M3tlO1Dn!^BECXE1^kT6k$P9`gJ3)jc#>lWG@2V&JH)Ydv3w zZGek^n3zH94YO-ZW8zZY_jYv9I7S;~Lo!68Kb*cr$l|G5Tw_*B)t5|_gq_rWaiZZk zi%l&g*RlgYQ&J~4zbO9tkAokN|C8aL{`0@8Ga*0j9UhK-3VG9-!Jrn1k%2*a0TN&s zhV#j>pu-n)t*@W|{_5b@pN^Jy#$qdjLAN%PkELEkn+Vz)1broVz;^cXF#%3#NV9Cr7X7 zZm-=A7;5sycKWJ;|ESV-89u`w(PK)itr_?A)^9_n_f*A4Ey&}rHD<_=gYH{R(TBy1 zun&Fz&^pYZ3;6KkW2Fx8^)rUGhO@*2g&s0}FuWu}77U6{O}h$JB`R33Bifz`Ien%x z&kpLF{?2O4b~ZP2!BlHNorwutX~*I-AqT|cgs2oW(Nwc51%y11iYwfGNS{J^B8b_u zTz&njI{SyUY09AHW>2xs7xHo>SxsHWMyD9y*3kR#THT7~NF_njw3<6a%Dd z;Eb|q5(bGuZGxp(P@UDtFH6$idx)jmXig_Vmp=UgToRYZSi#eyiDHk+#Um4(OsY7=HJ_LuFtYc9mk;TIpfaMQlz17M==e!IPYOs%l22 zDA}6?&78=!`fVe!kSkX%F$UH;uI=>*g%vxSQSgFoct=9Gft!AsNfr$w_lB$;QOH0r zd^bsK4M978c+7XG_@-B=NQ@Gi{*rQCN=XQ9Xnd`dULL?<{>1N?@FU#x>6RQivc0Hb zTU%R--Y~RCw0kZG+3jLCB2LI0Xw~pcy~Il{&~3bWJ6lcP0Qi-`SWVX;i%pS`3SU4L zG>YzS0JSh{MfgexyTNArNt*r>PAH28!B{=IK!<)x5*;CF%a3G81{!ytsvFRN*mfsD zBnsS)V5t~}nhZO|K8NFCM8W2p^M1wM0dFf=4{8BX9;ciCl`W0I>ML^72d|tSIzIkp zk5g0-c}g*Cp!!OrBMgLv*s^IJ5+XNN@T9%f3gW&%Z)M%sPKj%Qyiy6NoVSc;pu8xJ zX;%6lNF^{1SUvw8$r9+acCgA)+dDCLbh_0Sysw7NnIn*=vm~s6eLor&v|-}(q5Wk1 zcz5xMm|>uUrH!d8KS{H>XScmuC&G{iJbILNNhfINpP(eNVelHrc zDWi|E&i(%K7p9M@n+9dSL!VdU(O@~O4ExVz4Da{$8;Ue!TtW;G0KwR9-YWzbIyabX zdw;}+Zx^3#ySMGk$#_ zF*9Mw_m>^pS5SDb+DkZZc?DjEGdws^9L{kk(yyyKa6EucheE@!Xw9$YvQz`AB<Ty}cp3HW5Jpf`1mmJjJP5?!Va?oxpn++M75{1QTKr zbNAalfK!oeePju77Z8_n0Jl-pMo##0NNcEE?JptR^;jZsJ`luWk4&f0a&fnsd%G>L zak~*cO;K6+5VxItJ1;p(%luMdQGziq(o{?voE;;!ug^bo$E|(O7Z)_Tn$;-oDS?qT zumi46KXb2dxu7q)(P8lB;;Kfk{qui8=u7uJrdltuA)bc`Ee6~yz7kGiib6?ra<~s?b()lPZoSmdFwxKsSM}htOS)CTTb)r;-lSt(Tsc*9lWo5 ziT9h0h&Y#@les{(S$=#+;OkKbTvjCbO9(SkyW6dzqu4&>AxKuP)v$&!JAgxRq;t}z z`@~ov`Ujp}(z-3Uz>@||lt=>qH&B`uWERUN`FFEl3JQ{dmOpgnO zq>#t5i4XY5jfW>bnxLQUR=TsHP|>jf<=qQ!W(z&sF3{?rO!Lu$m=7y%U`N6PvHZOB zMe|@vPg%6+rNK!@25eOQEw!Xk!cf_s(7s#`Vm*iCrq~5OdYV}3X{`(Bi6KDVIIKTn zyYlJdU3V@NX%wF*ZiAsqlD9m;WlW)-J#;QpNrlpWdl`5p>AFrsUv3|?wLD*0Qw7J7 z=xHK}kkNrbzD#94LVp%GLSKw7o!RgMpKe@65D|<;-E;u^$x}aD&pjHdaf2tODi}V8 z8`)6}-N_yaxSYU&i@Q!wAFn8 z-PpIRyMv!0)9?%({SMai8cmK9Z~Oo(rKyl9B)uM*-45uOd!(g=!+q+J@_Oj)_#!;z z$3$mwtp=gZNvMD!Q1s|EX;tJ|FtAAJV-L>%OKtJr&rdCgDs=W!{L12_| zlG}&51IFVtkN~UKNQe7C+ZLL&MhomLb&Lh6?BW-x(>W^&1GdcP;O=j)&%9xV{&$< zhes@Yk0xu4wx*?^V`uBLL6F*@E5m{+f zOj7vh+otM(etI2Y`KRU!*FX`{Yv&n3DDeNFXbtt@<3cAK!k~p#qa%~z z?4YWZE2)rLmX zzr57NeUXknfW;9YUbH1a_&@o}Qv`s_V}Th9#b(#skac+a3W$(vd5B~Yn;XmJev-iLM`TY5-yuKB9mqK?$l7hZsVd6ZfV^38b$P$zGAoU{~clc z3r#2?VBJ@@9YPA}NVo_{+$O1>kp`~S9e_2gT}!wlgkvC+tTG;nGs0WJN2HbS(;)Sq zOrf1#>k;T`Llnb5$q}GS)8M$QxA-^&(zgyhI&ZmE+IF1l5p=d2hu!bd)!V%4Zi{bp zy4j?}TVxWlt#%!=S%0{?{UFe>9p^8R8i=pH!XzO6Yq{Luw*^God`zNxS_19kA7=^Jnn1@=P=(~?4F8^l?TLdGMyBf6Ds4}U8|b5MkX zxnyVXJwtQ3jy02?kJ-jRoV*wi0(h=FeP(G5WLrlyxu&}IqaWmgB;-VcyLFr%osE*+ zSq2*K;xl=0rQviNcJXZNPs_3%XG{d)31&E3aup9}<7{~PdnP5i3^5sVQPYzNO+GJx z0!T|C!ZQWzF$t5-noyR&2|?7v_-%|)G{|4*Ma9b9Mu!`jaiR8=y*q^$s2{Kzh=YVh z>F7Hd5u)=|dr96*&L2i30}w4-V!6^9Q1aMJGDNz)=3)%WYX2r$}n>BWnX8R=F20y7DY^N7LYhm$OB;P6S6Ji=N~2(}$3NTx>Fv5+{Z1 zL6|~mUNlPsnC`~a#pM={PxpG&e7jZ&bPp`V(MztTd**llo&>dW99Fs;s=?eMyV!2> zg?AlRvOV6*J~HrCM}9ikz}%LHcP@n5n!L4zWLdf1AFwrxC~jtdKvye7@lr7i3n(La z9N9ubE6Gkl@NjjGy;8FH8{CU4GF?OCZzQI=SEBha`q z9+RygfaGwQ(a>FuxEjPv-2nWIUBf|9o>0Fkuu3lZHCjrqCLXH{Ij)uv2(vk~c5PBE zPPD@_xu|!R@em47JNvE$VcX@pFtf8yFVTx~_m&0wI!$O5X+FUghV_QOYcBO9!w;cU zG>(ypwaDj~guNR;w&K@|n1CjFk#QF$SE?r7OJ%iK5UeIZ= zQpAKCD!L<_&ygnhNZcKAu+P;-03uhTbYM7Sb19NgJ8U7AHSR!>Cr+eO#{4EOQuK}h zy{1*<>5fumJ3L?ka*0ej zHS8sQg+NE0p%GyD`uW2XAkDLE_aYVmj*Z71Gq$z^<@oZW)`w82TBF+IO)-&?BU>=B zfXm1x66{;YKB^tr3yw^{M5kNK97zQY-*qjyn zPA}lM%+>qCk?q%2kzACc!jMOk?2$f;SC3codj1UIj@CC^PxFBP&(W{Phc6Fa{S@Bu ziuKK^omHIdJ(0_N^uAy1z>j=sn!OkA#%$)^T${{rM;?^6G7^zk@- z*I^Q2HZAw&0K3ypQ{~Q96Mi$S+6Pe%p;R@yRIL+?r82{JElpeIUU{Z zVTEx|NShIim+$~y0F#-YBb0E3_ED7AWp8OlojjqeO_T~LjmL?~blY^M%p5-!d6Yy>B+dXSQBB-CJBYxSmv4$q62ix>{^mWL9gWR;y|$Ed(e3FzUjLw z%M=Bt({~jRgTDX_1~c$0d!n>vjvE;NPuk1Qd)xGYuX!YG5+=>*yMSxM2(Rxn{A`-A zp+gM5k6Sv3KY=e$>B%cXShK|h>kpnL+U7o7U)#J zriwF22)nIuqYFQQ$H{eMWOf2LYuUy<7JOXXZdg?y7rNV-06cI{4)4NL9D1EOr@LFs?Abn~4qu$9Q*r&1f zglPDRn|qcZ`?dM14;_m9VqIr_Q|c!VlBR|_BRG|PC!(+Lm8xXz9u8Y)fzS)yq@~db z3&^gx|Mx~Yv00M+4J!uh8C#t3v*$MIylnUvyhy=_KA+WzX3fxpL%iqv%2nZ+a=>YIt;El zlt%toP9k4CyraW~W3gS%X~wgPE7ci)Izhmt5MkOSk`7MHZdDPM`XvpKRB8BaM#G=L zNBIQt5cfPYlAj<>4E^r$7JA0t5}}aNAD#PlT1Mq_eIVVV4{Bm!1&63^q$S>g_*}%< zT*+Yain?0{XmqRPM(k`=k-^CA0b;R_1RIYo)Rx=|vf5MwsRVU=$MW1y=$;8QRBl_- z(}-07w1*v){OzXc%fe1l7OlI&+|kHF!R7T5Dr|2B0r)dlSMa9Z5obssNp|({WBo)z zRbN>o4*2K#!ub8~kOSv;n2r?voYOjCr3T0j^+en$+{2ney>x51QYpliqe7-T;R~bt z3T+cp*cD0X0e*|Y4ep;M!V@50m~zMgJT;6PVX6xM&$LIi~fd$U?>wlbFpUR6zU%4eVlX*q2nRJ2cf zCHd`uejJ!ku9#LEkqM`TPH*-!N0~u4Sv31FCk6SZDb5`FTWBjH9rT@|W9`FJf-xv! z42TSAv|n1i9V~@Tln$p@0K9g5jJiNr9pfeXW=fEAhHJX)Y2063_jDG_xgiJFClwR% zhtrHHji8oLG%K(`g=sMN25!PBxC>RC;!tMNqfGftGO$(??{)qzqLr@=VlUDYgSAmcFms-JhJ!UDPO@{fr1PYrYo{oUZzD8TQ59 z0722Yz|!oYWh{@9buqoe*5hOurI{aQj#(=c9CzD2h;rdWGL6Afi1kIiCc9QER|0T< z-E&0S>vSNpS-~C$R~Xlfg5VWAb3#Y=5Q{Z=-DbIPi@?D60TTWRnWL~hgF~%8Xg)pb zq!1v$3?P$Oa&@M%u8q;L)-r%%6pr`wJq{B=n$*~IB$R(xiQBCi+8+i9qePGhV}hdk zQ6)IS?VN(W5VXb6mZmnyoOH`+(co@^3PD^`>6IPbUXMkPu>}8QzUVt49^u+MCHc*! z37`+x(rq@nrRr77D{>04UyG+bLe|%F0$gIWcjNJ?U4s zy?^tqX?ch=^PpWecDgu_&_M_kw5p1S$ph+6DaCzV5vdDo6}8U>w|FrzXpCHx{;?Q~tEf zf6FWa4R+J8KB2`z*>;Uah+(d-1(J%*p!u)EjX;DE(7IjKS{WfVJ3tE^Ncp-Z;+(TkGn@JWld^wawk)T z#?QCm*Y+E+*iY1j*v$N556!8(+jVcho8M7)yl1!~R<=0Hi4pW9!s#r=tIEkS8wS7R zc%I0P7a~2Z!UHoL>|{rJ`UI)sg6Dlz^bjEL2F2|MJ!p+Qi!d1}1^Y_j+C;4mUfy&# z^h0I$D(X%2LAikY>F6d<Cv+sYCkY0W4Y(vzv5-XHqE0ZQ?xE!`h}1?j4khVc2#TJn?(XK<FI~T?6&YHFeZ69Dd}6v z?n0t!7dx=7LzAG?$(_m7Cb?F|6y1Opdl@bEmY+1C`_&A3u4c#dq%tH`QF8iKFA<3y zi57t>Fu*^1Cz>oK^o9Ax^!YJKzN=L=sFTVBkVvDeb$C50rYnO|b>iq}O3)NXIw0NdlT*vjbBT!?5g zV2zfG`4xk6w-vU-`$gV2!g-P1u8KKvz)tWt*8Gn%5|k@4`my?n#rb#X^$frsH>BV@!0 zZe|x3cNj{DQM}MYvFou_;RS2(QwXPPfe&E9{%_a;qu%gqerdR|)~jc(QUR?N|7rd(y-=$7`*g(=BI-QGMqZ0VSa#Xi1ZI_$d*e>BE&f z1NOm2fh)3&AlMTz7+PjkVGbok1XB!Gp;k;ER-tJfT|BBrt-a}@M72y=yD5-7U6 zS@yUkK>tM;`CH3L(G-^cS**K_VQs(iY*vm0WQQQ+mykO@%xSU;^P)uL%h0SI+qezd zHrX-wBX^ja z8imt_1T`&l(@oqu`{i|{uvzR6L+D>X=YrdJ^q|c#`I@!m_Nrh6HI^lg9tQ2&c3<6g zy7WDxcb|dGEcsNb@35N1$m|5?mjR_Kv({4-dUrm3gYkTjx{@?dbs?8T0QAi8iW|ZX zag0(7YDbY5*pdwGI;CK{e&c${S#}Hesax{e(n46Bc&E zr|mShjjbM+0UjlR%{E`vH({3`i)N@fw1Bs}au^#-fA6hEiBzCvJ8BIeEvb^Dh}(?< zHxNbqXzI^8%)SIT4rBW^g5Y#fb*eQEn_jFiS{1!8YSKZ8gfVh)YwUl?%IB51G*fB^ ziG9soUy`m#i0ZKm;1{7-1_ys!2d2dRG`{&38%!f@8Px3hJ5sLHu(IE)bM%4w zeF|m|v-IKc<=qaoow99)Rn!-I(3ODeZg1WIuH)1|wM42gFEeKz*BK|vd2z}2UaAWU z&Mub*Eh%$YL3LV!}7+ZNc+yO+<-j1ZIL`O$=r|r zex__zQg_08xj69qJ!i`;l?8qcxG~Sj0ziqmj6((K6mQXPa%J1`4@&dI^LQ1_98OL) z!8L`bOMtZ!ua2mYM60gh~UH^gXC_e5YWJd`Ay?LHA@=CVW zyw9`F9gl^O@-vv=^ZT=-#Vdl1TW=w^|^D|Zi`~uCAhwcHy_mGGB?Nlc%u3QKX+dQF+ zqfv8Z;Q&NSLr_~7d?Mc7box7=gxM56?{=YEfkZM)9fuR(VPM2v9GBxha@&=U^a9Ax>4NvQ&3@q$o>Q1m4Q}i_>a}u!Fjx&gqP~>z&HD3`b zGNDO%dUXpjno9e4Z?<1W6aZmZmprI*JccE4Db<2!#{lA1KGE`YFeoW313r4lC6)ou zmGaIFM5&k_=ke);h^HAeO(kj%nINun;K95KA0_pe>LkB~$K0hp>lO~`9Ux(nmsm=9 zV~&uk?DpzyD0f`FMP&xf`!fK;6JSx}4X8Sm>U%hwLWQ9uw7B%(++UHy^8VnWe`TW^ zF!5L!=sk(oC-VzR#{k0}>8mgjMn6@3!a$ z=)ST57dJ~ep2zl+d$Wa##WkXjaPj0D9HW+A8QrsLr1I6x|Sb;=S5iMHVuqO4M%&Uqp3n4s-urhrhxU# zGatFZD#Ni+L~CQ$fRv5b3#6^-o!CJpS+Kh8uFo^798FqbDmnx=EREo@>9e7ToIcN& zsK7@9uHNqmfc7;wmmPnWIy-$|?GUHWv-UNuCTS5!W>VbX^dvI}qvq2AYe_FCwi}SQ zQWd1;f)`8~~2mrT6|3P@A#YC3s z%G5h~Zs-y%8axQNO=9VMtECj?)0;aqcN6L5%hs@t$<0-ni|`X{(xn)7BB<00o?X z*eQl@$=s*+5WZ*HbgWP@+iHfp4|GI_&m;vlZ@{EMKadekpZ5t~>@c$SW$G!fsLS&9 zxJy5`;^SG8dNi)4&l?6UImhG0*0kapeYU0rIKIj6#0ZbJHhlqYpcru$5R}W7jikt= zoa!}VW1zFnfFmhN^s9$O!{M^yX}H}nuDI>K7oNt17{tVmT8E}cd(|(}?f}5^vq-aB zbjcmLx0j*096DmD0(aEj<89-vSQnQ*d7u0+1#+_QWKapANndy8JI=W1j=SRRtS|wb zX=D^K-rXo|18Zvesbdkvhs%%Y`H8lMMvrY~G5YD%eUNl|P_Psk)Y@Ix)>h9Q98;zM zRBY`=r(5WNi^pgmo0hCG+l&>TZ59EFRhrV9>_q(Ksvo&m z0*M&U)nlbpL&gYFy+Ekva!I7t`2$sbi6KuH$9v=E@Ze*D03o9Gg zo#4#N)QGz0$5b-HId}R}g>mAH3IK1QJxiyLz5#Ic#JNo(%Lr_d%As&tInYsz|{w5?v+a^w1f*2 ztY0w3KWF#g(~{vAm)2WBjc?|Pq6{w)2L(f$bl0FsejVp{6|5r|ODA9WgYp91=c6sk zmq-Kv3qO9XBn4?IbWdrb`{%pU0}P|a#bG;3C90Q4?O3N%i?oYw2{VBuTV8Glq7^#Xeu*1Nb8`j zLw%bR0JxsZiAwbpx^P}^*nPS@S57J%vm~=wy1`>~)ibeu!FzUA$VwL2W5-Qs?soy_ zgm94dW<3pFUv7q1%H)KB#mDv5>EaTx%Avl-*maJnfj9Wf329vK^rNx<0O9Ly^Kjdw z&&bgSM9gMDpU>?i3D1L<5rGHiKzHo7g+I`zs5Wf-g{qQZH{ z4D(H>ifFJUt9fULOx_7&Qc7Qc?=NYqX38gDKtrbxV7eo&+A2ZHE|fFg4f+Dzj$uG> zy=DibegKl(k<)GCIfraYk9}?J>h|)gxueIH8A4@9jqz*K{i?vhl5Xz^3bNzRqH?nG zdB2%7WQMuyqfHak9I9b`eQJyR#*6{UbT>fH=e)%4M;@hv^ZZU@ZobrIW}9f(1~5x$ zONT@omZsb>1piqle<9MON68iPnHuuE)%fBbql-Aqc+9`OZT%g*)nrF};t36>ibo%L zfF{+z8+6+z^f)dV(ZHh|){?m-{i)qK1{SIrvIt0#u7SVvk2AFGPA!S%8H(D?IBJ>#W=hxod*>6`%h>0tK(fP7s$?(0%z8Vh zKw2k5kSx0uQ)0^dzMqcu`Q*`Y;|f|(NSAdj2ZR-$l1FF zFcu!?&rro|(kVrsxmTu)+i%lb=jBnbJ`07^n3phexRvjr70Zj!18vW8{(x1GXoV^<_Zl>R(}*#e&E5Uf(JMk6YO(?xMk zQFvNuyH(eq464*nv0tzg6S0jS483k^`i3S_D5(y=)yUCIR6CFQ#2di*)`cQCGJU>S zs#ojxNDQ`)t&!jjzjL%GG)>aZJC%tfplgPsFZcK3H4)HHnePMYeLnRL2^egWBhc{_ zEJp6t!~wjx*kcrfSXHKIDNA>HOrCVsl0rrH8d#}AOZ-Yo`bh@Z{5E|sDU#|oj~_#7 zd=Iu;K{RU2Ng)Exya}5YWI4I&j7g+Fq)Dzc`qek|hfK#UH*JlNU2<~nJ zDEK3j8hFKLk;HdolD+%#g^MJq@c&7w8>@_6GH$WF&Wu z9&G(is(CFX-0d9OB-=qGL|h!)qz$E8`2HVzFL0qenK9*ssRCc$5wazK@Ok$TLsgqb z=aShx+R_!&PcN8iC(3dyx;#aUmd#L%y;Q*P=PG9lz24x!HHpQf-Jt694VTe`x>9Z~d^FOTy%)9G^X%&Ttyfo|Ugcd>23zmt{&Z^i^VU0jbE- z{WmALa#*r3hbNL(9e+v$GJi=Pk4$yC>x@jBQ4|Toyodho9`EikS}TH|PHX6$h}p%8 z3#DNxk@e@+A26D4f95`13})oe*E$+wWgRryy*&4c#HF=@Lf0@ zEN~1a&?+5NA-J6^4 zt8LqNoudJ##0kWk6WTcGhj-Y(4n{9bo2|{xSfaA$f`8YH=sya%*scoCvd^x zKR4sO~O-$~l4+GS&G zyRESSTD!kI^7XCmT5{`8ukf*XQf?3DJ<=IY{n{jhr7k+#h-UkdkCi9BmIaVYku+t@ zjoQ;^o%)BTf8!;3`uA=GzJ0H0&>y|3N|knnz=jLq=(nh#dT3+}1zgPwh=evGyS{z< zo`tV176%|l%)$aK@}~wVX0wW#E;c-QeFJ#!!GL8w_sm40U1< zabusYB(K5V0~}1g8C{(GaMk`zNg=v1lDHt6^)XI{s55 zdl);@4Fm?oT)m)lPLPI>ug$H33wW@_0COMZPq0uN7`lXhz4yb%{f7h*gA(fv) zmBmeZi%WbZe1~#2#@9atqueu^k3s!TKFyed^q$dHN-@=Nhj^3d_uB)iZi*S$E>lJC zP}2=b^n;Z3*O%RoY5RoR6O-Iz7Z$4;MKUJS7pPnztLr;I@!P$Zb>9n@giNU2y|-!q zlKKApb_(|QcX%F?IPxg=vhbq`XZe=`OA&eKf8{Zmbg{b(|Tq z9Lre%IzRgd`c@ohjSodyrQbBy9r|W z`f{mEA2$Xp#l`gkz9iBgElREy5BXAxe-{`%rHfH&dR)21-82PQk%W;k-K-9%(lK58 z?27Q1(ZQyFM}K7xLzj^9Nyy+SGMGMEImw_l{^}<}coNSt1kK({QVl!nLsE(68>N7_ z=5<8(kBgwB5w)!w5<{=6>3Dh7;7>0WJcPOQY!N!>Z<|b){(1wCWAF z$ykX5Q7Ro@%|oC_wbSRTC=`y$v#=Hp&voGlgvI*{BSOsI@;YI0dcUWSC$JO0Ui_Ip z#ZFDflhvtPHiYLsSiP&M!Zk6Mn?#j#n27f80q8b+L?3G%hks(MLwr2bm2(SYz=C(% z@%i+Mg91r-w0e1?JaZoX%R57#Ha2{wU$&Zxg=xUBS(Hf0# z8Tg+GFL2J$)d~}Ykyj2kKQGzsZhBAj59tWRZFtC59zewg zL1Xc{CJ#}419R!Sh{T`4ci=#9Y^ZEOj3!-~Nd%UzjN*D>XW)7U?DGd-p|`OXdwqWgve73Ayyt;C zn$4x_CJmlwYd(c%R!T%fLMwMO*I0gLei+OWd5)#5)2Dx+0tnAE)Wl3ZMII`vTNkKg zQ$Y`8d0HL@6X_m3mI0tR@%UaP83;;W#VCziUKI0ry4-OXIJTXtH|T!z^gu7G@6cFl zZ4mu~<7@JoWYA8ZKj7_YvSh06v&XA{5Boj6n*l8)@CAx&x6^Jhf;NV%<#z*lxD$<9V2u zX4b3~7zvw&h1q7EU%aZQy)z=_sAKGNy8V1;7CPz5AnR_s+NzlBA5catjv6nM9J{do z8ohKR%D+dX>t9m`B{T6yaJTUd`2U-uMQBv_x*o;ki zdu)77>Q{AHLP>x+@j2uAp&x?UniA~va=%}!UAVeqUF?}1jo2kud?-iyc8`x>D5O5mp?vy*U0*%j zbP#$&YP>|rF_{K`4F>JTY13K>Kj?8lQvF=LWrc0HA)He8(IJ0>$NXkR!d#CrlH}a0o z(d3ZrGMC(p`qKQysiWl<(LDqJ$ZtFCM=D5%tff|N1~>e|1SnL7;kI#OqF78}bHGC3 z4HRzApD+r5D@GP4iplKFmZH=>hbo`|luIs1E9?OLDCI?K%iLlPz#rNy%V;zDQ~TRJ zzy^&mI%r$^DZz*0RN7Y@+U-~r(0g7*tP@>DVj-6i%|3|a69WA1;RGQKvUJCXj-TLw zxg*%Gh|#Vng3pe8uwAm>I!*B>6K8TwzMbL(c#Lv}7nq(jz0`V*{}oL z5BP>@&B#l=sR)n~)^aK99;@>?Ps3*MDNoNM*iY_0)Wckxw_s_bb4K$*LVDNXdqLO! zX|rCq(2CEoJX6m=(61Mcv2D)6?K1LMBIW<%uP5Vd0q1S++~9ybw9Yj!Dns)GO@V2q zP+yp2bnrNQbee!3E2D1>CEk%Nk*KNYHQW2+3U=oYF(1ISli3G!!X$$CR`Me#bx#+| z;Whf2WJE9Kr%xxd-Hx_hct>nqLZ3&RiS)*$kwYerg43;Z_1UxCZf64|qvbEZ@B7_e z_o>THXK0lg`~=Idck*2{p)UcX*Y=0T-=K!h450{iF;&4>t_lQld^qScpy~cDNThJy-$ACMH zI%qIv<5}JXLnqGzV`6^4qI2#4VlwUffzN63Iw%20xlK(IV*+k^>82bAK8>k{km$#& ziqVz-8dm8+hP+p;L&R=C!=G=bZ;V}--b;9U2j!r0-)P!#8XXiq*;kane9^B?#egt{ zQSHexh|v=+fMaE>Y3hC|s-LJ{ZlX)9ZT=j_SGt)SjWav0>3Ir*_x*FB=&c>uYS$ z@k5VvV8~XLok~(kPB_}}1BGYVw2J=si@}J6)uwn-so$O4XElAXKM!*6UO5!WtAB{dC(bS7uLvPy#5f<72~Lex}ENE(tAi* zit-k5M>ZL1ko$;Bv6ay@FFM_arjO4VtUJ3G!*Cn1UDR|>PF;?6AZ8NI9p9x_Yd224 z9f@8p@@Ar_TW1z1VGpB}g-s7GhqPu}o7%Z^z866f(2dqQAF*)J3NSMHkKJJkE*P9_ zK+6T6Pw&_)po-SD6P0XKb+~EEM%<^c;a*4Bm7Rbu7IEGn)IZ0gEiOmxt1?pZE_}`Q z|E*k2HP`9o02s3att!M6;kWWjp){8#4+_tf+sA6Tlq<_?0~TnncJv}_zqub1vcsve z6i?j7Og50|v=S>PmSp+D;|L$L}m%s~Bq@9#*wv;o)x=@!Au<;JVU)1v|cpWhml zo-|^QZnYZH{Ap-e8d}A5jdF)DtNjorf)+&j+DWhTy^IS?LGa7n?Z0OiJGPA2s^*!G z-gwXAz<&d}6^y%_5qcXTSiaV8lLMQ?>7k(E7IUbRACwqV8M%lt*)a=^_U!K87{n;W zk5?r2@?*jhDx~D|PH~n-3=v^txI~(P_Qd|2eTdD-np&tQM==jCnkH%>hcQ-}WZvR97j}{4lC&ug7$bqPv)KIOm1yjah z#9^NK^$LqA%-Qr{c#J9H<2Sa!bz45^vOL@^DQN@6i`zOF;Kh-Jc!WMKIp>xjHHVGu zE$t~hvP+}xV5i-l#Df`ySxXWsEHEaMYE~PuAS$8@&~zOQ;97GS-|*q$aCztu9i^(z zNrFcj$FnISM1?Lj0JgE>vyNIz;w83n8Ye_aZyobw7ELYd2=2!a3zh=TlqjbB4z#o( z{Z)BE(3}`)S806gjusvO{4R=4*`Sa8dG)9_^aBZWEBgF#Q^WE_6)}S{rF9q_iGDUS z5Z2Gz#Qb>Bi#_BU2u&A5AfhCkqw1&R>hURqlSljuP-9|o%IXo~Q|(lguJRi9t+muV zbKQ;n%qL|~cOMvXN4(c(1u~=x-$oLQMtBGC0FKX_2q!pspQX9cN&{*9i?2`?G(E)& z;y}Kd{}HV*X~xa_>nV1W2WXc(eJo2>de--Bwk>dma4Rb=6`3W+q9aeAK41y7Ke5%7 zS|~lq>1EC&*wlQ7-G*PP16U7`C#lM$vti(Uf3L%o*V(+Rd+0}6am8We=O`%Tkm`-i zb2qbpYY?r@ay;g{(W0vPi2f;XKDLSvigQRHRN}*KHTpq5ar=JhhEW9{8GL5N>KXog zv@4TWQQo^}ezp7Ff8AZ)GAAt^DZ$~V+i_|G3nNRhhF2Y!!V=-GyQblqZSFKc()$hS zm5#dQl1vPiqtBVZZpLWsS1qE7H<$#_-CAO5_`lEQwgO68wgw$z(oGn|zi?+^%R>AW znV?%5Xvbo-rQ9{;mbc$K`D(^|NzO$3B5>}MnGj2DpY!Bz*H5Q>hgKARFD|dTG6N0a zXef((Vg^;W=&{pSvSOtm0Stztramlx6%BAW4xW5GDt9Xbst?qW9g{eE4AY6BDspR9 zp6I(goi^KkG3!Wr{|)K`u0~T&8i~eLdPT;M1O{s!(}JfEm;Xi*+>^j@cl%6>g(z=s zDiPod06Tr;*h~pf>CXN&)qIYQ0C<+eMD*^s<>@oywqj8c3^HON{}N2)PX~u+L^FLX z3@{QTf(Nd}RV5?F(l{ifPDaur>drf$ye@aP1_EXP*g!~JxulUf{s`jI9i14|`gb4; zN4I_VdXpmI;tERS7jKAEB(w#a=KoV;oaRd%T@+s{!$jX1bceiCiElBo3Kp+Opx-De zx=)}7_*2PY`z@pu*jGM{T$wT6ZoQYZ2jIvOzty;gjU{_SEWAdmyG8hP@2@z|AblPD zL@7vxvtWCCRruB!jgPwZ2<=b_YK~I}u%Dp@CzbEN{~Dyc#PzR3FB*KxUcUwrgC6dw zGq?k^i(UguwzZm>>XB@a&Jh8C-DoA3o(a%DC8eI52F&gE?aCEhX3X@WFwWI zQKuu|k27Yv)zC(;%=S}&T$PJ}&+L};sbyD=47d!u=?i}2k1Ab^zN5R7*xXTX)cX7LW>=|jwAvO`!`0||Ky0(kuGIj=H&5PU zhg7Y;jtyZ45u-{78K^J|Ocj%xX}E*q>2`X|mz>gK2-Z5AL*>;8b^MiU!|{_)()2ml z98^tNdT>n z2%grAuF`F)hgA-`LYyba>Jm0o_>+vn`;9abR5$j1eN@ z$sSzRpMw?CXXCp^BIY6~YHmJOlv1jwOf4QLOLbsj)h%qP<+4ZI5fc}~f4uqe`K#yu zNjdR<9Q=6fzy9;Tjt`H2eg2>PxOaF+7F+#l&2hfnKF%+Dhu`k)9bmNJly6?#Q_^j& zwQt&EiyDXLsk;2caNJ=`%{N~i{r2MduZIT=V>x>B{MC=I4_+PrpTdAmh88>g?fLT` zU;Jl&IQZ}9um5wh+DazuLDRCZU6a-f3m{K})~uk?@_6wMbACZPEQ=-;>~eJS#)vN? zS|T~Qo%1Jq%=Hi2Y)X_iI`;5*X>u`~M)5^WppsVF>}SK3M1CT(2r?~)>zOtAG5FA4 zW~X1E&%g_<2r%2NE1NlzIUtFdr~mn2df^(LVyNm&pb^0wW!9&|Ly-g~d1i)J_~|+f zUA?vLf~V=ruVCTy)*ul(W%cqkc%wS1{0+84i7W0Am=SB^k+!{d>vf49G5_Ms=B5(g0@H)_i$(NP_fC~5N~_PVgGnA zKP30AIz;eroHlG8EfuE0HsIxI)fE_NJz#rdYyc8yJI41aTWq&oZx??NEDE5(C@JqT z=yPWBRR|XD7WJ_oz%uO2TL#&x+dKmUMurgtk88GZllg6H@nBRg#wh%&|tNgzmRYzs96nC#w**^P1CR`x!5=xXNZsi>x(*hV9QjnBhmg0@VsV=Y1P9OPOt4nz%Ps zMs*d#K)@A0pCfLyoV74I9O2&FW^pHMK2%}EzKMbKdt;37$0}v+o)WX@axC6y#Ao#A zU8nz8A|-u#x7lvKpZE@5xu;AlXy?vw!17a0pF28l zh=u`!!WVUl95dt2ZV&?zFJx`xPkMtt_53Xuu+XaDiUPGYJv8SqS*_hA!g|tGE&Ht) z^^T07nG~zKPsvSNvy-X{{{yuaKwU`V2_N8VEB7?UeAdwYb_vp`3Ki8s#wra`;J16f z;Ab#j$jB6u3!AuRB~0%DYAAiXqolhH)!@O0Jy+{RlRz< zm|a-F{UqnT@&9mo4#UOpn?B7uKc1~fDed8dJy|Rt8ndjh>uC46VXP`JX0Hwu4xp2w+Y8 zFD-i5Tc&>T96!d1Hl)$(1?|Mik;>tpc#)zdEV(teUBh+?gBhuBs(lcHX4k1@3*&l! ziBt8%Ruy1ziO>HP4PbKTaW)OEi;0V6q`T)4PGjVGq`WnpV*Jw7@RjeG|B>ipC5YppkC5y|C39!UU0f8{6qc~)`$Wt* z7_`Hcqf}5-;<$m-p)`SPjrlAh8`lLs$6g@#Ak4u>hT{&0MpGk-^bk*c{= zGw{`f6LG8(5%G41URn=NCmnEp24UixlVb=xZ=iDT;<2f?WaHkwnm_nIqGUc^()BZ~ z$_d`Z(FN2^7d@r-dWQpsK&o>iXp$0**m#S5ri67}u7H6qI20MfG}(!Uhvc)FE$;I3 zrF4uK7#_|%x8|Rbz`aHj-3o1Ko%_%liDP1Sn>^;LEu=<2bQF))Se8E{X`WDHX9^Z= zAe_=G9%LWC#}gN-Ap7l}fhHzI4>@oszdVht!X@<>q{MUHar593V@n6K!gx44nX2os z&ap76_?1~#+Aj(+9MrB8l3Y(zxj{DC1R5a(Q?FIC;%ek}1o(7&Wf&$=-IPRJ%M~9j zD4*kG(KBf5wm;V$6^-V`NJ7+8gL3-fOqR1ZGbxo$qMt6NgD70gi%U(A2&!wXWJ@JC z24TA=g+H%vgK#jcV*;>Bb4MBnC*JjNJ6#ZN0RyA=`!yS*cu`Ojy?^@mzai;D9}JV} zGqR>aOaNvXsTM?Go2&4Bu%~RX>4V3q5HFgqCINL=zC6s(;kXOePx*z&^$s6tF8+#3 z8vw3gd=QSWpksH$?mlC72uxcGV?18sfCQq1(eAEM>w7;XlKs&U1ok4(2?Srl`1g5I zFIW+!z{e|E)~f}s2R_apEa3xse6gAhlC4{%wl`d4KVL2Q6$awand%$?6K$P8xC5?a zZ}x#wmMJh@O+z=kyH#yuy;AI$4fRHaQq7st>NUzw8b38P`|G_|A8+m{Mhj!qXXIGq zj^KVkER=HuX{1}Q`b)sF$6amHMiyH3RE#gDt99p7&C6XcnKm)yirMNbByPQHTj33A z1bWaDrR;DEP&1}R`s3K^3zDe+e%=XRE>fo>Hwto?<&IzA#ynlmrl6BCpqS3(-n5$K z9e{tPzlShghwO)NX~a9PGcSg&{MDjE?AXz(w*f?s6EM-{aAmI|~r!heUZ z8`kgkk`4Ph0Q&AnUbWkcHgOp<4qM=j7gkpU8$a##)(zVWkx~)m7ExdeYLWGBIGBrp zg{9AQ3^ix!u0~dwD|BvGrp`rZ>+h4mv%%P+YrQV3_V%Vj|5#Bvy;rEkn3+XTVM49E z1WG`6B5W@An{F<0ukHS`z+@gVM*KC?ne@X&lv+0{W!5pvTVricAF4nzxqalRn-PwS z*eamY(j#b;B&;04w@;73!<7{Wfl9R{>E#IBnSjezYJf$LzNx}=t{yUU*n6%EN3--0)WNecf z^eghT*!OOWY(Vwjd1XuMQxY^3%Bv0F)XkPpv*r0sAqfYj0SqZZCPGSphR$$vljL+_ zM6`t+kXdOE(l_lfBvw5T)AB=UwA*nukp{j^E>lzti~nm4!Z*OkJP6(x8g?wbjVFj0 ze!vomrGVm?5_DL_YziI3(p0@g5UCvzFPM1Q9E z7GwnR`sJI+l8SREG$@m(R#{yS6-unyEZ4B-n%CE+>Q-PC_bB-hAN2f8j|~S!r@-Nh z*Q9iNeROkd!?#>8k_#7+1q<)2-|zK1O_h;0=>QSc{RwCAFX{^*OlHW*+8Q1$&^?j^ zGd8}1$chRy?Z%@w5yXL*Wj$3B6N~k>v7`-@@M}>5%DYXc!FOCGttLA`reLA8hq{{Z zGj}!xE(PJ&U?0T8Im04S-%w?Epf;sGfH*}z0KvQpmW9U)ue7L{ z-jV!^Z^YlH4?iN0^n`_N(~UBe%-|%;9`PkT-Q0cz*OnR9^!bK$w)!N0djR#`5n4@3 zzXvRt%|GC;H(BBV2cLZ8wk&<`4qMrr)9KRj!li^q1-N9Hx!y^Lh)P%F)L7=ZpTuGmYD8=z~zb0EJG@W2Tyv;;= z$_j1J`iBKsQ;=EEEml?aquj)DfyxdX0qAQhpget;1~l^fDp8|{M$rKr+>YNu(>EZM z88Oap(`Tt~e)lY%hbq_|6Qk?NBd9}4=xedI#%0-R+nQaYa%3DxLc?Rda>b% z`O?&M>Z2E|IJc|``LGmD8VpO-hzDG&>BxegjA%psXy!0I_`rTjg!Iwdi2}%IYIi!x zF$|s|#nhwW=>QHaP5sM2$g$}v5wIg*pQ9TwNtcj(8+MbY0VfEddgBMH9i4ozC4lEE z-d+3M-nB4=_iQ!)$XNShAF#ngR0aorb6$;qZuD&a2Peq-A8?Km9O+Xcjzu1HEcyj{ z<5AG!r0`STZW?B^Rblr{+%9@Y3%pc}Z1uSNUBd!Bbs;HVj119>o z_XkdQ;t=dzOodMpgUXJRRl9O6EckTW`m_a(`L5MYN)n#8^eC*PCpTI{E*clkPpN-% z;2JXY!(CFxS0f4-RXT^vSV5Mva#Qb;!V&vs3*5Ba=zOF`9z6aQwG)OwxJ#;L=gt)M@?}oEua8El+`r=&^(8+3!2} z?g7t;K1RWo!Qze*4@G3161tbCjH_}~II79Mm3Xasn;`q?6LwZMCJ+X3|F=47b+RuNi*^L7r6*j5?i zKeLsm$6^Xke8A!=7a|NQd?F_pt*_}b@!S8&jLo!Cy*rUpu<_*j$oa0PUqAM8#`}E6 z0aGto35>el{X9X~?m$uYxkW#(@q92&lf`geB!NEJ_D1rJbxari*+EbP7X=h8?$xXw zt-?yLQONFQL3|>uU=V^JmMUJgj@CBi4~K)vGu>ktH{M-9P;kd5g`sGm#M|@O7xaei z8u`AtC$x4^!L7yR>R0)8&&WYE;32>IH9Gq&L4zy8Ks;s2$L85xmoc#hK6e&!|Jl(R>V?=V(5SngNGs?r0$N>8s z=9#hE)XW=b(w*&l~VKb~Q?tH4ss7%yQ*KeR1hSl8c6@uB+ez;KgYr^ie^vbC8nhdP=bZWTymdgrV`7DDtZ z-bFyBjstgbFZ!(b27bv{i_=F*s=a+?lX=>q*w#0$4$uo}+lpuu!)+qP9v=Mu@s@QL z8I;BGS3w^#-Eso!bCU*=;{CBBEu~|-E`4a6IfET|etvV0@eq@V{|H1KBN{W^4uCt7 z_ThTyeLJ+KC@a|%li4ZK6)jFaN{{fqc8cDJE|%CM`-XYAQXCRo*5eIr_%2KTK&k89 zgz?`IP($t`dxHe%BK|B-#MGx_9(yo7bGmO9y!vo;!E7nTZ+X|)NS7f z{P-6Op5}{ZM2h9Cr61hh;n2`4T6rL)=&sBbzypGSK^~ce(~n1-K)v@ujMt6%nbj+me@+HLZE&m^^JQLhvYnkC)iCDL$|7(*5Q;$(ZN4gDc}qKXyP zK!9@W2X;$CO}<|H$4Sw&f)=Gw8}JI6-Pf_2pm^JRfV}&01OK};xAfTnK zf-sJERS*D2=@ZYUSpznv{MKk0pA9hVmS^+T7zvk{Z$I~tLwS0#k>XjQXa_9O^e$9n+s<0Xkf?-^8~#oCKUzDdvR zXzb(n$Rm@vxe5Ca@I!IQjy>D%p!#uuCAOuvEhB3$`~`LUh$yYj9@`4T$u0Z?&c%dx zeavbC%{y3Dge|wPq5n*f#9VK zDC~XvVB^@Fb}7(@aulY8ZZX9YI{AGTcLx1xQS~!I19xZ%pd$@#0KVLL$tNmCPFD~1 z;}-Y;4iriaE>;eZ%=WQ?TobyziP8QX{Ff}ek2-y7HvF~KJvBn^)d3ekZwx9hbATBK zl^07RVJwa?c<#{16KJ1zOpkzE(q$_zm9enNj49}Jjb&+vjiTb~->in$(yWVk28%_+*=jGM%Or?3Ltn6=`Fw;YEctx{_2l3GU>^9))Y~G&{`|+#I&i+(e zic3$E6Ya)Q$U#%?6rp2r0=W`LGO=;#B4IdV;wj0jzB|2WJkF$+Y8{4*l2-X2J<;ft zrAsW0&m~~fEaBB{)BAlpHxph<5^5xpe4~7hivrgIi7Iq^rk5k4`SN+8PD zO;;My{B(8Z;_zN}s~VJm1d(V}YNe`)N-fQB^Fx6!_$Ak@BfHW*ns~MtWz~&*n8t%z9I25c6wgB#6xiEmN~{88DYxnLIbEGKnS{Gh{Y$dVTqFt>n{YOB zutf)(=jLfNe?+sTlZ9Ks4mq3@Jl*9{>uUyjjSbJp!-)k=s^7JKwe+T&KK4`bXo7%{ zrCH>2QZTN)Kn>dF!ks>Hn50_EDl%E_0=#eM87!8BZl@2)MF7`QHXirKH6ruqNZOqR z-1wiTYuKu&i0*!ACX#5Zmmk=lsR4tr>uTHZj0%CYCyiq17%}#T>ZGH=q?gs5RmBwB z&l7RF=s6xpShKO8@d)ZK`2l=4ro-W9-U;;00f>|breBd+z;2e0Jfmo zDP{OCTUm3`*4N(9Dm;?|es};RfyIc;I5-EGr0lU+kX#!D?}mAX%BB z*!O)g+f0QwguPzJS_+E%g!N59DG50Y`K71YIn4!eKJasCfRo70H(yO#4={r4=+~oH zhyQ#-uZMMg)4p2PTZ^l^h&&1=E@=6z%RnbB=fdl;zDBo4SnnR zF#8tXyUz;H46}jDKSe$urI_?hl}iC1{n2=wrc~e!r6K{}4_d*TD2OAx@F;sB%~3{w zGn?$v9usU|8?#!~>9@n9an&Zq2c8E8<3J+k_?rfs`?m(cyQwurjxHEJ}zQx{>sS_1G$!tIr2wbXun z8***hY3sA6Op0atMIbAa=Z*;!Bcs{9j_K&M;?7WIazio_qc^YT7oXYODJ^+}K~Eu~H40Lxk@-;8M>7o@aV=RrZOUVVc ztt(x8T3VqnCnOB&R~*p@m(RqnI0vhF(+7Ka_*$wt-d`?IZZ~wiJ)XV7 zF|*}28WRXX=D^2&XHY6=Q0>d;qBTAG`{fe`(@MFSaKdEJo5xL;Y~sOkwB;aSnhGD% z&7-ML&&~_Lwv&^O&uHIon3?FdX$)-&9Xh=u)HCcfIC|7sr`_|Ku?4!N35TP2dqKhS z$sG-#RkV`|&8(4|lV`@t%z8x8UKWN?Xwv6;Ns^Y8INSSg&*W{*_WpO$D3iQEmMDL> zBH4}QgRkZ{r{EiD`KE~)qNt854S!Kh+v1R+Pfq&QhnAb9~f zsGyMF^A$+TZ@%)4HJIB!0)d4b^((iOK$56w;h&xm6Q93|#%Fmm0*#e0g0Y(%TWh;Y zToNA6FVE7+=$o(79y8Ay&Sg902ZP3kG$dTC*%Uby?%^w()s91OY@tcF?mRh(l9!^%9sT8F{T_gh5IPH#x<_mqs0 zaB2y$bJ%9Bi7$cA<~z0p+;oT^sQD%vGwX$>uISG30N}if*~10>HU{5lIhxkVj}_AY z?cwhFj-=9MsIP(S1{e?=EiRePBN#y0U1jp67szPxmG@-IE#KKzOvMwwYgo)jQ>x$N z{WHq3Zu`3{%WpwC8B^Xiq^&Pt7p3>Ju3xV`+iA7D5sNmc&w_vz8; z#t0D_F3XpdPd?~(^GO_r0)i(irZHgC=Yi$-rbOPR2TiMC)O(cQaWuG#X5ekcqSb3y zLZhj(4{=A;K=Cz1SzLZtuuQw}6RTbr(I9ej=d4lB&(U!YUK}zj_!vN8?=sZwvkP6Kmw#uPO_~@?`1E$Vb&%}PhW8Q+s?`7`x0tC zp@a{2rA(#nV?@P;ZMO`bqz))jenZM4N55HU2a#Z(U={ULo{7qli5hJbAR1B8wk zGw~6wublYp@@`H}^ga$GJWLlxT~dBAZZz@RSBQxUV?J7{jc2MP%rSK0!*>rSP*i0& zcY4(Ua;7a?vpz}T#(x_LNaF^CI{{j>g_Q_wCYgJ`cwNyOqY~G~>1sgN>3XZhPbWOw zty%$RK_0XhPh0HuMUwAy_le`?$9H=0KS`=HN$TbXcOJeU-0i}6UxHr~;X&I7l&WasozV9MK zCAarOCTWZXkYQ|^vp5iKB&XhXjjq#!kW>YNlSzQE{j3a^+<2?C!X98L*nIl^RvwL0 z*e)`rO%fd9``Ca-&jyRFplRN6&bs%$fX=-51K^MGw?Bf$yo)3ZWbo*EF60%A{iWIRv9Bv7=~blFEzo%K~ug1G|SSg6*ldP~{|fH&@| zVkb0@#|G%I??oyLm?{TLai1-eK{xdj0wr8bXuoSQAo5XQBV$m5U{Tj!N*mY#itBj< z&R+}6P}4ku zw>{6p%3`k+50&@~_xOKld$aC1s_$WzjN1tuU+uvC z7P*5b#36B#`DwSIwo2n8c9))!HUfx?WsXm+4~r~)8IZ?1NRb0gr5?a@$M$#UTEOcPH3W$Dpsz$xgNjFOVaa3=+oYk}UVf;_C7gk=;$JvS(7H5+CmT zPE76bh5|nUiU;(csa_cyp6OFMT$Oh_=2DAy=-(R^Kyqn_c@ftJ?_l4%b-gkr1+=6R zv|FQi)du^3Zd)mOJE9~#7+Y(9QX9!82y)a78*I_(*>SJ;;$6?b$Av+3rV!|1FjSKp z-0h;oK70un04m_3W^nRs!cY-pU46GKpR4V!esxteg%Yut5h1vGWl*Fx@J)qQb01|4 zKTdi`p{ETqH(`J3*rW6oFD%>3G)tka8mcb4agPg5>5lxT%!g-rz&BJO4ZpA_pitG` zxZRzyvO@N`6a+ss_>`0~NX7b)fW~nYZO?bniGx@@HWr9wDbJ(O(F7Q;S+w@1mJNn< zx+L*}Zsux|jIl6B9O7ig+>8mmQn+1NM4Tx4b44()p|*@f*P$8H&B0;fA_Y)w5BNxT2*s>4Huo8MI5hr@yc=#ALQ=OQ0Pvw07mZ;*So6*F4ybtD>%$ zE(Dw-&HP#NFxfS&EZiZrjfYUWu0W7#^P0Uktx?+=#DIS|ElMGfwVvXFOmRbi*n<6Q>|!a2&o#Pi)?l5SlQe{8!&i1oDq9 zCd;Jt^^j$%kSZJ|#Sv>k2Cnc4Mj`Rb-jEW}%Q|C^S5Mrc26Fu z?NEs+4nULG3mZ*du8#+6WTkuNKpzjN(kAxhTtSz%GQ&$E1+-&=I>QHiaK5y%ZD+l~JZ$|MnsAL}Q4Zo#8}V(g-G-9{ z0rJLic^*{&_W#B@41;(!3S!~4BZ1UAihsp3bqy^d71`ni^bLUXD!*a-l?CAOJ!1=* zo?4t9a*NJL)MOoBETKkkydz>3Z(0MZZMA;$25_HrxYS!PY%!nR0d!JA*60S1JGi>W zGSpg4CYklaN1mHVXgYzV$-el<2q5h&yerWNTP(R|n^p{NH`MPlX-29tLzbFN#>-ey zO9sbD&*cX1mz3DFyj9D2l z-CCP-j~qlNr_ZUnaMzwX8&*_Vi%})^g3R*%%hrL4jG;My|8{BEIbl^wT^U{L2z_;P zw{JpBVrJq#kQ8?n(f9$ZVmA`muue(i6`XnY47L1$t~wQgf*1*@V`S5Qf&2#Jb?7oj z>lRrOI#udgL}XG>p2N{sb^guXf3R3?7T=JFaq}19do<$w1|m0 zf9-w0`0xPG#=|bT%poF<^XF-yNO-;3miBu!b$M3cSglQ>C2sfU536Xt^X*nhdzz&S z@h_r+g3RVVR?7mgEBdw^54V66SSpAJ?6zw4EaRc_qi@SU4H2oSz@*u5e>u!UifFnkUl>+zUL=>Vx$4=+)6V zB<*NU;e3y~z?bm~;(hoc&_?mZBh5aLai>!57h5|*4gaAnb_#O&@za)$1(WHY==Wv0 zs#tRHD`I8NfO5^Dy`gls!8V=bo7TL{g2B??$mnH%_#Ye|MzqroOqbc_tQy`!a?kf? zv=?LRA{bWc6hGL;eZWx>wG_Xk;fqEe)aBV@RoE>0Ir_P?-}mJ<@+cP4IJe>GqsHT^ zP6j$$U|4+0zD>NpZH-H;HrGK%l^h75Q~6B^u&w%s(ZZ6$+u_o~>a^Dx zId2g*WW}5?EP@t57e?W34UC*>E2*B#@0U!^PJP|;M>slZ^ zytW+QrmVdJ2oi#9e0d^3WE1r|OGU-sn@X#k1p5)EM2HXzUU=gzzy)lPPFSTDbnlS# zYs7V@+}KwfB~3vodtjOQ3qaXwUkX+@A11<-RGd!}5JwDK%tW!u(~Lq57V-ddcMRGv zW)(D@t~i-5^_s!gt+!={#j7eME5pI4t7O)RV z`s;GtH?R{`-b(QXb>lamfGFd9V+msPA)NYZZRw02xgyrhT5_X1X(=d{{fcf$S3gl| zJj|t!BMmyefJ(xkDlRG1Gg)VIK6)FqE8qqO#QxFUrPvmrjfz{eF#^^eP1!U#_qJ0tJGE$2@hq_5V$^8e51mG@r&gK4VUB+0}y^Zolh%GzRrI1zX6NX~eg ziTt`u@0Zo8gT2(#j=3crrdU`7AnFJu_5|ZD=QdH&OS#T!?*~d6h%b5n5OL(E(aL{p zsVL?)n#Vit-THdLe_Lwhnoy8Q;qd3u)mTMOo2uy zNRgmO_8$KVbIealPH_7tcHh#y79$7X&-j&S-2gx(l9H;kn$l)J4Ze;+Z7lOW@8?;6 z$VEpy3PM41yd|BLfmS9ra==91OKM{5_`Oss7cH?|y}Rs=u0iIyug)1{cK0A%+Hs`s z7i8Zn>xzmaGjS!JbT)4y)0+%Jd}yWZ_QI{RKWgVyN@K^2h)og_%^B5G5QVwnJGLO& zIz?bHR6|M32kfn6@)G*;w$UK5Y(^TAK2a{c)wMV!K@o>!XSK6+=1XOB^h=BSElmpI zz(uLM9p8q{MlKtfO5fs!vh&YJke&2tNI4|Ni{Rx-?=x-%xICBn&_Z}3{&RY3-EX?C zq~b;LB zgyXfNR{>K%1ZolyX6IC_!ZyTx^H_}u@J>R#82Ps*k~NuQ^ghJ z8K4d2r0)0vOHEB!)&CYwr#^ymJ!L`xaalb>tVfkloV7XK<%YDh633Y>6t-I61odIj zmX@rsQ789*=sD0@-V}xn&CJ>Ra|`|)EAAGQ=(KHR5@BmD>>IC&!4T? znIy(ij8U^HzDgP}y$873Vf^-_^HyE04U!|81e`LhO?P|KRvIktnAr)Q?r7DlH<*$= z%MPhHN64l2{9BigBE>R%S$o?~r}NV+KPy3v z168f~a~}|c*C7WZN$X3IuJITm7nN%jK7y7!xGER^#;FgC>FRvrj+%DKXj$=^k1ELt z$ew7GFTMY^gaCZln6E%cD=d+D$UR>j%MWO0ic$H{{$af=)4FvWYv&R%3 zd+-n~mrqGuf!>s_M5J%im4yIH*ygJ}k`TJ7#oq;z153Q8$rkN~c!4`gk~66oVR2Z0 zJ>N~MAf$F}TZY|Mvu&^6Fg5{rtdD5SWY?7{_dqpI(6hPd1gq6-5QLTABBn1+_2WeZa``S%kSrX+KA1#x&seQ6oDD|HmE6 z+P7`ft3fh5PI3$MQiAU7sslDG92_KRk4dG0$>KVo+zIPZ1YOxV>3hU4Pe1OtO%Zbz za$`{(^1=rFP#|Kc%+AE!=G*K=0ycoQbut}gL%QB$xYY2#l)J0T;{eVpfQ;J*e4Bf? z!JW*O3*BV~l?}DqA~)@(&IEU%Qr|ozg((Dfd*?O+5XO|zEAa(sm|P;46ZAk9rs;eo z!T5`=;(Y5G))Ch~%SD=-rD>uaeD=c`w@&Ta>vhBGElx|2-(A=z{j^$av(H`_mMr^! z?LT|w{IJ!?i8-E65mL~c#QGpqBcQkBh>h{i(eM1VpsZptGS$j^UiLUE07~Y8BaYwr zDkvoOs?k6|CnDEcrw{K=xk)rX%xA@BeFHv+Q{6)ANF16@V-F*De5=Vljxb#OuyWBB z3J){I!zzOwU#PNaN>%Wl%d6e=`&N5_g`yBO77_vF=Tq*MMl>=25;BeH^92THtNC!g zQ12b{e%tkNmz~X^h!YLQ;3r^7NDcxXCS@!rNluBRdm$7VzMan^wx|6Uzrc=+V)O4= zWgEzQx=m#_e%^Xm-?yuu^Cgl~ zys!v~DMU38j@(`oxh-qQ-|MG#%>{nHk1}xNe*D6i*G)D;zt!f85xP?&!O$IE{x*;Z znl`eS14yZ3d361wm=t&HijylDTzy;70s|mn_XYrOHcWXrf8HImxobI#9EK$X)tHt) zar_W-8mER_I@jrJXb@N$8rdv?^F-CLku=RE3`~tMPM!CB>~TN!t-U7V>%ILq>2+)6 zLRRF)+f!-L;Yl!;_bJ6ZuE>vl@faLKqI%qZN3S=ipKZ_H&E5TI(3AGRu@eLJtP5bSs;BKe$VS!R z*#_wu2qRZ<#BygrkrMM&eVKOkKh)7_gM5lprhxcm zUOGt@60Vn%IL54natwZMN$tr%u@q$OE7l}Ktoj&QGnz{7DHco+wQ9_gR4=$UIar-$ zpwf#bKxqo~$)@OgpXex`)p6&<)evzgTgKRb59g37hJmC52LVK-B$MHrOr=X2ca>uV~ z8qF`<6qw#(5=As>%{# zkvncmrj0eMQ;;au1`iLY32*60bx%pZD<-vM{U zG<~EW@T_ieo9N%_&48@N!(!A{&nx!BB~0BiQ#ed;qQpB8A6E`L?dlJqrk~FiZb%gg z7AbrQazQM>|D&0hjZr-M?g7E_^hA(P(@=Zr+1rx)CtEV{2q3pMW^G;+}wWBH~cnO zT)8C#P}Xcl#V89qB`(A^Vm;vtpM5Yg=J+PB^$(g$Vtp@@a>K(hG~~A#Kt4*9#QdQj zJP~O%cOz$FCMCwkx_G>zUM`d=05^Zo5E#~Qu8KgHhqo?``o;3IDd#=-0az%)eg5Q+ zmw(cdA>Iyq`UsnChxkmsuiqOs0e7u4yI#zb`PoTCR{;;Vy9g^pS!cI#&iD zB)M)&Wb^C2ADevFqE5DMk2JSd`{-D24z#1>DuNr|%+n7f`9 z``h5QbMq1S(R+Xc^KVzd;9b41idKx}n6ba!E7oBHce_+A6$h`(-u4F8%a)=H% zh8h)qP$qTf1RL4VLsYsh{9%jC4o~et{RIOWdHv9>DHPrGAv)Ey+uI5oVsysV*$syb zr<%cy_>kx+iV%9{<5-o|Wgw+dYp5ev?cAJUsGJy;(Kt$)DK<(r5=)hL&v)gmuSe_} z8ZK^Wc1IM)w8iT@&BHJZ&ua%*WsTw5_Bdx*H~zso9C@c#kr`>b2XUZB!Dl@O>({kYhV zA3Cam+|CzVFC`EHAk8VV*mQXg!!wC%*I4tP5BVG-%bv-|^cU;V86fF_* zur|C^`HZ%^nUBY&Xb=z+fQJ_r88*euE{?vPu@S^KedOj3zksXdL-+gv9bE2>d;-TO z3r{~Me zRULWLta0`Yglb*M<`WTKd0qDuxml1EX~B5iwuStPGL=A5RSl}hLSH_-o1mEnhh5Q- zv|uAo!Fo8uE*BSRUd@gvPOx6P{X8JCHtD|jipa84ZfND2S}maq61l#jNMGlQOOltW z$1>pe2JY@39*=22dXv8CWq(;DV76`~IWy3^k=k)6R(z}PF=ixK#V~1;^iB+0Ko(~o z^V;LZ@ly3(#y!9mkr!Bwc>xorE<4(TP5j%L`RJ8=RIS}-kn3b(!~`B;{NeGxQI+-! zUc$9ia*7~&4|o-0V5E-%H40XwKYP(}ku0l&5$Sw-yN~SkBY2Ldn%Cimu9`3HYl~|J zY<0&J+eD;RV3G~GhtlvGJL`fHBuowkm$BT~6m?xhv7y2=^fPBX?xvD!@4fCmIfJY+ z671fT?ct4phMlm6>uUbfNfp6LrM!^;(oh!~bJ>e4`ZX20I@D6nXYmrQH*~CGfeEya zsAs!3jmz6Hq#t4KzKhb^0T^*aP*hv@VrAGeo^b6=ynu7M@FGp_A^E;I(7iTB?b90h z2u1BBY&V-Wa(?U!N7NSDF|>vQd#zYnzMwJ-Zm9yHdTp5A+>EQ?_91{=0`O!-i*xLX zt?}$TzGmA~u_d6%=!#3r0d&Y1txgi4cfPw=K_*IiCX|@hk$}fw2Un$+j~TH>P;`s@ zW$8xe9Phu8A-4W$?LU+MA@o7qF5-rc4tHP&47l2lZ=3|x?t~oLwFh>`VuXKx({$$DV+y)7#GN6kA z$0PfsUBG;<2i71OZ;wTf7T{PFD|BBOQl6#dPxqaSILcAK`$W=83|2gBp=C{P%n;^^ zhfF+YEhMoF;(xwV(RLScLMeW`L`B$Ya}dsZE}=pm=gfw?H)4Y(uK)v!G9*PDm20;j zyT!dtoJ58yt0$Z%I7qNa+oH~ZeVRnpq+GVF?la+!|9riNd~O$!VBJ+VGGlCreN;jE ztApQuI{59yQBHVWa>L&SX^^48!Qx4vp%<#HulI&*B$7^Pq1vLs3Y=MAT!Nw2y*y-C?gLgN!!9yvA^%Q&o+9 z7?$ZYhymm|`*$+nCgNHh?=xKqvTQEG<{YZq_k5#%46+he8Eci9iHw^k$mUUfz$rom zu+{pt51cko9S?wya3n0SVBcVMOKV+bjxS+SO?%edX!TeXZA=R23EMZ^e;Cb_Ys{5z z&9Z1WSkvYUCv5!=^?26?>Uh--UJNa~GMO*`V35y{#B*4;?$4=enpS7y=_5mOXD@8Z znpe7i`)^MZjZVa?x`fm|9 z$l!^deDfEgDJ(&E1w)!!!;nV1y>`PyEm6heccKK&(}Po=ypzhmYgskNI|xP&X6&iX zN+(dw&MmzebaP5(I!H|#S*B>ET>r10J##6(8k%ioE2E6rMmC>0&=d=@C~WT%-GS2u zbx<4*8GA=hxpEbE8gs)IE^~x-lDC~Ygy>kMau&|%l8+y0eAVran+rbfpVQL{T$I`R zhDQ+Gy$c!}br473#C#*LP3snRd6Qf`k$x>kUa*@~`OH>I%CKDPna`@eWAm6?K626p zh0O>t%CJ0r>_bc^E=b*Ji?_#rCN)?Aoh8+<7ic&GX_7kR`2V{Nz2S7HF@He0^){l} zF>v-*oD+p}-%+08HD&s^3#uCEQkC$->AD##DKk+JT)#L!Y%YewMFkF46yV>s#-{17 z*)n1q1&8Xce`bz90uvX~Vcx>h&aXo;K0{KkAr^fGT@C=@q9^+-K24NSR>^u=EkN)0 z9VXL8IcG3=w}C#ljJ5{PzTLKyhyuQg*P!D`3PG&f*HuGo1N9LQG8?Ym2$fa3Eq>Xt zbXL{SrTs_t3h&{V<+_4JR$Gxr>ZU5=Pe*^0*k43O%!hm0iaEr(lyaYDwSkZH!lr4C zmhaD8;y}mB59~7d4mN6~_`N6KNyzIC^aTjRR8+VawIl{O zEhk&c^}Rww44ME6KZq6@^QSgAxrW+!)k3M0^cmKR4fiGbV5aeC9;+)vwA1;Obho}2l!%4`XSg_D&GgP1m&iOaT$j8=<-p@u>9l=ST7d|*a9sD~p|v!u z53`s;Q3eL0KEtUq=m!jP-p9@PzK{Xre?ky}OuVtYgO<=K#?olAYG}MFRPYFHUd1DzqJJgMms)GJV)z!y_$>w%W_5vpz`mE?s z>TbHBj#yZ7pDPvBeXN*n%jF~2)&1!P+DmT}B{4$;7Wzj10SCbNOo%>I6c6i3=V8v)uH_S)V$rsPNf3SqU!oUHM2TJpBs3*Seob{4=~XOz0Etj!NbIW-cTuwVUb9w9ZKF(TR8!!r)Pierhhjm#Xw0#SW|)( zh2X~eYRu%*(Ft$9t)4uPxX8gd7o#9+D2uy!Nc#g$)l2=F8?yvFM*hAbuvPYow=c_n zswQ%q4Z60$eD-4}@LztSDGr{ia2(mJgs?ohitsbythw8Kt_MUBKUF&{J6V&)g1+zy zTZXLMdkBaej1|OLytQs6F>le~*r^4c*DI0^9`CxegI~@*K1?2|5o#sM^Yx;-yfMd*`MUH~`y0SvWsx;a`R=7@KX9ZfmpavRf! zOCE<>LPliiosSw!*<&f)R`+Es;f`_bV{?)f0h{)i^h+}!n9cIzWcpY3=e417uR%m* zc2oV{#sz_TxF=>3Rk&_WYb^^sBgNqKLdL6jQ;uc$X#Q+faiB~-$T#XG&M3TadH>zs z8;t@Lgq-hsRqNg8FKpCWWhQ>Zbr(u@E1y*QalIgBK-o?-Y{oGb*~zKS@N-Y=C6Gz%GL2!}W-&A6jRv zW@o@Ll_|sF4EpktR1fk&#zcg6xBcRNe_hN8K`qvf>N&anD>%8^rH_UO0>3>&soyL$ z_=#kE0Z|8+Gu8o>R+>ipVf=%syS=`6p{5Yg^T}AUM#f|k&ePmFd5TVt>^%{t`GLHy zJSWVgpEMD`d*!gQ(Jx@9@D|u}?n#I3~WtI1aWpCK_yS?Ae?8BWz ze{h9ROs zT&`zx67xl{dlOKa5T~u!34Xia%Xh zH_4lV`B+=aqho4XSajgSGsXV|*SfozkLN_uR^FEL0VSb7W>{$2^d}f1uaR-rY!-x) zDkxTL)OST8K)xo8>1`z;OXA{OU`No#(FW}G8xu!uq6`G$u=0%tCH5J+4fW6I!j&Ix zA9^FPmKX^$^nv&KCdgq|3v7eM6F4)OeOOj(`+Tbp>tGF*CCnW74^v&)2t^y^>ecd_ zhiEfsv?Y%r2?fu%=2mj#z_fz>an;O!&Ca<)WLJ65gnq}4-K;+2muczeJUt+zN;%c; zc#hI>AQpJ5&1f}n^j%6!h&9z=BVW2_f2lVzjMoJZiy>2` z8Ng}9UF=Hb#5vq&#D^Z{wuo$c!Yxny3_!G(7F@w<}sz(|;(Xai5F zUnn-(eXY9Cj-_a!i0QGziWaxAPLRPoh zK@Yw%D5xcTf3~)~Zx#9a3{kW*d*UgDl-aC%ENBvIJgXGGA(NMAcZ8N+gpz(D3lv2; ze%i{C-*}QbLcYb6Nu;m}U2sLm4Ownnnp5fa>38ju*w={7i$_aozp9MpG9&V!?nM=p zU0z_-?B*zaW)BZAQ3b@&Ll`bk+RD}sKmtaQBoc)%U?~U!O*>MU4dZqzm$roUK3H)C zjnu^wIfT?D@;7kOI zW4SSqTMpEUGg`u8_rUx(rejN>5-p?+vl)`Y&1)?O`kB&sL)1PWVsWd&i0DH3s#iu!)(lD?D3!9l{_f>%Idbe zS=`sE>BJuRlLQ!@u3j*`u>r2mS;;g!6|)c76gKH=5`-zVql{Vx?{@UxgFA;UDopMyUK0aW~appYbG(S>NbG0X&|)46bDtBJUBW>vM2 zMx|a|!@-b{6YIlro%vvIG0E{V5Up{WxOE2-0;K}xU}z8{95c)ewxR-2yNQ6$aO;sG z6IZ^SbF=wa?2}HLopp0G*^%Lvy}I>ZNQ3^;-YAuWh^3DZKQIEHa7u9%cH5i{@=>%v zaLhFf^M|W;y#3-cJ6?AH)~2YmIOHmFf6|oat+uk+G%bu~lflqBP^9f2=C{Ly!64Vl zgG!$%3H{xI#vY12cv2r{$xg`=qP~K#dSdpQAfK)zJYVBF(MOzn z&yU%{yW)B)iSJpakLC+BPN3_CEv;llkca_Hj3Dd{vBMxG~X4=a<|ZihV((h$nP9lBSyR`#brm6(%z%TjYMT(?3r z$)salCzmShVR#J%f-Ei>1O=*fv*mk(F@-2fZxv3Sq41jZor(Th$uXzc|5P-!;b43} zZn#ZgS#P?!Q^?xQG7&&xFUL8%kJkB1YI=fhB`Y^o)?|*|nENxrenQ6rN+!eRBzM$9 zYpRsE>A4AbxA8Uz6I>E=l|Dwi*AaOGW2!_0LD7VBzImQ=I-7W&Y%$**t?Mu$O5o~z znp9*&dNHRhRaFiSsd5~>Fe~hLm8$?(i!X>6Aw|SwjDVwPs<~}!A%s8V)(sUtr7xw1 z6gjR*U{K{yq6n`PmLI{0$nKW*nyMESnB{X+D>0ZT14V8z6=JimiASoiO31aQ(mJ?a zVV^J%R(PmlzKJg@Drg^phTIZ=%f-+V6^fG*r}-0E7WX^iZ7?g58rxv~cE)uWPVDyC zx~5;ARU};{FJ9>{C7UQ!kTx`vW4eKAEi?r|7atoHGx!iol%NM6Rw*EHkTtC64m1ba zyi6@m6seSgKXRR>mQ~K#tvGh%K=_Z;))ci^5iIp1*e>8Sq0sTG9R?twXhal7u(~Jma*ei184cbR)!m9*czTE=*$;nAkwQ1hYs8TT$TIl|aZF#^X0m-oswZiwc+oR{lB@JftQc3H{>t4WjTc~j;1YVRKw;ETYSb`+X5)2C z2NvjG{KCoI;+!(6{iCGEC$rP>+tQQOx<+WCo9f&QG3o(Tg8`Elg@5k3CW3#|72U{X z2RFFiodU7vDI@G;Z;95D>2i78J(U!-^iu(Y!1C0N#9;&QwZ0v z)Skfe->d&1g5=$tksCax+h(hl&<;4`seA9YYq|XW` zEl!FlG(NCx=nY2jk$42u2%jPqDwjzoR=URAHmB^Ke2v?ycF}1!ySL(CO&Nl~p+)hc z+wU-?Z#>5iTCpV<)~w^!pvv$EUV5_2a3_6@5zY_JfQxy%F{KSsG1Vo_%YJ_YVQ`(` zo*aZ_Y(QX*uu0Djd4kwCP6TIJup*z(-T`B;Z-)BPMsJhEl|kqehnrOU15#fK4U?Y4 zgRZh)+1PgC^y~`41!kr5Eb28yw@L=ofIdJkyhHqCc=;8?S%E_2M$u9OpCzkErA(s0 z-R6Kn=bjMP)RN< z!OrvSl9tedD(JWFxyjdaifqV|-8eDl48=>|<9ibf)wtj86`wlhMRMg)I%}=vsh8~k zI(UBcKOlep^S`TDnjiNL|1*|OGhiCew!h{Vt^nTKJIFDq*gU+y*==@i@Xfo={vRGn z*S&8WNV&bt{iq442A0)yv1U!nirp8;v1p@*SBLeM6|0I?e;TM4gYKJC*nNT7?9No2cOy} zvQnSE$OgVbGs4@_E}25v11|>(%1&FsTIHXw z_UD2RMbPXx9t;$(;97hDV?@kr=lu)sfg*|dW3WrZKI^oN@*>(NS@q9P^>w(bm2=HU z1?bgMNuL8U6yv8nvO+(E!K(j(jNECHm#nQ_B<7I}eQGLKJctB>vD4;96)Te)1$1#C zZeg7KQ>oghziGK)pO!-c6_$nz*DV#NF+m6jm3p_|=ZLbhIQD#l1jSgB4u->k3~F7L z@h0Zm@dm?}eGhxRKl`AmfYCv%7ECX0)!H~n7Ff9N7HfLsN-la=EECG&1Pc){{nMO2 z)tcf+%)ci&9eIb>ds`B1;dq~mzJ$QLeZ^*3ayPJdbQHZ3BW>ykhueO*8;i3@gJ|Sv z+^^HiKOV1CUA(ExF_XXNoAbi{69_x`4}U5~{?8csEru=~@Rtit@r%d_rWhSg5L>&C zSEty9e*-JYLMv^cUYiod5mzUV1hF^^F~hb)NoxjYo3HEu_?6H^>^d_o*gZspK}5X3 ze(RtSdLuO~UOf3*m#yUs9{?5PxY!Geye4vf3il53xH=%L3v4Pp?jJ1?L#i<@>MmU_ z-L)aFU#^4#J3`Rl9hwN=BlCnjSWggC&4F+n*s>s4PY`c;IcC4EKGYyuEoHZE;Sub_ zj#l!Pzb^@Iw$e@F{0>8w2o5%N3wZ25JzDn=JbbA>c!VU!JMR9TCjAl7ZGwpkM$m50 zyo#VLnSBUjxS$ew(aXD}{;F&mJrxoDY6qc9v_HnlWgpQ(5fpPpB*d`9XR8i5^*E;! zlM%7Pws%AR#94w?c?~{POS&kvJ!@X5C(tUB)O72%7TpxH3Fa!;+bJs$^($ar^vcpJEGq+!f;{Qt^CNCM!DH>l!dh{g9wwix41`!NZl75j7s?oaN@I#(pV z2)YXM;BR>(BdD>>w-~=P&y{?`%>^8=0RFE1YnH?WiyL1z{M6as~ZgNO+2T3)hr)k`UJ$-Qq$%MNYoz*X_=q`PUDEIbr%F>@<9Pt7TYn+*!}Ag$vE7!pjnI#V5Djv;;zBjK zk=%D1VgqS=M{wa+ih}#w1%2pRs|lyC^?*ufb`$*HU;9Gq2;eSuQN^>Mxa*e7mGD^8 z{JLt2D8NspJ4`B1j8y3Q?8Ye>;ee4+c6N=XMD zl8D10Uiw0o1bDhg1Llpl+$M)R5b9#S&;d#X$KP!V+xP*)qlCPxE2s`ESUjv`reZ&C zo1znz#_*ocrXdc4h?D|%o7IMmIhqD6x5OFv)=GREe$!G?969`aLc@&f>Yi|f%mlOW zlBKepZ3=#?O>t1K8tGs++-}2iUvqgL78p~Tr!Xc-;Uyv!DfMUVy#5){-W}2Hzhejf zQ$uiuMKVdt$`Sg@L$9$zW>2kHA}z1eXt*>rcu=(kd|)G-7$Z{UL}?L!K|4(VN>&Z0 z9@UP-(N99*-$1=!p(GpM_Z__Liyrb1QJH5}COoi_&;sbB$jPdc_cCETu%Ixqv}gH( zokaNLn)?eb9{^%IAHncQ!pja;5}KgYP{}?LI-gkp(xFA#C5<;()II*Tqo&nAkv~+- zkVwN{KN=WX2?Gilw7#;#lNNg)nT14bP&&IO@FgU`SePljV|?Ux-ais)L~C>8P_J84 zDSg>Rj;*_{_Xn&L(h3Lx5Z?kS_{NPA*urxoykCFu+P0fHJAW_iIs6?Emixvq)zbXW z)+@}e&zK^K!+qu0S;<5plXu)3(#!N(sJ(=g25r$-=qze^&%E4Xf5#ZaHj(>wF#9Ka zT4`2UO&pf-g2?3gMgZ2tcO~Kmtp@Z{NQ__FCeOCBTp*-OifO#t5&YY_LWD9V!nl%e zt8TF@hHABjx*^vFm2VG)FS3Z_e@lg&6aeAItEv8uTeN~^HOxjBsSat~)K{FglKp?z zU)z3Yzub;JM-zETKEVKIJeiFv4IVoys@vn z&%KL@`$c&*)SGNTDx}diR8#UlD(x`ja4ks;SN3#cn>GWnGe=vCsLJ@7Aq}N$dx{aO z#KsrSpT8zFP4j?nYH**8ZR5k4j^;D8e{q2bSDf^jtAEf)z8bKVYk=M$NdX~D>UvW- zx5<`x@LL_0X_6nRMNNae|00Bu>SF2@o0r$ zf^dFhZ^vpRu-xjssm9>X7=`o0k0GhP&{*(69N)p(+`HTlEfhAIEP?sMa-1>Ha?CMF z?X)zM3e4UtMfPH-%u!{z!>5v>|j1LdF?pin2oGcdcC?m<~H>4 zJKK0@yQp4{S7bldy>v;p9kI{C*>y=LdAbC|M^%JT)fYyqF%5avaVpb2JY*gQ=n2WWq}p>)=d5!{^h%{RAj1V^^npkLfh>H+o$t}>rixDBTYgR z9P3A$5yfJz<`7Fkzis@!VFRl}5hfBs@9~%;g9ed&g$uTO_T;}bEt{WyJHM}Tx2J7x zhl?^JDZj+}kI2m@w?Zny#$kqEi;%g@?+W(mfo-j$t{6qD{y1O zmOUh(2s2@f0cE1e2-60T1Pfh zIX14o2@gmu=C9UjVq+XJsFp!-n~^aY6e2q+OrJ3KL{p<~`A6aH1GBo?nS{6Ez!8Ee zxl6W|JjV1E>4f@hKJf!cL@BNv`2k>C#el_td8Fx?!cjgjAd>vh4kF|l&&PX>Lw;{xA}}NG1#Pz%mvX9RI`~MC7+0EgD+Hq%GTHI4 zF1((bG-b^bk2{?ux+jiU7Ws5+7ltsKnp~tNh3$5s{A-8RX)UGySeo&U%xe6%a>uK& zBro4?AqQnd!&Zza1U^^8IfIZ=jLA z=TvT*(GuEv!;#wsM!ups2dDEHOYPU+4u5(6VsCmS-=7E8y=jH^BsE6BC@ewvqcT7S zlGVZL&TqV`N#eloWK$8U1OYW~{$&|9`Qdi`o*%+jOaO3|HQS(|a!;=vYC0Fs? z=cBzv03>be>`#fiazsuad6on8RlvF3b|99LHlx5zS(<330!=5=cTNu7c;Df z9iHUIlV<&s@gdBi#E(%FOdPQS_i~uEDYgA`zJFc#U!@>=5jVp2i7TbmBF+>ZwmE}E z^1-%95T(Kk)N-Y(7QzR_fJZz~G(f9nNW(&>Pi%#3wE4u3RM7m7w0x9|_JNAu;*4NO zZZ{cj(~g!A$CogA&S@G@x7DCm{KZ7Y#o1?wPnTS6?}Y0N?c?iq zv(24NW3s+YI?UM@q^Ud+CyBh32$($C&eJrvNx2Oimaaq$U)j+fHC4YAB+Gog>mE_P zYrf7VZu?!Vm6@Xp7FTh)IWF!uNQU8)7H|ZZtARuBW1BzInI|GQ{X%ui+h20MP^Vkp zY6NK-o}N*|n~b#S_B1FbkcdIe{?5uHbsGQm*t8JuX$FlH;gL#}#}$CIoDfvJ(r6~c zcX^WnuN1_xHJLnkrNMpd3*v-PwFHyqgv)VdRQ)z@7BW9JO~n1!j<)e4b{4yvj_apf z_Q(|Wx4v+Tk+;U5+a5WiNpm$^2r}qpc5zhhmdYdk0eM#!iQ3G(IOPyDfK+CTvZ5kq z6B+g%myux-y{)Sl*=W%Ct<9;L%CFs?P}hc#GL0Vo#33d;+*HlloY{Nyn@c_Q#^iR- za_e&{vmk~0C^R;j^oKorhI9jd3H4|sV}wG3{?UBmIpus0=+*G?D~nyFI7=nIV+_LI zX2U~zw(^+vA!6?hK7%Qn5*WDx)=`@!BLtUb&ol(Y=StSD*l=eW+J{W9s0Ak-HMtyQ zvSZl#CKzx&)9}ZMEdVc(<=7NF!(|BEynt_v>_BF84-it)yFncCzQ-H5henJE6cQqX zNeK)oH0MuQ4=1#FOQZHbR4WSNs4@yztr+0~)JI5i{?zM;g`2F&+la7tPcXt`8iq50 z1#bFEE;DCx(MUm_|FS>*9bT?M?`R=UMtmi92iNr^l!Ujus5D>%SkDD4#hI#JnPYaavMduF!?TY5IPO1M6#(<(jXWsl0 zTBx?`^T&S14Z@}38aZ+!02)9Dm}NoIoRK}*JHo82`f4v))J!Rnm^aPNVrF1?o}c%( z4cO_zdM5G!nM~P9mp3-OC3cvUk^byo{E2>$1Cmm~qF^y9bTU5*kYTR*O4vE&NSF=j z@$l%4O9d6Uq`L5me_+b_gD<36p5V=UL_?q1m74mA-)whVHWg+=kevB*hm}w!X;4j= ztzdceO-|qf&9*|&-w$7?7box|O_P}-7n1p^)|`(+`UlS7?G11)MA!C?*vI=+&rm35 zA5}O6g9G>r{SF}iMJ4#et5So7_!AO!3p02KZ!nZ1(th)u($@T?CpD)7zYwV=?Yc>HLoq_?*jO3SmDLtAg8UhLZR4}YY4 z7(`wXUbTC2`#NF$5`{#CEX+V|lr2k;`Wmn~Z^Su8?Oa%RhERDYh>&8;2;i zuvLqRb(CBoqArk}u~1ZDYG-9^Ub>yU&g&Dv&#X=~rUV-U!5tSO7R4i&!aQSC~={ z$38nATLmgYbYE9{6kp^%^4AzhebQ${4h&RVNqF{s+uxg8#hiN%=AHDS`Fxv`E&ek9 z!(nn3n!}!Oi-woGom(o!$JLlhnKq~H1xK0bD1z!PByFrv)8h>r`r-0w$IOH#B377u zxv`bR7|5LxfcNN7P4Bu_$-fwf(HiTtEwPi94vb*_FGrOOET{>e_ZeR2S&~;#Ri{f3 z@HV;e8Akn?4UqDTpF*^gEL!rVIP+<}o&R)Hs#SfLmarZ!15b@L<#tS-& zLr8{oU_C*ct6MHpm$qWcSLBLYgGdmV(*g13q2dsOG{uVl#czil)A#X$THc=N4wbm< zA6)rhUV+Sc4o_&^F(4G`trrkhX+V?mwP|YO!x&+ZYmO`EhUr~eGBfl*QzQ!bx6x2mo;$I2f$=Gtk1 zxw0n86~bO!E!YBCKvf_yo<(AqCCVFjqo#gm=fYRP1OgN26+Gwx%u3dnB+fq}sJBTn z5{TWWtAsEAYVSx(O8`d-x&3&RrYts~BuKE|*{Y`r*y0tW%p2K4H5sH-GKOCS5Mn## zQ{?2Fkt2%un&!nZo=`)CkxwOg&6VCw>~G(@=k3OwTy?3aE<1d;W7$M!6Eot+V?qQv z0NpI}6J@CDSyx#C5GM4|U8F=0O&8iEvN#oKJA`0LnCAJ(o)}G>4Z`#2_8Fj$y}wZ- zZ(!<&z5j5XO8jMJ#~q=a6g>cWfb~lbq-`i?pr_n_57k6LC5HsP4@YATBO+sdoS0Xb zCpo%t=xL8DyyZ?{nnPM%a$UwXX(9FwimC)gFlB!JVzuKbRTF&4>4MD(Cp_c2<-1Ps zYatGcV!mdaGG(ErTEP6hzXa_(Jrn6P#bJg^Z{w%!_xq?Nvp7koTaFeN58p^L*E@?k+WCJTp$)xeL`qd5 z$vh9a-Il}a>6Xc8s<-Twymi(UBQwT8FUMaLdY>66PgXOPHKTm`@35{jLiF<86<8o~$h#?JC*Y}F8XSxPj)MS~&#|79& z$Ffrj%QU-DhvcT6jczhvqPlDF^r}aX5EhyHA2>Ebg5MhK;$o2UrJ8-8e^e_~KRN1I z)sJb2-di@cKpKCV?{E6#;b%Lka|cjZc6?B5K^jDnBQZ>_|8= zLBtFUVDku)@KMFKNH*(M`jRybqDK0*z&ibQLtI(Lk7UD~^m&9of@CG1TCVVZ3KQq; z;)2z8c|D*su;wFp;e4qFOwu;{70)<+{Ohja-`dlpA8`SPEHOFOcC-7UN%3KX%o5N>MJ|PcCN-xNIVW3 z#(!j9=U0z_Xicn)(^@Ni4?vL!p2vEc^{84`MfDj>3Xj#E{PA1rtsIAI+|3Q!F69ee z(*R;0-(c)XD&M66IE=z=L|#uDp3(4KC)LBWiv~w62PZ}+y(;h%@W{*=%f=ErCe$sz za_*hx0$%m7%rH9gotQm>-AR03+(43~84y13@CT}C3zwAL93mlH7y|ZfScG;F3$Uay$f9k60PGHZf9){;8b^1~r6$1>3$J3k zpJ7hTX0I^06LL3;y2yq`rN|rP*Ok9#8#=dT$=<0BQb>1506FWUtw7}T_F~6k>g`BiaWt(O` z8Nj50X)dEXD2)1J!*=(tT|Y-0B|WDgUH-6cy-Suqhix(lDKDje2BSv;_=g9w(rl6xI-k4wa|jt+bg=2KDQG&g7oCQjwX_KTr%E07G7n| zBi8O#(pHk;guf7!fvlQB?!=sI3rj5f`jSEztX^2R3y0zfFizq{tzvPM!rS(DyS!M| z)s-(Zh|X@!*)^{`zdNG=qW%v`$|5*C<_r4zJS4YqGE|}`6H!H-nF>2NWZyzf(LRP9 zmkhy9utVF)Q%K%i(%MvFiR`mI-Zi$425Xj;U}-u~R3nza2Tv(>75XHQOt-v)6SRLz zyKi=I7;rB&Pr8YNq8~RsE39N@rBwQ4qx#L$qmU5fVcqH{(#_*XwR^cKY?wfm(Gx+-m<^DT9sG!OF?<0LkY%{ zNqr-wehG50VyO&GmRrH%apv3Te-l5w!p@B%_Utu4#(nBe9;Mf$y#70uZ+9XRn|qOP&EUV7-D zO>|GZmiXUufB?Vxo_n1>g6r#H=WC4$)=(XW9oON0;FR7YQ=NqST$c@mAge=p_~87u zM_Aro-q_XDofW9u(XZTe$iEE0m8i$r&Dk|t{Yz~8TUX@LhF#6g%{z9V+ceoDlTb-P z_fT6Gcc=?}^U>7Q5S`#VT!n$&N(?Yh%tMzWf&{i5IFJDFWheL=q8N4lH2w@ z38@LjD#+5!AtY0mK?kNxKcG)bLH`g9AwuvPUTrLZb|7_W$pl<*`)ER#>B!lGNJlce zaOOSU1G3aPI6hF%wJwu!Xvcm<&y6|BVO@D)hhgNfHfPYIca3vL%ta z)MbQJamyt=@F-u1D+9ZB2{Yqm++rK`9DHhb20Po?!DewC5Op?qIo>xZpSZng8A3rnZ07 z?fJ3$wqk}oj3l>03#cs@8wHm}wm}l#EG0_i2!7PSQ%F|N9=PI^BbbP3o4nYhOMU5E-j~ zIszn??yilPYJKV1Vwz{M6!;@@OrC|HDgHS=i(w1peYM9*BhbKyMeF9o0=ri6n(zU3 zHZiuGELC0>#wq9RN3&~H!=|%3l3Lhl*@2guM#5}i+``tUy+ zl9DcYX}n`;yH00a2N#d{xmI&ZzwE35S3URnLn6%F)ScI zcdv5KjKZM;7c_rC)^dg5{5vL*n#)D{z)7M&4rsZ@q>!@)w-pmdqI~|2CK1$WlMlf6 zP_!*}z^lfj)U+vB>jW&}{W~`e64}x5o+a3E7Z1>{^gn?B>5^<{6 z6#Pa8l|+gU)p5$}NGqraK1K-MS+C?~1hCDmgWUsGE#n8?+I@|EC8OoK-afzCED}-b z>PX7th|nsR%`&hsxUQNu?8nPLtE2CNFRoN{gO>nZ)@{%!-!!T4iiojy5GQ;&Cla(O zl2AVzi#Ip_S?3FBh3=IK#!sk_59HRZ^fiAzv6((`Y<|=zFFhkI#ncFm>gudy?0^zz z^B@rKfdlU`UaP*%2r-$G2!d{~@5{6_!Nm+D1Qj_Utt)lq>apK^y3#KzVgolT9 zS)hGIqe7;v`l-W_;Jd9cT!6)hU4M$EY6DNFwatjxMBF;HQngX$``gase#z&7<|G@W zmYJ=7_xAWct>n%jR!*_Ex_<$EB}ViPi>Nj4qnwtbCKb+7?7dsU`FRhzXeFbVhi~Y~ zAYBvH@jF z%JHKDe+?_2^z*dx)iI`-%*P@3&=Xr2x|vB(iBHU>2w{y8&M@r|`#!1QS~}sDpUG0$(i(V~l9JZc zw(TJ~|vDjh| z(SdEv=v8UPt4llO-~ynt_pxJB8pIqsq~jDYFQ2MfP zf(lb%bj@kzP}*9+YOgxemiNgstU6^c^EdCmlnO&D9D$KR1Nwq`SohVD?dv`{62Ypo zIBm8Dn0{%aAZE%ZxCo`v*H~rNNV!hWkah`%7cvQ7B-+$mnx1%$3^ra5>bN{DGm-@F zeBIgm8PrDL-$u&i<)+HnUlC&Xbccni^|NwAphjs{9uOT%6f#FFwkUo&mnJ zIH5}v!ZSe(JEnIYmBw(k*J=$VBa#H($(yw7tEEweaJxs-u*_#eXoDNT`-+)EHHu^2 z(mi=3WpT_^?!Yh2|0*w}BA->8GN;O4A@@f_1MAYy+|6K_c6e?80hUYnY8Pe*KAF5^ zKk4Z!E!|kBxpvSc6VPxQdJIBfQJG#6HJ>g%$jKeyKT>dW)?`-DWbe{mWNdLY*ytWP zz$;8b2%4fU0AV+6PKP>*Kh1^`{RiCd9{fEla(RQ0jJm~@A!+)BZ{aDTl6eKS}S+zWPL-D4pN1k&@~| zGBpL?=2eR;eAz%ag)97{>Go%>_Yc5(nms)|F}k+hk> z)KC`_Z`eiN)9quW?c5<_4dEA!g5OOkO_&3B|| z@*d~=d06wSmM*E`ce-RV1W4jv%R+@dmA{=lC&!7<_$8fr417VmuIGHgJ(a07;zgJP z0M^yhs2ljL3#@yCYll)?$B$|x5NldYVi$mIs_#j{@7Ma=#_h;1$GrGdmY_x)g)|yp z^&+LCHR3(n9>r)Jn)qE>T@52iJUnSuKYM7&_qsWkEHn<8BI+W9W8`zBMfZfV*L@%C z0Lp0X>Qsf@Yn9fjW4%eb`EI5g(Ts)B0G=jV$wul6j}P#`jRTU+0O@!S0CS(N!{jgY z*_u-aCye03?h>N{t|R0|N(Gz=_md-bY+vJb%YK7tWs(d;K}So5#BfgY4Q-c_a`4SX zXDt25{>z2dI~C0|khV~iW{ z{jPuN?T~ZL%ej98A%Qr3>RBBiiQ4&er}3tOVNuFjD@FqyHf~m20xwPY67u*++gjMo zJhxoM09t3eSGQ57DrvFe`r-FdM|KjSB{sl0|8e^XBd}OIx`USli=N9Pf|r$JM$}h- z#8mx|YD1#1PwA-Ew_ahMFbbC^UENC7nb{=?=|!AYy{xn^|989{>AArcT4PEqcXX)> zj4{JbeQF2~LOGl6KJ8UrPQeHrU+5Pnm)Pfr=Gz$>hZ8x15O!MF-?*QB z+Mi>$EG@vG6IkSC$HLT2GU3%=JhUAiXt*P~Bc=qLkI;&DpinBm)!jujMS{=IvI&5t zcCb3+7rU05eTJKXvN#%&-IBdAANhA?7hkEG3oWkR_y^Lw z<9gg))Qt-tt|;*=6-b+7U{_7a70QRxkdrlE9p{pObGpU8F*H?!$dlF@~s$yINW}I|`b#_Ztiv8s7WSjLr4zfdipO8^{#{6nFHV zJAh8!4-qPsegS(XT_KvaS=;1(>+hbtBRF!d9YW7E^$DQVN9Yj~IlG43?BvgZT-i)Q z+tlDkLHYy|xf-SUR6d!BbUOr(FoJMQ!~rtk_U?oC{<&^yI4_?Sc-sXs-AjMP;e_6sk9s4nQ;Q_r!!v_F8m zIK%|_7z-ZXO`mV#&4We{Qp*0rm*K~vu&2%I-yejFDgdBW2BGh`((UR&E)K)q6M=o& z6x%JIn#Pj54dP8ND$2OM1rq^1gs(u$)mMIpge4vP`4eTL39nnXOcm@lB1P zs5j#3g!zzGD3syq^Oc@l;S06{F9;q)a{tQ|$Y9{u9$bBrFm3H=w;pUy5+GjGXwV$)(0(B z?+Q%NG(>?3QPiG#82mQhS`~bm6Uq=&I(L6SQ3@HKRL9RB824!i59kOR&NoVcA5x$N z@WTAD-$fY9)z_8t865;BNFKP23)_%Oh;)wrms|Rg+r)#(yO!m&_-&NBN#evBSbpE& z&3p;rGXh0sU5E4<7LGlhCU1N;jg$4gma>Zz#%ffO7;GY&K1~o_I`ef0et@{gW^~hB z%mVI*L8f>Av#lXk(Hi8s)w{iIyTJP;u(cY;)2oZ@Vj5~x2JBthrcSr&kvv3ORn>65 z4oyN^k94s>!NFPGGm4h>47NnHoHYvAJ>futtG9hK@+00-mNWgGN`T&DE>@pFfG)fP zUGzy>=iH98s)HX?hpcA#MYw~z^Gcp_#&b3}sJce}_Ax!*;E5a2`ph}5^CEW^uG}=~ z!q~Gg8+*Wxdd(ePsF@&vJiy_vuDb0|a1LKUgb;_4tYA4HpQ}ewfBMdZKez>~Mc&uD zHbF&aL}A95=2?aBPrKZrx3LZk7$OOh7dCvgEL`bnZ$HoO3rm6Nb%MwRG6dqo4KNL> z%n@$>5{QDKEC}p&Gsnd_$U;+_EPfGA19tcG!5{6T$VDpHn>DurSN4;ln|v^X&^M=C z%yx-IsPykx@W>5$_vnpn&?06STN}tKGEL1&lQH}H)patDSM(8Bv3i^672M)Au~cJR z)h@#u6W-7dkJTwCB>!kHFUy+*iw&<~NA?!LtFcy&KcjL^cjWU;0`-WfG>q!`vR8CT z9`4UMVQ)^}O>DDGBz;bZRDVL**#(iLS(&cY&Jb^gkT|XSF8}Jq!EYiO_&n|ValUc0 zl5U9)gM^uPQR?=1M9?5=siay^R!7*#UbZmu3yL;^J1sUcq8;vlMllO*uAujxtik@6yo{gZ>7Sh7>Swiw8PgA;p#JxDuVts!HUNM zspR8hTqLtJ0w=A&=;EawV1X!|)c(QOR=`eH#A+aO*Z^U3v37hvc<|qqnd>)<5-k=vDG9k;<*>aH0Tb zJqsa}&m}x|)6{4yEyzEQ51tGxo&u}<=*_A_Kdgz6woP`EF8ZO~_SGI)oH@7MyvNV1 z{U}}*Y-%!`i--@SHz0<~r+}=t{ox*|^N=nm7Om2E6jM~H!;llWcL7s!ADs8=E&7zg z8Zrh#-PuZWZiMt51Jby~sD{@=Xt#AFb)E{vUOLmd%?95sjAA3O zPNkt-h+vax$VR0C-nHC+oj4nj>-!KuvQI`d+%CY}aK|$?o zuSSus0)#;PUVqL|=uXGb!_FP7pryTpvdY=AB&dq81J^tTIK*OcYw&|eQJom3U}H>RguQrEp6IVpzPq4kPQ!Ntqe+u&9jSpj+I7_1c&fp}7w0}Zh|!f(ys!cG*g5Msin`m6U06oR4Q z7+-2U=qh&1GITeQ)~qt;iI>GRdpcTnYU8ZBWom!SFMBCq!ZZ^H`b4;&8MLoY^`&K8S@ZE6_3mYC z5jY^Qq*?uI@Bds}fB5!fIaPN};Zjxh=J}Pfr9*TOw8%z7gU3Lgh-?UH>ObOV&3B#i z4VztU?XlA7={(45n(Y0NW(B(mmN#)4YH_QRtGqKfNq9%8wbU>wK1z&ohfd^uK0$LM z4N6s@DeW;LN-z`YbfG%u!LmB+yMgu7NrrwC)=|i6F~GLnmdJIj!->bR?&D3;6kJ?H zr}lrTbx(mG=Szx0JWH>HFR;C7r@2qI5*%QC_5I(f6yHEkIBxCUu6@K`8arSvo!)*6 zbc~i=+?0w<`4p9peVI5*Hf#{Uf@~_Gm~Zy3YArT0RLk^?i%&mMw>sG>xnw68ws=)mkuIoxb7s&RqLH`Z+}J@ z2u3vuQQxD`Uspe$Sc<|XM)%}Zo#xhNtK-G{t?U*K(vyft7bU$>w)?}oZtM%XqOyKE z(be6R@>X4P*~TvuB1kfGSaGjsWBdA=NtJ=M>5RwT4<1H_fh6kOoto1c$3d z9TE+`{L=8J-AgC8Qif`s!LqSei;uerg>6)j{8cKlNMp5I=(lm&C^bV(rZWs?>c=lY z(QiFX2pN{F@DD9|*(z^~q=R~ch!Wrw^Jf= z?1nEd*dt0Ic6cWB@`7pR9Z6i_0o^95PR%-6bONU;`!GVt8(ad`QV?)<6Wk^a9p_9u z$|CeGxLBM60;hZFaIz5fb{LXlFpE#|Pz?9XcXHf{ug4n-u`e+$QtHX1L)@&xX7l~T z`J@upM4(Jqv?!5bPfZoePp&_z8UEz9-u@~cWXT(L} z=z(4kPETi%mzeIn)?+^#bs1ylrj=_KsFDFE%Tf zk`GOiQ!{@;x5?g)G_^S>O!E29?ljpv^)Beh)@~wG>KgWKR zDcz=aTY_=C2G`ApjL0!hD#`;ixO9ra!5eVIl&GMx6s+})`Qu|?2Vh#7VoiPJ@#OKW zT5?+@Z#oz5cFdnU6M2ly2^k&JgB(p-<?b^q6 z+m=xuDe}{r5QB1AafO7eo*xVLsx1CC-}!XXL)}NpL;wYa`kA%9JD+~%594=MByB~S zb5KBtmp9))de-Yfkl`V<8iXh!zzNJpZqxCwJ!H^>l++JroD+*HI2arA$Nl^83Ikr| zkuc$bGpu2wZ%38-27cX*Hn*PoKGnsU> zBNnOXt@^(hBCVSh+olE(Sk<_A0(kJ;PMDA&?ZB(+KI~97hfN)^OgSRl^a`x37FnAW z5c}Q*0je7NvI%rc+yjvSNVvI3--tv_Zsydb{V2XhQXYPDY~WF0+~fjX8F9Z8NK59( zH>9w_R!XQioa_s~YN-gvsuN-8`Iy|3c70DPWgCy;g#iNc2FUSt?{_uD>qRdf1K%rZCzVaC;d=T0sD_stA;fe`^W*he^D#Cjh~w8=Yr> zP}A)+&e~7&g}tJFr7{~7h1fSq)l0ChbJmX1X9I9W51DY5TPW8UePm>T3=lOxB}p0ooga6Q}S z(C=?c4?>hFqiR=`oskc6`^gwa@R0fPJxk(oq}QUC=mS7UxA6=ARBTUR=9`XK$5+QI z)0YL~Z3w4lManyYZ_bV%xb$=W^_T7^*Y456MrO;762rTCg!{U$9w=J;y1MEEvLfGy zxkI}~F8c>C2!xy}(L9#Dl{Z#4uiK9$p?gFQ27y%=9nJVYeI+K|T!YPSpyMNHGtXxZ zT=S4iPFW+?rg`DBLqgr6Drv%?&{uPht>tI>m+$uR%ix;KdA_4y z`)vuK;b)6y(~(xjWWr>=W&WTPU*Cons!FRg3N&C+Qz*(_2_$6Kz*5wvP#gTi)o1m@uwt)(jJF9AF5Ub+miI}gt>TQ!j5p}d}N|3*o}KX6vF|fr)f0Q zgIO|ISkMZw+A`fN_D3WN)Ehl)ob zv|fjaXsK^kUz!dTU_!RK)*stscC44{8Y^nYr%3lAjD~p_F!l~9@Bh)B554SV` z%j!%?wB9HD`aN<)u%P&|d`}kbDaf0#6Q~^Qs6Hbjbfqbtf8xplAwqw|H3!xy`mYb} zI495<=M`_bcV_R&2#)8<(|3>9G^JkOD;}dAso zYJ1^|T=IL4A0xeexBbrhP}b%eq4=Gr@0gDsm2B|V1!p*qfa6Y#g-5uBB(ts`JjwYB z1_9JY24_QthMbhKZRw~OR&|S;1WDh_U*P<0Jcdo?6SiESl*H-4$@Zw_CyNk!jp6RDf{mOF1FIpY9WYevK$B|@~VD#zIuH7XSF0uz|t9A-f&cx4~U%>_o+>6 ztA$qRu{s9K@{=LKppDzE4IL?a>F^n1N82dmkUEi3YZ5nFZ+)8PowiSvU4_A!}D zU;^ugyU~F1PGdl3Y?xb;;e?xNIN^-m z8f=*s1`R%vfc+1j-|#5O8RTN_`iFA6MFsPL$V*RkN zTfCeg($Ktrf1o@8t+_IJR28AI>avHa=`HHk!Se*VFvgib4}gKu;Q9C$wC=?%XXy9j z#gpO9P9~pu%$J2koPPJ1olz=`_jJyCJ$1_PhQXn_)lIB$MTALO%Cj4MWj>dXo5{Dq z+tYfim4#~C<{}2ThcEyEhNl6e|x7Yy)b8Oudr|CJj73Cj)K`GzNkG(=) zZj(6fyQeKk?g+Oz(qhL}o7;5Bf24m?aH*{!OVcIrJAt!t5&yslbC%;fpMC&8nk?|I zy#RzstES7Qc)Ai*8hg@Q(<3Q$0fn|deY#;1Nf!p+`{C@yl*cM_L{UZ17a9pI#JlXB z?s(d_Ig21JQFKpQ<0w<3a7_xJUAF#qCH0{LaO#~fRCTv29KVgK4)>|s%b^vU!!+8^ zyc-OQJ;0dEVAN4Odj?m0>@;RT5oT#A^2vLzWTb{Fco`_S`m4n`vL1>h@A}>WLnS2o z>XW2$dGqWUi_7<;!qfw4ifd`GS1By|9BwRCoP+#@R1Yi+t_N<{a-yn6n4F_?Rd=^w z_+RajL;L|EF-|i-B{~gJe5LLE8;)FS@d0RH_l-nN&2p7+Q^zGObPhsDTw~<-TOo1w zFBB4WmMggU#iZVUW++C%nU3MAw*=`R2RET&YI%>-V<4nkee8 znbxn{Gef9_ZRqUiBK2b`7uK2O#%NAhKQt3h+(N@ITBPT?jOZ{19c3aJId-jb2A6ox zAu=2<25NqVLt5oP)U8-{)IhhobVV%})sMaU!1m)bfiI0_rIci8N{p=uaVt###VW(c z<|=6_#XdAHM6&obY{2wAlSV*d=7STrUxKqK&9U=G9sNi-T;v%m6-X2b?`JWcTRHJb z>O1dcj$O>Z6c$IZg6}hhUU-F@CgJ@OW}i@UsDdD;r9>wp&PeSeCZ0*EzuWKdzK^Uj zUR>ZPw@abEkLBM?0qk^Je#@#}72CUtOI`{xmH~jp?;^UYj-x@ob*ph@08_e%fYTxE z!>=E{+`1Br;}VhT8X!BEEu5_q!{6m~VmuXPeWv2SzDLE__>d1)CY3P}kHKe8c$S&P zEm}P7EE^XJ(k}v9VFTzVMGa^>`Uh5>6%?VULG1>1`>Jcvj{FJX-C7w!@ET+TDjA~W zaVZ1HX42r8fB@Eb1zjFW>RtE*1Dq@~G=h54wDQ2w$@pyVsFA-UHyIzUS6AVIe#DTO z%`Yo=o8$QL#q3|q1bru!0YIUZ-+}@uhww?RlX;z|X@gVAzn;y=e3)@l-vwrwZy6(P zZ&)x*X3XO64EU!%R>M&DwHLEJR+$c4L7yb7R5ng~>#HH@ImqLK{mEHljL%d74t%la zsrJQCq(e5BN_%P(hYD$bga9loTz#bth$zD6B_-ksd(wopcK%$;@1V_Uq$820oJO%H zGz-ls4Vki-Qe~+Y0ckx$!|mx6$wiEc>M}3_0lFJ>Lws5+>1OL}Ep#_y&^z@=P#h9p zEKCCRjD^=+Za43DD|=080RHrmD{U2a=;cB?7Eq$@O8JkDY?NVsF{j8C`kl&7qb|&~ zoTuS-xI>kydYVYK>R#2qr8zhcZ8Qk=C2r73iiEg$l}67w6>S`un5KN<+K7CQ&bBh- zKwK9K3U!z|`YV`W^o`SX)G|PM2?61Vw$(U`plX*apl$XFedN**1Pei8MjqCCGNWe~ zH31k7DS%C|lmQ*=OlLRp_E|{{6SKfrMGqKd%%tek59~d~;{`TdXI6kPtUbLf9ucL| z5G1otvj5n(mpNpgXY9&hx7y+;1o!mUIPdfn1%O3H#J!OXIJS{OdMjHGSo*TP79CDI zLj78kihZWC5^t<1ky2{hU(_-r*GJOARXP!2yKQ-EVyIN6iKSs9Mogb^3$~Y}Uy%Aq z;(*Eky8+*^`qm8IAC8W9C;NM2O3#cZCufsW8`uLhTrICEK+TN(>p*p^{;;z@e*1Fo z?dhKfVYL|bm==9O5Cnl!YsFnL3a#+J#@d{(S;H0sYmRe4! zY;Iv1^g9Dwxf`AmYs0yt%mxwe)GSOl+vP~9M=mK#PG;=oD3-K_)y!9=XD6gdt`?BR z!l-sRqii;`!#bhY+6$SX6Y(T@(>X_X3HedPFpWw_FHyIj& zahDmmcz>fv4nnbB$;p_ic>CC{MzQG_y}4RB$SbIOTtT3 z2m<7~dVIzz&hdaYh<`|W5p#`=bKTqqY-pZg&z=I_V!bo==eI^+Ud|4dPhztEF1e_iA`l6$m2}AMET` zSrYF3`VuRJsR^QLo#6|QNYK(!B{kKRlSoar{@^>1hQm8y-!c)2Pi9B0p7W>`gonin zmg0$#7r340A- zbJ(`;d8Q->@T9P7Pp5Ql;_jnoYM^}&ZS;*}npV{(LbY;9n0HsCZ!1L0Ns?tF!~1~8 zS}i;fL2W6%Rd#<750mwzDkm3mz;IH6F#1-dqgvrL>m`K@m|Bw37=y*0n3T&1Z?Xbe zb&`baW-S)zhpmi>A!>GF&_4W#B4ZcxX{{2p(Q_z$8|-$fx_36Fnl6^m&5S3Aqh);c z8!Elwlvp-8i8`dHnZ_e4apOcc4h5EvC?#KBXH#+%p%?8~Jb>PFAw+D-UlEZJ6DaS; zt@&|FxR~3|-s^Ei_8@*$w%{Squcq7FkA^@{F_(qjXry4Zu5ghk#N2Nra!8fNcRiCu zLv_D!9oR22mExm!y}pHQ(^HB!R+_qC2$+u~qjQM!wN4Yt<^ zeWkq8a{jj$jaiWspniUd07^|V&j->4pnIsTXS`9{hM~F0SEi{WfmBmhJ&0ZdUbDN% z7hA+-nloNtZ+uwLH!NL78EPq~+ru?&cLCPuND%OG@qmo#E zrg9~d4q48|HWw;iYSCs?U*&osO(=l;Btv>++Ln0S{FCa3o*H##&Xzk_7iyFlI5^rg z(JUN8VNzZa@5qLUj!0n!7RxY9*qbMV{rl(&p1@GpV&;%e4>adlc`2Ro?o#m+1R8y% zA%#3}7s$Zkh5gL!_GyKB0(KZMq~kqn1b56V$jH7xsUgX;FP0ImvP*hFOX%wdT4#y9uuE-58{C@Y-$sRcSX#EInGs3!F*VX5aqA~luF{|CrJrY) zQw2$ywFwGy5?mTm5XBf6Wbo8Y#YjD3}kag zeYuaevP&{9n-!x=y8?Y;Q)3U_PV6e`ku#2BzPdrWB=|3BdcKhN6o8viAq;8uJsi zl@CH#nZhO6a(&CXJLkW_(@rKy^l4^xs&A}rQ`I%rPYR0If@V~>x`DO&8-SywByXzT zt}D1}ekF!M9myhCv`^lXDZK0ErNK7=h5nk~DN{=(rW6pVc{ZV4^(fa*nvZmpHOiEt zsLOtI3JfuWnNlLE-6>ru1lg8-bJ^?f2=(Wa^3f_VfeK|M1u6kl4?t6`&yd%t(gS4; zS-zE1tYK04Q@S#qy+sKt1{pI^ME+}G0J1Z%EQJQkrYxxi+xi^m9ekoR^Y4hUu^Q))}51^Xa^w~#Sdqc+V&$uAnvU8OrD#rbs2w#Yk z*DAcDBrKermI@&g4@w}6uEGp}mgpnp)$67kyk3D-s-c=~h-aK)Ri9@dLqo-@c#u;( z&vEO9??YIaqfgC5H6Q|DmB}G?Tl9jSNdi}_S@3Q03@GMH|V%>bmTjY!E-ReUS`Qp&-2yP6|f_mzU*wxIv|>mNy@#x3?Q*4o?C zcxu%D`-G$DNK>0Fv-(DQh4$z`4w#DTw37(7_5RsKCOZ(uVOZ{@U94W=pjMEFU-%Bq z9_wrIN9VSCfA%L$hA2Y{V%R@+oiW(0&$Sl4{3cC#=h zaplG=zhTyMDM?N$vELf1jp13wF``I(?J_Bm44wAzzj!o7}5uEY~ z&^~lx^9EA@w}c$JP%Ym^NfkWVU;?bU>UR+UAZB^-Buca)t2$ZbSe`w@Td1Cu74mD$ zW-oEDO8l@U2yU+DFZwFZ64XZ}z|OBDC2Y?{6jYtbe27*f%5gu4B_eKOC2HT;GSE!G z@f8#g{y*u2gfd69hF|2O3tJ=2M#ebfE7pU6`nq&24}rH#j*nh zz!X_1%Ao=;?Umv3xCj~I)M$9Dx!)qnB`c9yPY?^qAn^e8X>&{4$_Z^CUbr%bVl9XP zI0ek9@@NEr{5eG5M{pz2@dg$XHe z(2=^AFGtU#uW586RY4L$^8B0)8Qo^N<#v9kwpuYF7n>?Rg)iNOqq&c1t|61p4MA(W zjEef%i--QPq>|{qLq!YN2v>H}Ggnr|9R0g1$XwsI=FDya-ZC^4?vpw!-7Q0b4 z&1pq+y`Ijv>kt!Qt^&Pvla2da4LoLu+mKUpcK%U&T{$63<(J0(p)SPX8jj&gb|pS7 z4Hz#ZVx`ESJ|;47UdiF&I$l&HSD_RtN^}*YJutH<>iN^V_o&=t#!Fr?o%s}7Y79|c z%1D;#^Jlj^u1U0;Da#JxNp;3x140x#i&I@me=#NhYKeq*Dk%vKDk$1qzWk03l!h%0 z)J&N&36pOHGd6PS2_-dw(-3ZFV9nm6kk<4ZQV#Vo#2 zefbaM;nUZU$vK;x#A_>Fs!p@30l*c~qfz$eS9gwS@tFP^NkuDqQ3c|}a83#<@mf2@ z*)~rwS-jKSc1^BOD$zN0qn4}i!Vjv6iIu30Ng|E*h@~oW=n$Uxf_A0^$a*rhDlD1N zw4OG3XZA9S4ILNs3$|s3&w7#hW$_j8dBvh8GkoMMQixgt8k#lNSZo|XkT#8(%AXKv zwzV1KXBLcZwIydfK}i)1WSlW~Ovj*7Ft0uNMFJ$ET{g459F+;fEN14jBA*RTmz-p- z;>49Rg;$}Ap~kKP-Nu1o=l6hH7UE)9AU+^okmQ5x<-*oo>nuFS9 zH8f;^6s|dmQKkY}K3YwuiZ6w^6*j~7Fk?lc?K-)OlE=JL*ji)kS;(y*Ps=1)XFlPw zb*^MBlZ#Y}OAD{@rieo{thUw|X(TI;f8@HBTnC-@VOIE=awbdBY8Kfn7T|~CmA+sb zodka2unVMysH2M9S~H)oKSEt@8=|OTuarjM7|NDtSci$2UOy*Mx3M9HR=Ck`ha8 zUjfE8K~ShlO~Ay#;^tEeCqvKg%V2fz>F)A9u5F>rR=SxRw}??{Qn^`fYn9pPv}cKk zoarmK1K=@qKH5GxQ?`!s;hhkXUdD96OY}iWZ?4Lh6{wn$$Eo#1QT-?TIt5CI1Yy*| zspijCg%Ridq8udAOERxTcGD2$Xwt|Srno*sBw`0}VTtjL7g6D0Cq?=T3nNmMAR3c0 zOm%R@1S#730Tt@wCtL4h(LR~hv3f_1QTs^A9X&^$6Q>!fDXM(f4;5+rSz4!k7|zC@ zU2LU-qsl64DPiG2fdaygDXa~2RVAKip{nL0a8)?K<=}$o%c<7F_EimY*fpA^ipoq0 zO;VQ4N;mPSekTe)Ck4{7n4?C1VdVpnc$U+-&!ZBpEC8CRv2*hcf>Sv$eoHE=o8|m^ zOvfrYwU}o01gDQ}t%DIA`*rtwK}HqQ1S2thj=m47{_L7>Pu5n!Zc&b{N18{P-W7}u zwO#Zak~DCTeDs#Qh$^M3XrA6$g9KLvu10th8`~ z6%_SSL9*g$ID_LMD&`FQPiYF|fUMT}&ro++@PJtdHr{PLm*1=^yLbsg*l z=)bsGW6B((C!D*;tnV~@h#CYCD>tVKt+_Ctr+?QE^@OJbHe9ZHnNsHXJV|?+tf>i4 zKS-upVuai8f~O z!b~}}(0XA6-e>k-As(u(C_M#wHxC=k%;OjK<2^Vz~4T!jTz7)=3f!<|t-Av~f1xYxuAHEz^6nTWI-+p05 z5%>_x8ZPD$>Wa$C;Eauz-a7CgvF9Hykb0#OL*)wWB6^FDFoP3Uq3R!TK79E#enpv2 z(paq9%zr{~NGV2DO>3Hw;I$jV|M=#ye)&{=?j?n57i=b=R!H&Ia>4>0wlX|?b@|W% zimWzNw!oGt9nuH;fho=Mp&o7%hsL?36{`9`tg4(p{a%w1+Gor?CJ2FLZH)*{P-B#> zU+joQU|{;ic)6M^VGn=?Sq!HBUb9k|P5b19?pgXb{q4-7l$Dd1hZ#?gMf4=!%BY^m z$8xouI!vRB(oLMWBJNbb&M248cV@J5CJW7uuVUsss;`{`;8F&vlXX`MWPtOx0dkXq zRX5j7YGa~w2Oj-gTG;)bk~5ihnGoiZ*6hFsDYkchRVE1DD)O7;X4XdO7>t(G2U2~f zcU#*th@|m0tpH-b-cI*HQ8h_2Ldk|g13C-!a;|QZYCr;AX_hBgW-$#t8E>t@3rD9fE9jJq(i+8$&8HfRK>fLxoi)?N`^yM5?~drkf-LQ?hPaVSqAe%}zGwlx52j zwv{53v!=Dx@#8R~O<0NvFAYNyfymt#C45M5I@zaS2+>4|^6*1U^)+SjvhZ6WbGXL@ zi`K_kh2J~+Nb#&k8Y5y8W?gn8bT{7_920LO}t*w0^J?^%I8rhV1j@-r1hzh>pb36G^4paF3)bS-b9K%Mk5 z{_|}~0pDtx3Kj;=zJ(<7sEAoh5xl)`?ty%h>YKMwz9!!Y#xOA!bsmP^vS2v{1mtx>#7~fL3}Q=9g5Z znTsk&t(l^;#J<|p&9)7WQU>^7tU=pyy5BU*(1tQPcrg(ud)DCFtHrnd_#!<*dnjR& z|CJw1?JT0tg(a%)C+Fu{>qDds=<6T#4dyf`RxwgY=`*7PQ|Qq)OiEnl6Pbp%4z+d~ zc!qsjxaJCftL}0Zn7O`5nHFKI4!Q{>mF6R&5I#Zo{Jfa6r6Kd0MTEE^5YjqYStDi+ z@kHlG;8geoi&$H#oMun8Pk8xRl2oQmsZa;Q1>;1{4?Yp}%K2e_bxCF&FjHEJ7b613 z`52*IEnA!m{e5dnr-%F0X3ZY%!Do_uNHv@4R==8(c-1F9grSRM6_;pN*a*{~3Bx)M zRGCkv7gmvLeR@AErN4)cC67!S|I%IOcUWu4czZz^LFxDO%xwEn%=2e^pcgu=mhadC zpR!@m@||5JXW~?k5134QV3sk-sfF#wD;$1yICgUSG z6H+gxUn~Lcn#^xj#Feu|0Hc}@Z8c>cCA)Xey1GIs?0WS zE|t>ol_~3hGcDMYX8K8GxHY~ixor~P>4>i4?)%v<7EHzH3OlffrP2oFaACw#Z;TTeHOH(Jl#xK)lLOCUrBGZUXN)?kYIhn=iKA~LFzMyvWQ^ZtVA^No<@I%{P2o8A1C?>Ycy| zfk>_^)4ktIGJLd$Qv$~mbkTcYvdwj^bjy33vJ95AWY!i+Iu#-WF!q*P$YdcBt}b0s z1r-Xta&illLfWG~RdwY|?#S3@#R$IFG#33CuEnaw2Untu^Nr#v8R3Gcbc&4DH$ywX z%KQ_RYWb)ys&;3Se)O4cmc^x11N%~@hZQ(_g{$)e&V@)8)YYs|v6xVV#_m{M#5z5w zS*8Buuv6bduC8ozr0b!1X%i?oRS_#^CgMer@!{G-6mWnhQ+SA_&W593tR^d#F(y|` z_w1fb?>FMDkBuhNqbfYBs6GW&ufG<~Ks)S89i`=-)YxYqEsx#&JnQa>C zO8h#NUm1WVR3IUSI=bK2SRSA)AA%ivf%nNbsPHK|%H7AiYYNlD=jG-Sge9@I!dDkG zEcAm{k-rfKY7x;!{LQ2>FAeA%<})-6H%IBDmSbbsbxC$)Qq&ilpn-CkMci*f?R^Ah zo*3PtcUYvgyhRVN?+9BiIfKX{?Gtd(_vIwEvH_xJj>-~~{bo_?*yzF@Bu_zlfV>CF z!G$jEAfIZb+)^ft&vK^61PlI~MQ@8G`G5X{=#aOWhVL-pRr)biz--7=VcZ0B25)U> zNITZ&WlbbE0%rLsGv4SF_ft_;v6IY$S1F%^+ODn;@VE3bk1@J*f>K&TWzpQDoYrf3 zR8f$bB3i{tE!`_K`jmRg895Yf`moN5Lc<{tM5XQ)+h@~>PzifBZ$VO0&B-T+!XN-M zu4)0mm);gt&Pzcfkd@r@k};Cnr+yu|hDxK1*cLOy14hrrV0#5ydg9JTSt(FLaaWNSXjZ5=;&%EiE;t#3 z(dVg4ZP9~8al{#g?ONEjdVBfq=vK74tMYwj2*V9Lpe*+V<7uU2)9`zTieFrB4w);q zr;1ts zoA=6ym&>Q@DX)~rs4e@-nA6G@%Dpg}(qFC5S@wB%O@tmDT50*=q`)m~6AHyz#fo`_ zO)azkm=!Z*r|d@aRmIph92PJo_$W*={Y(~EnMD&5+}c=br4gpEsx`MLrd8vuO|I2- zOF~P}rWmVJ=QkE|;46wb5q>PgjGD+A4_L(FI!1BYQ63VZp{$@|WudJHRtJPPpV^Yx zYoJbGq0pvm#Wp%NTdFYrZvKULq2<}?lqvgG;CvAcntVh0Xlp>~KUx*JxU#I38o?@c z6~T%9(h}H`sun#k!Zz9gtQAYRhX;#S|^jyjL zhb0Urofq1tbWDS@WLWBylqh*-Q(Vv0?6xTq%*i$gYo9eo6$6iiOOT86!uP0$$-N$rcFhcbbgxp zl!glQU^&zZT$s-m)-}J8K%(_Vg*b}3OrKYI3k`uO9kTc6|Ddjy@m(3dAq&|MYt@9@ zQTRlW50!f#>T=s2l7OLAUJ6Y119qVBHFV3$dR8eAX60X{Zb&sE{nMljl|!iNA2w7v zT4iG+>QJyDJr#4tUa`CB?P8yGGmxv>+c+Z!vNJgh_@B0UL1Fuav^*^XVQ$k#sM8h$ z_Unrb^6U+oPI89W2x;p!Edh6dAp0a4gw(6D!dodAc%}sRYULnyBYIomwdv5L%TPfm zF;ztXy1(Zto%Rtwb$zG#z&^uGEHp6a9j`eGutmy zbtir2ocR>Op7gs8{~;j(sk8qm?SP5|dW2wuWdM zv1Dd_p&r~raOD`v=?F~J3$U4LQ-d3!ZH8IJveSE>aEbEyKQNAmpd+_mB*$ z8Bsu{mN`XTkuMf4fIa#>HTP-n5UnWh@-y~(PzO;`j_I#tr7?Sv3PvR?P_&BF0wod& zE&ck!L=$escp>qw^VHciGw&|dr|M=^V}ItRD^(l_Z7ItR*cUlSX01t8)67)Rs=BEx zZ>_euD3?)I{rCq=2P?*Eku3W*D)gYDi{SqXz2j2Dw3bVP3^`JBh4}hyRl6Vyr<^z2 zO3hRzvXV%7pgHLZCu&NTHsP2X^D%5f@fZ57{KGUF#F0{O>n65!NIMLUMC6qeKv4F3 zXh(6{T`er9?1>w2HiLF{pG#J$-(ZqgNGhEpAWvRDvjM*#R9Mb+_Z~E=OcZ{vNJ=ms z$X{u41eayv7aXqm;I)F=G2c&vdz2_40ag=EzKTTsu=Dp{=Y8&*9K6SFMP{@Csf034?f<7O5enD2f>JhJg?pw1a*VMMbHZ2DFbW6%F%wzX$k!sB(hNp_+H2pp+ zGv00x0g6>xCpmQKbu3w%A#9}$_YJh?zF=2Y zH1>eH2iT^l(QAG26a|o#f8xg3aycxs^5z38Csjbo)ZwpxaAs4PZ104KyQh3Y-K@5d zeLIUCt8f6l0cr4GXai0us}wy7uNITmu+s9Jy5@CGNI@3U?21gKM|q!ZiDiChmN>bn zv8EN&`|KrPa+&pK>K?+^%Dy6b959)OhSUJd%2hC<5MJqjxN zVhj3IW7hAJ+mUVCD;9H&FP0;T=d(^pL)fpqAt;gXVb{94uFiTj4aWcT*|Q@H0?FrA zSQOr>GgJCNl+%2?`?T#HiC?AKkZPwlYhDsW^AwMJ#BYzFebp=ds=dhmXx=vU{`mp{ zgZKIZoP2v8nd@ieX#a6=I-7l#Hj~L6?62HAP07tr*G+#ZyNlt1SH({^xJly#>O(+G zMn2niRf!OIRR|oRXY&~vYtp%ae2i)H{umZSbxL1D@0i;90yqMph-gz)sOUZoRta~7 z;#_i-<#Wk*si&%1YoomJ+bb&xZ2H(tYuXo=bgY_(g3jXbG7qWZCT9=GIjo#tl{>q& z#nv@W-Ykbp zms}{;VC8sUp`X!W)QeX8?vKx&RlUKZJ#H?i26m727LCwoss_My*DkxKcr|ZqSK~ps zt8vHl7$>%r63$WD|7hc_KdP~5Q=;41=*XqWs4gLjxuoLH)Yn1}kcGm8SF)8Nv8f5) zoK2bD-8DgH(?&A<=JpPxB9!j5`>g$`3UdfBmH@@rc}Zfe)t08O<@X>qs|<)e%t`v* zc}WGlnwo8(D7&ddWAt2Cu`ktrkFbJHB0}CtUpcY1Yjd}dO;#emG11M|BIa%u7<-9o zzj3A$wo#GBP`A}SWm0&mWjMvTl7Lz=G5xEI=xm5 zQ|_l2cl)6!^m>5J@487f)V%I{IN$HODy^MSEGORnnRFlJ{n15GC9O;odj9T@Ow&`o8wh2sM-N`^;7fV-?^Bt4nC6e3qaActlvz6nY`vf2s7%El(T z6`+&&L-`DGxn=UO>pI{qcI$0u6z7=G0p45nW_h)Pq&yB|#UtyirRRDoa?AEE&Gens19P z5UF(B9DSN{uw-XxWI1F!65S*YLeQx4isejPa2JDjG5h&(+&0!wve6h_Wy|BE;frD^ zu8iErg}FjpFoWSxXkPIb-y2|HJ#z840^mi4p)_XI&|oh)#9s8iGi8|yG7K7j(Kd7f zLLZoc2rd^+(g*_xt`y@(^MhEM5jCr1u)%F0Y~cYoD3u8XTfz|;v%9%g1GouH-A#av z3Zya>=CzF)Ip)RoFyKs)Yzikk73rM9B|Jn9MU^hmHRQj>rR0S+EeV3vh$8wre&q~4 zm|n9Ue5!;coY;k&?)Mk*Z&R#SfO|z2#GBBb^F&Uf-iOO2I zr@i8>(Mv^Y=0qbak!`G5MJ1iJDz;0tzfX+{##i1T6w7X{Y3E-xM=TL4dJ7e&4P=Pq zV$}4O#RoQFR!uu;FUrakH6{wKzjIX#ESX_pGeOBxXFNgNE$zP8S1rYp_mrzS4n_N+ zI`#*Gp6t264k8vjMUi9g#Rb%%ZYnmw*1qN+;&8U+C~hBhrch^9#MNx$6xpm!`PKJC z45IbhmDzkwjLG2~KPq}!dODSUA~$anleC)I6HAqTb#!vJces5rK6R#m=7j36kL#)O z=Ras{Eb!B1iXNZr9qvp9^+OvL^*d>pMYK(wK(C}fr0=>E;~QXrWd2%~wZaonKh)_s z^m~Y^xYigJy>Mefl$1Kj37a zs~psoE~G$8#Cdu&V50oW=2>~p9V=Ej2i4H1Nob|E=#f2cq8hb%m)MhOE!s0d!%_865u+fD$M9QE7@KPnE0$P>mb4)0 zCUH?#uhW$kn;4LC%w%)S5RC=*f z6UFeRj)fm7XDHuCQ_zZ}Vt%kH0E*SKU9&j~mHH6wMfOp3C^Yk0nHs&?!xTxocjN_v+5I8Fi*~8nQk0Ajj`8sdiI^&`mur+(gxoI6+0itM(w!U8K~j zwDx<5&SVITC7a8hp$!tF7rw?x|Gr4=qB)rSOZ(Hhj7YJn z^2F2{X%K&KA3`o%QDa5r+;DbMv{T(wCD78_qF3GPd&2p9XL2}s@rUQ%jd#xU&+-2F z#}|M2umAkt|6M&md=}LF>+<4vY}Rmjd1GU3@Lz+$U}JqX_?JT~tINY>&I~tJN9*gO zjg`S*xH1~8{nudmcWhJjXrj+h*xz52Zw4o`i@$v{TKi_;p0y(v z;q3gI7tg=rh8O*B+&7ca_AMik|L%)km43=_?b-Kl_x1X%^7WhL>+LC4yWf=!Za(Mb zj2+xw&?$pAF9+(Z$=>1MPm`U&`qF4=W$D@fdj6fegil6TP{aM^#R)48P~NDH`wMC5 zANcc69OBCI0SsRT; z8vw96rst07oW=EvgSV6Y{mIGTCAfA+ypt!B-EHCFAa!zk{Pti%gSmM(b)GGSc6Bm* z@(!xVi^Drgx1*nov7Bz0GGqa*EdOP=x-oplf7bp4mREq2T?=ZyqdSG3s#jNvUY(pU zk>4J;zWl%MgI3m8R-frVxeuBgZ11D+%`blvGyV7a>go`{uvW-^?~KQ*Ya46pn)bTV z*yH%s_@KV!_|@|A`iSl*@7bN~h`gvada}NZe4>Z%3F`e+sCks;!#$<>z~M^q#Npa<@JJZW9#^wk z%TrcYmR-A6M_~iAZU-MR1#n5~(;~O-OM|{Uu7hn8< z2X9L9@Zh!KYVcqU?z8>9H)055e9iJ~_pNOpO6)OPVfQ_JL^d|p-6IQuTJ@z6J8rR;6(q?CqQ!9Zn7=&rWuaj?Yeh9E=ao z_Fhj8Yb;c1#~HWabM@m`nCSn-Zd)vun)KV6^p@)s1KC%M!2434I|HuLmnzTWd^{ zQr?Zn)uH%UB|1dTdNRib%*;cnYwpkBDIumV>C+2j0EDYJaL zdQbB7$qcUpv7_?xRzV4(#96~o9PR%2^5p1v_h?(2IQF3Tt>r>W6jW1jU+06m$!_zf z{U3KGr#-oDxV$3rHS211w7RthHX`e)&Ae%~uG((w&vl(o>0w=!X4P~TsHb%J?y`Eo z137)?ho!*}uSV-3tC!|oVbUS-b|;b5*KX3S4mDYF#J{sUIUFRJ{q$h(Y!{M!9Z;~5 z?6Kf7IR5mNJ+z?K;qnlBY&gOm8;-=UqfUPP-^0~q6!?lQF<&+|SJy-H4kf`t9kUyLmgdWFG%}xu-4qb+DVt)bwqK4lPxoFQj`s^)uVBqlsLLMX>p<5S z@7FUCWzOAb76lr8Wuc5p=5>TsF{U@QgU8st_LP@|r+3YoSW`wJPK@@ldzg&M)^Jct@udwqX}+V|j9VrKBCoqZJS z!SU(%bSNdMA$5MxR3fVW@eWs>6lHE%{EAZYA9)js3UX5t(&k!6V_EZR`o1A)bgUs~ zj@b+diAXLdVX06A*|Fqh6TD<(qp)Hy1TbM z=05`-+O!^i93PC|?r=8r`<33gy+iDhy*J}`

QyC;Q8go|j|>xVnL`+B;r z$~f6Z%bwBkT7K$hc$wMw>B^IIdPeQ2J2^VtJDYfKJzZYjT7{a*4|_d6I2hwq*gHDh zg9R&rX^(Mt2jY^F0fgW4-nxf~)4vDIelKbIcW1^pUan*yak4rFQ4X}Ypy+J+OS zqs`$cDNes19|r!Z%0bj6PO~>YM4U47oPSErzWa$&hsMGiHTcu1!yyuUvt+>n%B`#&CXDs)j5??J>)_D^Zl_e+!D z;&v_K#<8o_PYKG_uZ~{lMvC z{{J7(P(Dgcn7*#!NZo9CI{ImMvOn(2maAJPO&nWpEpM&@j>yB+WJuaq&KxH-7Huz(7= zqti=P&?GN6pIo42H@z%cP2Gg>ZU-39qq z!V9x`Z>RM!k>WpGmfnm_G8%?nxxGUqs(WXV81c*=j)q!UVjinX#1$q*Q`TDWR__Qu z`t|-{|5ORgXxC6>Uie+dQ%IHg@g7gwDxS1LI$iXnmZzizH`(Eq|Mx#6UGYE7eBvGu zBi#&t-vr>c{NEL1!I0$OI{%kk;Ni;Z`hW6&pLnFxk9+=ia=%qYm#7xWyu+pCZ~p#5 zb)F_;Y&HIZntz;(PhQ}9-+ca^{b>jHC*#*|N${7qW$@hn`sr|dFnMukUNqZR%C2!o ziwj@w{jhh6Q?9tt;ycA5cin{Gq`25!R^I#kY(kEn{r9IGa*bY$Z!LQJ{5zd+hum3f z!uB-_Met4ANjt(PX$8XMR8< z?5Ax^wHGU#D*t|d^7iob1^T}I`+Rqg^EjXFU#!Bn+xvP4y;l7B{On}!SjU#>g5uBT z=w*|i>;YU^|9F0~y{CKi4^=29<9)lW`0M!&c}%-!G>;SHhUtm%q35ULos+#+dpnb# ztVLWB9r3PswZG@uRbS!`JttN5=)~S#{Pp~JTd(84&+$3Dp-Y~B=l-CbCvSJ&{B-p8 zOb5MR>YJaofBflq?`&sRcL%?Dn8QF54J)IyttJ}wDN{ya3jmqI!yDCkd>9^rM+GmAP@$A)9~Rct29O3UIMoFU-5mm@0SgdbUlPKC zPtU@FmN#*s#%F<{8Q=q@4H%oN37Qz3YH|C=*>xHjCwHs|4~gPz_vGl^>5CIB2xC^d zUp0N;gq@QTe>_*=Bc56_$h$q}KC|8aeZGHmdddW%NyT4u;t%DF(jNumtd`7D9H`&~ zVj{-bIXZwW&L%%OKypEJJpu)Kc4R$;;VQ~TUVAC>++Y! zJ8SCpiiI4h@gWYt$e2S+CHt|W(e{43Sv@{Oqkpl&9X59E=ks@a`!C&e=i_-n@ca2T z#G%f!e~m_X2mGoh@u7lEeZZrY@|qDB)hH^Dw!%eXGL|Y@q))0+8Z1Y@{WhOPXsL$( zlqCqa7=&MZNVtL8lmp1bOC(gpWYHLxl4NC_?kj&3VukMUIAnvTh8!~Ff`(+Fr+Z13 z6`0bg3oYy75H6_+!nLyg?FfVS#fM7K(#9b?xyd0be0e?kFGArWF!$vU(kh#v z*q+`|WlMQxDd9p1JvJK+Iiy0x(cP^r8BKtK?@xka1wPDxVkm|SK`|8dOX+aRhVru_ zQ1JN;P%v2%pb((ESub@%PxthV7EtiWE>Hwg9-6hX4F5(NS6Bx6=7b&r4%X*awt@6E_p&_Ez39!Y(*l3PK%CsLPb3q z3zb!-ULaMrxGEu2hSH>bcnP#Ha8szL@j0QQx7om@gbMd1gbIE0P@yuU$u+I?5Go_G z8Jk#nhfJAIFK^ONA)#SPwc2|V$Z(~U99HB?Rwt7g44Y!*>nlSHJQ)d+|L z3w_X+L`I`^THYX$*B8@YAdxgMNR^IIQ7WTNt@M<&(a_PytX{gp$h30^7dQ>IP)tXB zUQ$3+Cn{C-c|;iud33}h5gr?n$jZ_xoga|M%F341)dY{MD0c5-Ogy5@jV6(->!}&- zbiEwB6p`?T1d(vZLy1HVj2cFLh-9P&Ht=#VJwLzuR~jy_uV){h4lhH1AK(SD31vVutG*8S8gwN`@!JjA7{DCc~^TOLCZzC6UDp=MtF7Fbojxw0|XL?~9of!@OI3 zVqxM94=envkze)m{Xo;^}V#|ALt<|-&{_V6_n+&Dy+osqmicx#xG1PDFa!vh!O@W81qJm~Ifu+t0?ToN(_P4V!+g*6`d(S!rnXVJh7 z_do+riqY`;TD5#xH?m0M;cSt2skX;&m%(-#5hYX-kZ1}Io5{6XRqPY-1OlM}3$9AY z4q6n!;%R;D8mSIk1eZj(;9L$E61YudtUn!5Kr1mczF)B2N*WbApXOA|%&HV;w#|4Z zuD=duleieR1Yy>f>70lxq^5avG2Em#LTJb@*MJ5fNI)a0Rsd9=G=^|ljx{*d#TvoT zr&&YeSpg_QM<_-D+SualO>}J0KUw0SG5v{Si?_w-c*9DR&l%D<`AWrKMmeQ!=HoDV zH)^ZCDMKVg#yZ3914HI;4`9$G31Gw)1wuq7zK;tKm4ihOb(4qHW1fZulfS+sghdM~ z){>}@+c}F0n%f%{Ys)k%M#XC__aGBD6QQGOFtxgJ&~OFnGqjNnqzqe}>}zX`O$ZLM zUqf^VEfG4vOM^Dj=#a4=iIky?86O@zEeDSttkDA=!e5RxXiH9}$gkOCjiI6tS%4_K zdk=)rwJ}28%_z>x8xp$byXnPk4jV0+r`RtkP*nNFPBaw_R`ABaP?+Ci4OPMjVId}H zz=HXgAPojBLWNkWCny-QPEo>Z>mhf|tbaR(V07a{WmI3LRSp#r!cA0Y_GLlAx&D~Y zjEOO^v$%MFO`ZBQEE4YYDyrW`8IoE_Tu9bua4~E~d!ih~(s3Wte;9PH1tjpw$QNfO;?1`B1U7XyV$EhBVOBbWL z?G8|za$aP=LecN-09{$KfQX+jrsR@Tj79A1;Y(%{&33Hjdpi}S^MCu}IPCn~&WX0)XtQ@m!ITO6yVGGYj0!T$5d397;UXnhPi;$y_ z<%U?%)t*?990=)yp=_`R-JFmrP4>{U`;iA9(?uSG(d-!s=#?C8ha_#H-1tm8cL z&;(3^i#sE1MX?eth>o;JPnJedYY;rqyA$LgU`E`*yPDh~-60d4<!p^lg2j%B1w0wdjFS*Cgb6E!Hr2|XS}+2s97;$+0F?La{m723wG&f<|p0I0<_MBoUQ%fkAUKu#iqwLc|4D=tK_`?n6CD zfV~DHIF>*JM`J+nXb%rukh5Pn)un>y?rN}~QU&3`2{@>&rRo`hfgAd2AR9@VXz`xQ z3wHNw4htHu|sZ z3CT+mBIC`Bgye`MO1q?KpmZyrv>;GEnav%#C%u)FJXN}T?N^O7P2d$B@r)E68|j{_ zOUtw-&^_7ODyMtO8{!j=B?WSSQ}^726?3|$a%Hl*C+$zTIUb>Vil1vF^{`epHrG;+ zAz91TI;H?~z^;I4yyOx|J8}Y1NO^d zhx+%k>yJDyNio{$nz8Y8(mcg!q9F$sC6#2UMdRPhAEh-91&dNc?gf@jwhTysWrxlD(lFV+zGJ(% zBup~yvQBA44+YdhuY_RXK;Q`BBa=`mk7q>@54tC_#G)vhY;1YL5wkFmO|T$pa#(4T zLSQxl6+Y;pP~m0wM2b8B4O%&!*ll3&e`fl3FfSRZ(-LVH4>fTR#8aC;Id^xG4Y ziaIBdL2|VwNX7Ud!U?!-;zUulhX93kB!H4pS9nLK08)$Ziq@xIjFyiSyBXKTK<`03!3d_Vo4ETsr<8JnF3e(b}K#+j`(aLSY}o!aO(e2wveoe zfiP%UwQ__by%EoU(&#Ni+)fvg)BhH}SY1-@1Ud#3!l49Z#4G;IEt294rbLc2L_0Yv zmv2sR21g&t8C`Y@700X^ovUHzSF(F`QIx4?vNS8%!4zy$eB*Aes^P_@>{e5WXoLN` z^Izv5KBvXhuVEDRG@+!=~3M&iL@i--LW5L=pCRs_*2Ql`RY z^{_`VKW;Gg>k$BE4_*22`{NV^hEaWqY zb5(hEG6#B6h{nZpfS~!x#r<>~!X&5-W5;v#?@mLRrg(-jLsT!wq{EsV66UN%BvL}K z7m1`&7f;xqRU1;LWy<9I<21YOuEh>&Tg=ZmW>B#vF&UylaWfAUwp=8VED%fs7j#P^ zSWI6>xR8e6<6@QHIh#njahFj+l2~g0T0hJ;@vq=q$@lqP>=^-$m z7$wv~>61_k1;|N^abX0pNay5&=t!#;N~!?;8Q8HKATi-U(P1BC=0M?X%{Yyv)Ms+w zr4`zIPA#9|TtWv0ArIC;?`y`6Mylu)Q^=dE#q~8Kl#nUPT#a)CdT!ZGKlwbdOf?kdB3>JWfc98 z^N`Z{k6cs5mxUt5`ppaCWebbYZxOg60m22*kq(if!YiRjp)m=OA~Em7OCp6aZAuhr zGdZt@&e@DkNfer%kSL7wgCz=2tPxasW80vT3gr!U*_1l#*4X)Uea{e1rzhPy!b=pP z=R>&YyZr1k=5&Vq%z{M^$%Xu&+@}RgXbAT~%X-R1M*Qz<#$md1 zG5`7J$iZHzt1Ui|e0iB_TbP;wT8iXm#KdD6X_ER$0nIA^;?(pGXsF>L*XO0&lb_S4&JI?`EH_a!E!H zbB^V4!O%xjzeGY473RH?@E1>0lff~x=SpXw8<^mt1SWKAsIUrgo@WdSc!a*nZ{zH!$TSle0ME9B2T)$w5MZieU$Q+4t8hPSD3H48Y27iUmEe*QcG>BtDPE% z200Kb@^6K5#d_CMS*w6dY!OwZcWcs#G*mb}I?}4Mt}boRnn19ujB-nCR@T9kk1|1b zv@u9Zm(fVabMnPmS2 zdMRtTzWlRNML5<=>~NhgcKG@GiXFY6aj?0{tT{dnmOr6Odt0y7W@< zcs1ItmGy5o`VZ4f<^FCWqP6HD&7}ps{ATNG^47%Mu)_wuto6-QOWnU~_(Cd+r4<&{ z=*QZF9Hco|NTCfm!UnEZPHjDZP@SJU8mM6W6D%Qu36)dRd_C<{gZ6M#@cbGDX;d&+ z^(7%qfcG3t$kJ+}BFylKh(h1m8x?v|jEa|cS6co3l?Eqd#@XWhAzP6tHzi3GRh7(I zEhu5Is#1344<|Q-65ouqE#m@ENCq}g!LbA?B4tusA)UZOg%NVPiVWS{#TCL-2UoP9 zg1fV*;N?wJ45cw<kIvYcJx$ejxYd$hg4&i~(8CF=xH@SOSY$~zEmu|k%ArO3tz^+cV-nsDI`E-rQNh0Zp=B*qwrqQ`xXmab z-!E=HMZD5W;N-@*R8E>Yh+-6>tf4DCQA0P;v7V%1mNf(ow5+0)D9lz%NZ`n z$r(;{X&<_~L(T-3gmQ+aWW`KM$}qp`A&)dtUejC1tjkjq3z;{I+nZ?y3T^8z+u|za ztZrx?O5~5`dkKIwWzm*DoI)z^r3Z&$a~~L$pKKrkASMtYJ2!%aBKkfiWVhr%!Hc>; z!F#(v!N}K_grJ~jvY-%)G*-2!H?pwM$M)4dHlz>6uo#?9uReUrdC|Y0-Q2}nIy^$4 z&~t%6ffw2ZN_Mq_Eb$0nj}s`X&E0W_nL08!;u&icRoP`(OS-BX_M zw3Ivnvw}^l?_7zJHH!LZ9~%N*443^`#(rUW_w_8hh`V(`oR$HkwT6V+P%?+D*eicH z8j-=}(};Es4CRVY)1VRFo}v-3>W?v!McWJeG-47uZP?P1E*1&ohHGX_89XIER0d~- zt6X_h&9VlI3>BBG0T4+SF=B5X*;Gv*OqIwSD>mHO z4jC@E4`enIg}He|0x@!J$AC%VMP{atA7A815zpcUv?oj$jy@DGGO=pL>4BF`+a;`l zm(wpZwzy4G%lCJ9-{W=IPm304q600ZF7o@M#mKk^B;`aHz>IwhS)A zFGVg(HFNRLqVv;&fb)q(-BSPWM=t!$nT3b>KD3nUs`l~B(R+?b5vYbv1Szd2T4&kL)s-b%l z&_PtDpu>YYMN=5!tMHmnYz8{KE{4wc(+{7s3#-~TK5W%*1&`!dOJR+~NvYAUDB->Q z;nZ7608hQOG@j6F!WZ?_Zn@GICUTq5NME_-XU;_CRF`rq=I`*E=mpg!p@}T5A*bDn zlA7ABEVY_CV&%3#Vvj{LzydOlVKI0)yZmy)!fjrXP;V2!=#W6&j1mzFyp;_GzFHm0tFRvF@zfCW4 zHqq3%}d>1%6GUg&QLY7I6s|L`Pb+73OoOsw9duTrRK390%>5u+WG_~cvqI0vN1Qjn_pxNmt(f&rZ!oQ z9P$A^8G~solqYH>p+ta|E|-57(Y*i-@_Rmz8F+Gp2DrjU0~h3&fm2;*;G62ve@ZmK zmQ6IsD1Q=W7%9g+hQr{P4as;x!diK`xQe0Bp?h|u6M^eM^W?+Yb%{!{#I_1giYO*h z9k?Jm(yC0XElCgs`ow6IS=^vipgy{?s%PthON0Q(xsO)CYagmrAQx){Rqw1Vm3)oW z1B{dF*%XA3uG=iKI{DWZt#+_;hvS3E3!W60Slv))wgEd9WnE6MvP|^y0p8kfTNza; z1Yjou^E+>LLQq6sIh*`caAUoH4R_Z+2-S^W0Jj)|L*?ltJhnsSzQqu%^S6CP zR;ft-Z{J)l{6ZBT0!@}T1iziROb4bgsEfP8%J<&;h}}gK;)^5%QXp0SEKORkzT97W z4fEowDvH*WU#vZi1z40Xt+f}S2vh98r$$3p`$LR&D>n3nn9b_6O2d_1LQINymlO*w z33VEVCa2$OK|)iE$-`5VP^92i>jQgu-d2_sUXKZ-WuwJ25)R^A$0~m?S*_f}3dzc{swXtS^27Ave0IaL5}GhskM}J3 z-k}NW!+f|f=IwBi^Dqe)X7!Vzg@W%iCe(!2bDD4hE{tT;yY^rwto2i4Pd&alg%-M~ zQws@=sXk~~T?fuFS`I#ZyqjNV65J9|o=V)b`BU<{e~TxhQ7(lHyR*jJ&J>Fmb99tz zVoLViklp_sj0G9+bc)Bkt zq@N0R{7a#I(?HO5bI{=CE@%Mxdfd~{fT-$ALRG}sQdKcUKD@4(r_XNcBC@G!ITB$* zMAtKUdSx?JT63^XVSwC9%$6D`A)#NdZ$HuMMC``Oq}Khn>5%Bz_76 z%y1)yBsfIoi$A`QhndhGiHHU=E5&(c z7l|;CHAAFH1O(KVL?pssdICngQL~6L#G5Rl!e90HM=U~X`m)GqluB4@*$<1QpR-2p zbOnpNl`Efca^6=OXFc&tox@6lmp`19OJ{hX5Y{44B;*R+k-|g+65z=fD;m=bA~^K~ zhyYhPM1+9gbsip4aFAak3kTlQ9}WUmtZ*DGuFu(JCRZ=9xe<7Uhn6s}5~R|ApuHKI z7&h1K)n=msLE96Dh~Pj&(*TPJM9`p!B&75DkkE)d91;NoxJJg~FU)0}9*m2}5dkEU z3=yV*Jc1!sQ#$DG-TVArs}I{orX2P-29wAaD>*?+Qp>*!FId12P9l3jf8Nmm16`Q_ z16M|103JRTG*l1ALNL(x#sYVHSfHsjJ#eZiEhJr^gdWzl&@85h!Q1Qi+45y?@qXRL zt9nbX*S+0eNa=QehBUcg0k{@rCuxR`idZJnF1a8&((3*iE&-50-CRcnP5QqSsPe%j zbrW)Mg3VVi1%3awA5{@I0h~(p3)D-Ypy~ahfhl#9S9kg|!rC;|OY`2Z(YkVSVyI9J z;ChxzgvUI&igU^;Nzz(yLI5tv?6-uwO`w<@8%Ayw$TX#SU2Xf6kjlgSvI5EeYFFDke-0!433(8XPt z;F5a4T^sA)&guv*2@Mfru(UHn%-cvHl5O%NfJiO`fA?vAn{#~v2KJ1&KOSU8CYho- z(KJIw3N)19aB7B%z4%12%oPzLnA;6V0Fwk!7|Vp1&;$&Kg6_{T1+RVrR50{0P)hfw zgivN>ujsG|3-wnP7OtmymA4yhYQc2O6BKVhKfn7IuSi8un#M?>Y~(i0?TDbzFM-Cv zz=shOakyXTU+HE^gwa(I;Y8{M7eq%|6^>D}#_k$RtCAXr1f)+p(m43Crp6)rDW`E% z!Sj@Zi?8mlap>Z%JP!G6Y7q6&I5t&PBSy>a;zACKG+Iuk*?4`6b!N*C zcGdp0xQgY*gFPCIaiCu?$J_ObYFEtIqG@6)(k8edI?|zE$bt_%8YeRxT` zs2ugaHlYxd|J(Ox6zKz%pbxYO#yj}!vP8ju%AZj{rZt;K(WM$e)l2jKjP*?^b}@HQ zIOJmXl}9C1(d|Z&%u1GIqi5ODFjBO0%prP=n1iqLm_un;p(IH$2cMTvFTkfiv{Em) z*=G)}c!Yj|SgMr=*U0*}vrcJPd?;<2(30ARkztNt+#}}DxAu&_Z!9ZUB8EeIX2uT} z)ARY|JiDKEK6o}%p_`;c8MY*eu1jH$fU&H>)BM!bQz37oRvdTCy|Y^+PAX~0E0d1RNa{0?xT zrvqh#DbfxXE=Z!KJYGA~9tV~oXknBBWhA#LjEFhL3!U4umJ8SF)Qjp8T*6Z7x@tB29520dG+uM9fUiBna=%6Ph zmX#%$;~`Xr8NZmq1HG`KU``GzR-q^tiA!?}PkSgo*dH5nuVk5 z3pO%amRO__3us%)`Va>!w^PS4?ic7<@@fRBb43=C2O3-m++3jocZ?!rO{1#Zhh zfm2;j(A`zIH4TN}lHSobI-rRJgKcK{zCN_CDYGHhQw)S3=I6N#bIP=5!}P8B-stXYsqYgySDfJ~GBDgze zzK8~zkOB66VHdShZ!c{4F4s51W zfc|e^@L2Va9Gfr=5^MsvQi6w@$N^6L6tt9t;<{{hn}1Dz!gMf6MO@@%{7{HU)@$W_hKy6Dst6 zz%8-^+QpHqza&_y>QYaI7B*=IfB{;D8!MUR4is2wB4#v9mvCRP<2jZw-+q>5o}6Zs zP{ZpCiD>!K-5^vd57YPY;QtK2fx8p-WXm36jh_-}}wZOk6h z5p#wQI~2v4XC3NoM4IkaKYfK(f-6TJd{uv$qrU8l>G8n(QO9a(ceBBdi@U3jJSw4}oXs!O(k7$Lb^yeB z;64D!=0QI-LSj+F?*u|P8UuueG__cOlmmz`_9OsNUv>e6m$d>UUo7)*_A9u!{tO4F zie(Y~i&Z#EVGVoe8 zDW%hj%2Jj;oT!o)!jo0a9l=G~4QCARP6#Wm^zout5ZvMWVods{i`)UWE?*39twoWO zsP%6rnp&h4cju&)7Qr{gRk#;g0yR`3x-bt0-C!LH8%c zO9_+&9vYep2Rt-5diUc%2`un!2`q3l!U9KpDDd<5Rb2Ff`ztPmAmW>ADtQqQO9TXc zs2Hvf5Y|%u6;*hjPjku9C)0~B@nY|a9$I6A_Yfu}CGhd(p9MPvaB!dM!GQw}YT(%a zG&u0tIcDH%n#3TI$z?RUKI#Pr3c$f4F)s{GX0uG9{7E(-NT0`LFFUPaS$?58%y+db zu+_$8v2|3Ps5vH5VIh|2h?gp_4&8lw_S4Z@?I$7Kb+n|Fe*+!~t zUA@)UO|-44|Jzq)i=w}kqcqT7t+V=u1q;-)=F5jGeXEz|Ga6S`6&H!&vdaqpuXJBZ zn_Zk|BR#FmQ3{NBl=-a4PAS=fJIX&Dad4?8N8F_Ne~&o$t{iay=_YY-{gV*KW@-ns zSIR&8IK4@O;rqq)$E=!}T|J>-MpCC}scSiGR@asX#DjyDia7PHlpX-P{Id%9@#=&i zX*xu-67=a53sF&}HKcV+nowU7dPJB99w4~TTRp)yHW48|Vipp#rLSs%X3vviayb4Z z-Qr6I5@*dtSCwXwS2MF^$68X3EUpw1gk0GkBy4Mjpwy`_(-RZ)y3hXD9rA7Xn9!(Z zykwl}vO9G5Q(*$I)|gP}yNL7iGQmX~>MOdZ?P+4aXvGP4o|7BNmL z>S$FhhS~)(DN$OpNJIu9e@Fm%{4s3G6h+k<1j2I?LWL{+QHDU`VwM8Y!wG~z?jjJ4 zTE#J;P%(FII-yzV5={b85w{uwk-h3Mp*fT*UW3GIf4!2tFILp0?914sU?!bD)pV1ZMfg2As^n9w6lD1eae? zrI9DIt54*HWSm&-bxm7VJkYKrLzKsB8?&SY&Xj-t$NaFc8$Hql0|%c549ahbY2kQE zApbU3UTWuhI{z4H78NtJRiev@i##^bpT$MfD=g%t z1RwDqTG*NV^yEMgrk=o^IT|Tc6w|_Be@4x@YhID?P>nPBrkQYSw@VYY2y6~89@-rG zR=>8G0#r#hA^R56T-w6xq7xGil*@3ixZ(JV|8$uCQmjW7^R8c`@HMct|-f>Q?{F6SRUO=p<|!Pj`oBPMwqV3;idVCd!$F#0ZE z@ci4VXs;@07T_i0>)PbH7%c#(i58`dhRDG8Ya_EKo!L7ixk0J$GpR^>^G zWzOK;%!=RgumriB-R1HIq4hZG+Tjz8VM1dnz3*%4ghN}mA~NC_1ZF32e)oa$0Yba#hB5?m76 zw=^ZEkO(g3_VAN-#JEpk)~qVP7*ocp>*>{p%sRS*>BDNC4znqJa%JbxAZ00pg9Suw zU7F!Q;iM1{Jgor;dMlx$^6VH8k&3DT%c!W_+=U1(=|V(sNqRkXTcQHiU$V$oPhfq9-9Hl7LuzVvwyiM0SiGY zG-owQd~?=jUpo7j7k6QTQLP8ufw%tc01#XfVuBasFrlT7O-u;oSxhj&`eQ;~gP1GG z^80$VxXDP6u}T^?szr5JD4pIheWo9fXOe}=r)Gg~qexdyDks&68f1~K$)lnptyV2N zUDKt3_PM^9$@Y`w56D)#gM%@tP#&2Kmo&icoX7BJd;wV%%b$58gyP}gqxI0H7No_-cs4Ro#uLX(nMD38~Zy&Qhdlz;vw?)v-3UDs1JDv#&W-+s#^ z>Ld!9cEiA;4@n>l+mz0x9iqh!o^ml%?zJ&Q6?s(}R>Km3wG-NfC^Y5+m?39Ercxg6YXAncRNv7vbvBVzaN>O8y2Pn&j{cZdqBNjS)_9m$9i4nt+n zmp`0-3aC8&R9@_d!h!bq^Bp)SwZYd<>Bbx$IMs!RV6fA8&|{*l%3S%zp!9RRGTJQX#ccY zPDEs(dC=F8-B3Bi7#q8HH=lDWoVV|8KV*v|PB`vxN$5dZg42Uh4eV*olDe8!ckV4o#G1Co!g#uMx%~TNZ>wM_R4h)g^8I z9#}2IjjSeWI;00Q*)6%!fs9j56)jR_#)_s#`NOFm;y#Z|v~^cRCNM;U zOwtmDqY;tuB)?NsBHrnW#mc^}vDAY~)>oJL&X`J$P)Vra`2461`8)UeIq3zO3lwndcMVd@BWH~Z{C$pLfExRX~=xckbs~a0C2Oi@B?IXLt z;1A>s#)?S=f~=>5p-N8v@^}*xoC#$M2RzZjuMiC!Xn=uZ2`yE(hG5``j|G0_u)wJ< zEa>iPoYPPUE(r}6PUVomsU{Hgs3%Ae-Pq8|mly|w?b*dGNuszOW!*3 z%JqG|ddly1k&@F{xbaDK(^0e32BIJpcnOhZqNGHvFFELH|EDiVIAwD;B}c4-o^qke zeWl3eN=k}QboA!ae4Yl$!SrT&n}G?lGmg%6I@4ptgMnxVNnYWRVlTDkL^@D?(Iwm zFg&+GD=!zfxkXJ!A8sj_p8?E+_gA<#wUiVRyWGm-S7GB|HwNR?wnKyJdsUq2g$@t^@>DTF1hFG+dSXM!8 zbDL!}5r{Vu8m0QtT&n`6s}5$jbn99#SCDV{H>e`K_WzhFQji7TT#xe$>A+8*iU0=W zQp_1|=2xGm=b4oWZ7E`{evfj{yQg^x270~x)44o2;9-HaNVntnLONj%Rm# zVX+(iTXp*6$*iob=f7{R&`z7ikL^Y%E{l1MS-><*lQrxILk-%1G`LMI25C6mPNfEa zb4>`YCZtClEX5g?)WGgesWDiZPFdztv&QIbe)BJw5O?Gri`FjZx(eKGkm1{tf2cM5^%7WE6eb0w$(KkyM)2qC_oqL7`0a;B;m@!}LS1{5c|z?zT=4Ci z$G<&)_rs$N+z{Eq2WDXJ;@(><`RV-Xuce+$*MYlE-{`-{LkqYR6Aio&U_ z-sz!o0TUR93?`W8@aH8~*znd-cu}vtZkx4jTq6#Ph&pyK|0IYabw z)6zoBEij|7-+kgr7YR!XPIZAl7|3s&`IyPe~FILxH zUaM8vw8Q3fKm=c`&^o}Zga|$kk=n`u?FYdKGM@SMr*v{UaS~fOnx>(GH_M?CmUD2h zvZ+weeA_|}gKWh1&bb7Gi^c#g;Q5SDUaaTm{hjI!Shg`bXkvjB#O@_h@EHoEaO$#& zl*Yr_7o8mC%XFL`I#8)XB`IZm=o#f;VY-J$ESuCj8!T|^46xv8mQca&PG5>c+{2X0 zv9+iRBW@^;Y8xM@6gL{@jtb%@H(rrzMIcV&P%Nbw#%M~n;VVhRWel>4P097 zNz4d-RAyv)!2D@pM#f{Oe5I~mM$F=Q3`4@6c6`RHp5-i<9v-Z)w414nZ!GIq=LLbC{+i0Ox0o!0yBIpesEE6p5J6`MA<{`EY;Y50aeo^mQL+OEs69m? za&my~7i_{wpoTk3xtwB*G=eTSR0#cJPlfOz!`xZCkHsQv(XOd8c(WEmeMHFAS~cv& z)%@l!SQiH?&sO)kG=Co_hV^{R4i0m*mBEP2am;zwaiex>tHY)ff_blq_u9}$?l7l% zqazF+!R6CFh793?BPow?FxB&9@FZPC!Y*mtl=?B-?V2>`Dm8~F9`W@EY1yEWKzr~0 z=jm&%)vag*j~lSqr!K^lRXjC~yjgtp>g1lyvm*QrDvD`}CHsOtbAWNgk5&SMMU{np z$#H>iC}N~3w2re3T<~1!;2xnpQd|^ZfuFiy!Lyq_w_$-7Y!8vJpzQQ?D#!71TvT+2 zT5eFlz|-^w#*sNg^X+2o&9(b&wf>S#-QBBFMHiNpDg%}#_w3uapiC~>Sy&gs(y%U5 z+&V=VtqZy#w=U!#&$@&aiiTMiw1j0{@HCy)h4QS0P#^PhWYxnQz5jDPzqndDA(2;D zm(KBP@TKZGH63I4_-2I+!L~#SzlyXx6{BVkgcQc#1X51C`7{hFX{_KmbF8qrJ*D&>2EF4c5L~j$)C7pfs#CV9 z5r;@qgL`|HhQp_ZrJ>#OLLz+EzK+b1<;Xn#v|ilXfUqjtH4>x{U2m#DXhc-(RR9o~&1&Z=5R2SocRyx@&+@8iXs&_XqYJXdJN!lY|BfFbQbX zubWJzF<1A%Jq&1s=&RJG6-S6t#=!kN#=uWq$iUNeAw$MTBs73%4;pl?3}{GYu?sZ# zF>50-da_!>rFGV<{IK%sVR~cv9hw8JCo~7PrRE6D`|ES`eyudnUiBaKb`$8paSCE8 z&Vd8eo?_%9I1P_~cofwr`Xc`MsO1`b*zLx~<&Nl0wFR%{1U>Lm_?WHeDW2I-8n7~! z(%`{+k9o|f$PJ9#-OaB+6t;2U(qVg3rNF2_2--^`1c(r-v@t~5MC}_Q#|KRH6smOU zWtP$*)<|i1E`)&(?6iLB*%8MPk~ogCsK{@r-NwyIUKgMoOQoN8La$f;t5Uf+YJ#r;~@ zDL33^mlgXJ7@#+3WL0UN>L%ntmy3!?ItSN6AmiI z`C?Nop9e57qNY#*1Z1cHb}U~4Gfh`FJ?C6OF zo+{m5UK)^yYg%IS`$B;?Xa;R0LExu^Ah6XE1U7Og@KRoo48Qe8!U>~58VPUb>&4BT z4TRD-oog=`S2l6Lpt8dZp@y@B0zX}jbI<`mX&JDJ!~*Wp6ASn#6&7fnQZVqd{jSym z_w&GjpAs-gYhj|HNDM>ZaLN%O4TsTNRJ6SR>?(+S{6JgB{ zE!8NciPlzoiVn*(cX3g#%yU>m?bC#H2wy%@t~`qsR3o`t5fPx?a9FT+TjeRUnfo{_ zrxdFhY~C!8GqmnaBXMkQ@E{#%p$wLgg7NPfr2}+@qjKW;DWM>=f(m3njuiSco@+~A zE0Dj16pD)6wZ$3RB878VS~JuGDaTVQ!Ts#~npJ3=h=`S_+JxWzaT!pENeFR&%LmcO z;FtL4f2BuQAcQm={zyxiVQ1UtN`&IKzS86Ph>2JkX?$EvkBLfsuBw zY}v}VNrpI6$rxo)&@gIG(a3-!@M#I&7$17^4K!maW@us>rIhkSaoSw%Wi_9$;0e5) z*xtE-)nM8N$s>&LyVG3mq8!BE8 zHe8?0RVU#C%R;ET=vchpP}o>H_UCU-CNt;d*(Bo){dNKduoZy8b{Q~u;GQx^fny-Y zWcCO*Hu!}dtTZeR%u#~lJdl(y5}GH&2=;DEEjV>3L|_dyvpm1W*ze-%$_btoGmG4y z3B|S9wTulbwGnk|mFJFxWUw}BvKcTFLLnGsFJ_nvkxqp4`WG-?u>(@rVE zPkV@_0vuk{1g+vmO>kN+WSLCTcD_=wXo3Dp)|%7*Zgu62@xDJ_W$L8yXo}0-9itUZ zOkm8^1x2WZ6%n$@$Os+RqM~AQ7~VI51%vh(E^w5Ai-4)*s|PSDO63$8tXu2_)ByB7 zKm$1Tpuw9b*ubN;;Z-i`mSdyhbTA(r!_a}Q(Ts66|GaRl%E!eG3UBKAAPqizkC%OF zgTu!Lt1e1t;Mkh30~|IP;t4ML(*+wzqe6lL5A0zAKkWe%ynz#O2aS7>plZw9-r@uZ z+@~*F%uXrRH6BJEmp526$O(ZwLzzpjBC6xhbhy63rDG_4qb{&5bq3*|hRmA851kQ$ zS69YpH=%)0YUr9I(TtIPOKT{A`Hn9Pm!~$0^`iHiD8a5=lJaU!LLx8WO?|*` z`wfoZfSutrnroqs(0>DQm$b78`N`~$@(Mvv5Xy6g8)>-{P60oEJ&DxF3;zmkp&%!!t0vxPwzk7Zhi{aHR{EWt=So)x69k}CA=ob!v>$7 z+wk=wF&+9y;}@Wj`Utob`iLC&AON1`aXSVC48zCW8pGI?W0(PH55!@%h-drIDgxEJ zm+lR09jBi+pN=2t9nV-QLsK2+_F=+?Klof&9;sBygq2^-PHrX$w;}y#@WxE$jT@W1 z$EFv8)gxI;^GD}7vw!F$!~TK2(mrNP2D?%Nh?{u^5SQ$7vT(81y46qFZr3?XxQgC| zRain^8(7pefM`p@0+vi5ZiFZ3SxN2G@VZBjFXq=EAS=V|gFkP0OAetMnc6^XOAZMQ zmrmq)I1b@tI$;H8UNN;9KFJheu9wAZUXMkJX$fcsgGB&a%ge%Z6E#I!C}TH$SmcDG z7MevyuVGa#Z+sTXf_NRCR)@{utEG5ByGXofo8vCuhy(X3nlQkK?htrkjTFP?FcVFq z1x=gV91b3NXbE-1huIvom4y~OO($CF$m9^4bIJmD8ZEC@m;O2>Z%PXT`w=}(9Wm_< z=L)MHjwWWpxtwvMwl|#An7CWKsxL+V;jsq}|X58eW&F=o~i$_e$8S`)MJNrV$X%#J4U zIV6%-Cs4zPG(D0-aF>ijKm`VepvzJd0q$VLH|3B5h2W>}OCbbQEnXjCkO;SHhl)4^ zkW&aF$Zf|f3gNr{FbK{tfrUm> zxKeb%P8_hDTlY&4d_}UElVa*RUxv{~$1Xjj;UIQ#+D?l4F$9(M}!$_0LoKcLC=Ff=xI@FSl5A5dMXnH}bNX*~>5m9S& zh#p452fbH%IXa*=6p%UgiWbapWNN(V4Tn?*{vlSTCqsE~VJUMLF2N1#%P@W5LRW1d zTu#wDTA7Sq&TkgqY_L4NTz}sZeRxla7VHpc!2>^tE$TadfL_!%p~%T#Lc`{aI^ri6 zB<%NmDhKv%%PIW&hEa+cU_1MUQAZmi?^YMU&t9YSp%s!W25gs6;!aVS(2pSMW@IY~;@$hnW@8`u?#ge+=KyhQzL?Nm= zW*sMqhjtooXk%txOF*DgdjcY-2#Vhn5kc|Wh`>)>h+q#jX^?{vffq{VQU@x;A<}RJMdAd?Xh3tz!Axrx7BJmts9s~c zlZh!ke=MxLSqOl5gS(v}cfMm9FT-|OeB$*@h~)rN!7&`5_9XFk{HQ`8)p3^NH7hGy9>T9|0x5>B7c>XdUL zGq`-fLg1IT3#Wow2+%8=vrBLl4=s2Fja){H8z^gjaPfjBR|qOH3u>Q`3DSk*gLh|n zwj74m+%lfo!PUdC-(whj!ND|G41>loFoQEJ_ZIBY8C&I0t830e4_%YQXFrXxgC0w!3kSHPakQfkX z6OMzX$~Cm7tz3ch@2gz#(1~&dIkw7`bY@5t!5BOEGE=gk2E_~Th�FFMm@kx>r~h z2`%pk*CV<>5*A$NF@*)c0|^V}clA`RVN#8R1#7PMgavx73l`|SF6oM^w1-IP3WR#F zuxA5u(TA246f7b=^$B@Qt@>Io7wa2ltNsf&F-q^fTHWd~-i9uU;V}9lCSg>Kx05r* zly#^iH0YM6&|t=`ga)Z8gNDr*A#Ng}!8wRMp@Apqf(9jGhgTyGk_w(y(Hx^c~Bh5@<0Aq&E-xDx^ zSPhI{=T{etTin5*jJJ!6wYy?P0UFU_Xk?&S981E_S+WIQIspaFl#syB86vRHKm@u= z0fHtfiGHv@ohT`;xCb-{z(_RUCl3v{cMb+LlM4mDXy4Gskrgu^opH6rwT*?>Ze4IM z^3jnegvS;B3AQ`@6UPkUM1?Bli{JpYr|6yFaN(K&&4y!wo=9QEkOxkzA)9(2yqYuM zhMz)*Sq=$)GaM4^wHy*2+?C9ZV_M|&@nMe`bkuC|ezE>*?ybF8tViLFsMdinX@)15CDH`zjZ zjqofOk-{jfkBcc|H4+yT-3czj8b#8U(8uZt45LH|)8Km|16|dH41T6fx7)}-_qT^g zWZ*>-WZ+sYGFW*lws9dE}!Z9p?4s3hGf!^zahv<779(Y3! z9zd!G559H|4;jT$!~yfr8yzQ1W6RL7Sf4MPspP+erLwjWpLh`y7wAO?6->O7pujZ} zOrXo^rj97E0D*l5Akd{65E2hK&4UAe>VgB$?qCZ%L3@Z~3!Lr20cYiCAhMd=E(Hc$ zt3Mdb#N#K6b7zVR3{3C1c!%gjR|?UI?@5ZzU@FssHB6@Oi6yRW^&!ZfJ&-6aNhT|GrU1# zilm*vo4a;~z{zT)F(TM@{`QszqxSeBGEB_Z7kA#=xL0!&J5tuBQKZE%xd#d#6ZAV2 zB;rRIfPns&fIu7f1O$#uKX^E>Xo$ZH9`JA;9`KYo9PnZ60}aPh8bghSSBsk)EQWWB zdl)Q%ibh$%yaotK$1tEU!U)>pV(E;N{J6MU>XqI%a$>vMR(JxTEwC-M1wIC;Eh3fZKp75H zW@k5Tadf~n6@<1p@|GLnQAvgHsfBS7pt3ppFM*|r543cgA5K@rKO7Pl_;w9 zf>AXV-mJbZ*57O>u#VJr<(na%O)TJ594zEPa|wo;os&5#=!30=D9-)6s1H1f2ZjwJ z;GL81$5)nY)uEIa8waq01&iB=y}x<<+w*s*jreHu*P~7UpM)Gd4TjOj)yfHmzV<^J z-r#FzGBN2^m=fT#!<66%8(8s&R@N%RfreIlil$_801Qh@f~E0*Nz0b#B(Y#iaDJ{p zxCE?gOKObUwj>nn$fwETwYN1UgBSA#X5bEL%>2AWQIA`jKuY;u#i5nR6Ix~ZwglZP zZ3zzjk8ob_;H~ZW?zRN2vLE`e61+ypA4h!`cjevJ^mBCrV22#-2#(@?1cP9^%pjg+ z!imZhJ&(Ul52la|M*1iWQPaWBXFBAR#$v9QY|PzZFHtvk6tY z^O9#oY~uhoY}MKtmgQ}0>57WF%bWA%4c0TmV60hG_ub+GR1$vjit2`7dYP?u)kM@vQ!718pjQ=nL=@1^kAK%b4%#rWOB^ndfao7_$A_%|0r;hO zV2?Ukg=DkG|MTMI2k0dr=BJ-uKxImohMXq%BuCVpPrx;YNIc*t4-e@191nz6Ln;am ze9@j+h2zR{az^j(udbGVdGq9~q_H?kH+Z-X{U|ViA zRR_#|69Q)Jtqsn&R7ypJ3%Q8@$^b0-*Fy_OMqISuHv=uWt%Vls?nDdcjSNJKRn7k; zW-hF5C|WG#!72P}KfBByyrFe8yO~i-)z8tqFd-xUU7H$CX-jMkFDh*fe(xzhfWx$d zDL&B3sjV8E+yiBDOb!W` zbo`)Ad83B>1!tfc#CGin;Q=iU!JBq+$lz*p;}dH&o3|LHaRzXnU)&>8z)e9ch2xnw zX?8O0q(dPUhdLyOaEvD5kihgJrQPBXoMSjJI8t#4g9s^yU~>l!2{4Z#cbh}-dYV?^ z`VP=(o?xYHG88M$Xf2515Zox(vK}7H_=K~bGa2#(sK5;jf7TkI>0pq~P%ORz!6BVO zB)U90LKnsCz9?jxuc0^f3T_AQT2TmIESDh!wI@UPTj)AVwAvJcy<1aAfP6GW4R5u< z6fIgOVL&rO%gg&acgaP+Ji8aU9~gAQ`%8nvmr4x|b+*YxfSIBqWnOTi+Ea|W%??OV z!r(AAZ+JM?@GqVw$4V#;)GLmkmBDR;WCQdgbO$XOgVvnF`O2HWNe%r%ni(0YfI|KY_~fl zE8f&IH_Tzk&5d+#mARp5?HM7Nj0t@Wil6Uqn2P$%u{eG{q7ModIyXOh=C`Y_K5CAz zcyrv?^ ztcwWT+&~0A4_AQTS$ZR4LLSqI7`j{jLo)Hwv?#?QhPYa9 z{+L>;zdTv3ZX5^Z$$H_A$Q9E9s%CLQ{oz*xrvr>5j^#y^)J8?uf#KP8i$^eU;3odv zp)|CV1Ta{XCLr?@3&k>qwIOiymPVF#%(Jx0rG z%6Nlyq?TSmF{}>n7J?~_%uQqWXTab(DjqQM(K$U8H`H4h8{%wKY$fvffB3=Vr~XuMl2ZXM2eyDCi6E=`JYYXq_L z9bi-(i;#?=;$TF9Bn-TK#uwOUp#rB$&xE5oDP6G39J;_I_h3hG6^|~^Djp;#6LUz& zD5zqKL1x67zRxo0{yGec&!2sL;kzakhjK#SXLP{`vV)2#=iy3J(6R`DfzLz2f+GzX zF3{Z;E{G`!41zZW22toC1BZ4YqX%0Qkb$T6kU^I&M@GfPVO+MrZ@t;#h&-&>V)XL< zV&Tl!<`S0nK<$<>F91fkD0rT-i{e%@L>DT6DUSpPs69m=1(UYX^}9ozzHD&ET_(Z9K&N8EjTwv3oV8dAjMDN8-Eaw7Hv2XDko^( z94eQ~J4b}Pn16E8S3&3lr}b+>!M215K57v}r}Q#00vohfLYW@0kgmjtH}%e}0)cNf zM@cBR?7$?fUN6VRhsBF$G-$5SU#IUR7Nj zbqJl$_;>Bw;u?u0!Il&xFz}fk!DnFU5&8YYdBMF>HG_2kw!%iHry+{6IG+r?df+UOjr4JtfM8Uvgq zbA)C~G7LGOV8W{T!5yu|nZ6cb56bnC5gY|Z1a*w5P7E6w=mCYGPCB)jNUD-ks*W?D{a8|3pW^crV}tQ z!cvUjTj&4`jxYh!6vT4@gH{{@7%Zp_hzY=8flp77u=%wl2~|e2M8i{by$6}iuwv}f zM*<>vbFp0~2{HD&EoN9kawuj1c&$r*#Bj_vujV>x^sjVcYImI7ji%BbJNBlc9hU~D zaUlYUZx#eiV(fUCo)(ASvUGq+Cp}FLk=GGn$04mf8EkUUkDkdPK60D0x5&thiqI>0w87z7A4435Da zxL9s&@O&O~;3wDC;5UOfa0iPwu)EXN@J*W;5e-=tGJ{SSg~?Fz8r0xK+mRS_zSkfL zUuzIX&{HNxa3RG%3K2L!?J34cP7jzHCO{HazcpZjFD>ODS{B?ahsl;+uMHE}Kno^3 zxJ#1Yy#~UBqm~*bkf`&|SgaBkGXyM35Mgm`Hx1QyXc#6pJD6$Ns)wUd;i{bRZ!?db z(ej$SqglZj@z0L2y4=E`zf%K)Cf~uQ0aCgQ3~w;pr{O#228L#*?ff0QniFOVYfO}8 zcAqV-{@SK0l{>AnDH79~g$RPM#z)+!5uqdpnq2|{Z<7H62N_N-pp$;?aB?|lmm-3F z>I}x9)%f$8W*Ft7`Jyz^53CA#4wrBDMpF? zRA`{BI>adY-Zn(Ilrlt0SO6hvPa-u)z*3G*4j7UUQiIV$Lus&hY6>z+4eZLL20sx` zl5QqsH2P3zP@E^WH_q(#VuJyr7DmlYeNlQmB~N8|!SZ2l(U52Jn=E$PF}!;KIzh@U2@TGNGNa zdSw8SrA=|6PHie3;68AfSxJVkkx91x;zZit&hM`jyEJB}G(U<Oc#J*R959z>a{vxcirC_`xR@Aqcu$iv zo3X1h3i+_Q_8`IhLY-h3hv)Kfx=v%0m8qinl?1|$NhiEC(nX7!#vy8DuHp-J;|awLrqgqe6nFB{?&do_>XAyXa! z!|lK$f+Xsv)TDt&(7WP1OFW9pBlym9>B?`Z)f~#n#TfyHJ?2@?PRQ&U6Bvs5=FNtB z#(Gt^%D8NZWZN(*8c%2uyg->snguR3l{Ay;T7W6y04u3IiE<@nN#!~{;DR=SN{+Ft zw#g(6(xf1xcEzh1VgzmOamXf#ve5;CG{_4%iQsD6lE~mxA{5NTTV{ayh$2>(i`zSI z@UBoz#>*z^G(j3QXoe2d;0ROQ&{ivTqcDPoS9^-6IXWPl3Dj`B*0Mffeda{r5HxJ! z95wt>J=AQt(MAmbY@i0sXQ2k4K&R;$oGb#=!2W5}Je}VzzTt~AA}8+`Sv`ne0+vIZ zW*4*hRToFgGhw}-xKRsO%7;Y2ayV969xB|-@?f7~d2$8I#oAKKgY$N)vZB)EoEblID z)8ny7hZjt=r!f<3mpKG)XyOYeDtIZrzyWGckwL}>OvjcC;s{tkURxd9Eawk2w?`lN zDfFA=0OMK)d*F<1*(1Px!0dh?1+S=)@&e;@-jV<>?!$tq*Y4nqKg$m9W%h4@KJajo zJ%EvfJox>K$uhV_MjdEmgF4tPiD}q(vKemL(n9@=?RICf8M-mita#LxIk1rv2gSNe z9C(!8#4)i-gZzjE4V|=iBXOv)C4LOP$?DM(lhX~ z(l7WP&VGOT(}&+S6?a1c-N&fLyt)PB@<*?v?jij?ZFEL2*8iH{EWMmogK2gP(zCS& z$&(00OZ3_3=EyT4dl*`d%z0REu`-RnNRh)UTl&TnZmIFXwj>tp5WGSzd?2ud<#W^V zotd{JFUd}W#iSMr{uDf7MP8iYf(2b`fCZ?qz=H3l6D)Fua35+pvChU$jvCv&mb;KEpl6-n3^SezE62 zRRiPETY^vnI(*Td1Hx2)6DrGK?M-yaBeByunu$1H+zbC4ZN# z49*-PTVZAJbV8AY@~(y?shHcA!i!d>EqIYKMTqq~vp6}k*kulU8U}NKhb-p6g*ur- zW;*s@j`1nSn=-h(SzbHE-i_6ZcEL(8w264dyLVs(yEhQk*wTJSgqTF`wKT5yw2vAFbF%B#v2AB2`gC8@*9HKV_yBv9lv6g4GH7uUe zgGXlqJQ&z-|kdy5idqZW&uEkDxy5<&C?>)jO2U(k@Rq^wyb!!h9ovm`;A-^du88 z%@6G`f-*j6b+spn(c(vCj*P`>PP|#gUpc`tbMtcH|!CX2-}8FguR+quq)>kUd=H= zyo6_j@Y{YNf(y1jbsrmaWYy+GA(QJr{e=XUUz~|NjtT1K6flI~l*|h8y;9B6lq}Jj zrTM|d3GU_N1V1XAU~UGCi+~DNmvijUm|98$hMF};%qghDgA7x*T+G-mG|14R72KrL zDWWq!5Hw~gK2m6JR#)C4Snn_AHx6tN>e5xSx*>xm5|BY7mx;wK_SpR3LT0cZ&E!N0 z8Q+n%3=xM~49MV}J;J{BH=Bvm zJx%k`R0eK4ImT!!COgKBO%e;h_s^t@nrtF zFxrD<^pvOEgzt+9IgSj{U3$0@g*CC6i>-qAt z2aa+vc@EIq=qMDEKhEnPh1G8o(iqcIlB_|qkf8%_YS2c=o0I6EJDh@pbzD7g;Lt8` z;91(9ub;EsuCto}Z@nwd-(meb4;}n0a&S~KK%&sWF`NPLz#TL^Ufka;7i)J7J+9fR zUEzj4iZ$iYP&Pgo0(aPBN_#BXgJ??NQ85%D;X!)I;DL6r;DK``Ita)V9d&Ell?i%K z2X5PTy^RiBr9DKZ!{~Lq!`UV4&B# zXoIBFl{G7lk;uTqc(g%w%z;tKK$)Tqpr$u#9J2&pjtnd{VMD_hByI?!G#50~#xNO! zbN3y>7_&Nrga#w631P5yhvW;+Fv@^|Ml|?>ovwhP=j8BqvG+^>fQ#$*=l*a(33$MSd<7O8jFlD=uA1N zI3g`dRD@+&5?BQ9MdE_WGQ$O|kAVx8_ffd0g33+us_XJnwk5%%cOgTNLH(c_2KX%W zuBbB#GJ|cC14Gu?RKUR1dIMv|Ig%O}qnC>tk1w9B?zIGsag98{(UG=@ZV}N9@Vbm{ zV222Zx$B#t`4zZS^a}^5JxPq6ohRd_SZ4t3E`l4(_%LFpVKt>hq|nIWBdD>P#xWS} zNVn7cfG`d?EPRTu-wK7{Rfv{i(jr4atqSQAKb_Iz(hSNzb-<<#U+TD^u6MR5mElTgQ|Z6Q4(TUf+_&NYYwdlhfc zS|zMOfPPnVf+z4u16|@-696#hjACs8yhwk}pjOjpc>gE-l3N=N%#HW(P}CUomrRYp z8CmLsk06-j)`ao01QFF?&5#Ks7}1XKXtEQ569W_2t1!W2>J$_uQ-XHxG9{wRZBs(n zIU)lM>5;oy_zZ%QxpKvO%%9%7aqhR(i=f*d3YdDb37C)1fUuF!-2s_O%#vsZ?ErfZ9F_# z&-Ic;jdpMcNkZt!#sghSk0#jt5)dTF)OwKK>#r|b3_5o4?+)t$OQazIRul0jKw+2vpL<(Hadu-JH+=T7JgbX1I&tbr?&}G&JFx4F-)&Jm6g%lAzBl z@qnjI@PIQFLG*?L8%e@}?30KH9N@77e)_k;0Wi+V0T|g28W{H1Xc(0$g}j?zXC?0b zo8oo${P!BA;r|V0)2IY?rkiPy0zZZCxBoO;oUv_;0;jgn=@Ftho>{ebUoJ1^SD&5H zRE(8f&eyK6@EK#~Sr7)B9m+s9mF5ge&xADcVRpI{J*^0AhN2Ayc~ah>F4$gGusb@; z2tW4b5NSqmZ_kWSisi&1XG$ucv}V^{<76|t+5=WFb1QhivzFO?VCn$dnHCAXpx~uq znm9o1DMmSQ9}(ijwh`gP-V|GOm>cA#Tmb;2n<;hRr-!2s;A0SV%w|lT(dMO`aeja6 zj*gT9TF{YZ$NIcMbt0t(j!jqtKkrX!06r;YpqD+$AkaG2pTZ$6AJos-ZttQz@mM)y zfbH5&SA>BR0{VIktj|u4akR$5=;{2^da-t7r$2S_p|Z&V0uxr~g1z#^$pLMapbT%~ zfig%W4WcVV+ftVJyO0LntHnG_h4cT)0K8MZfj zmr)1Ry^$$c93lQ)$2!r@32Wpb3|y#Hliz><;BkOnsF|v*~E|0AHWf3!yQ3@07to9T&r?CnM z=LQ(Mlc~umyl6Y<)MPen3;4n z_bWAA-l)NmjeD~7#H22;pd&;YhG&uD{&t*2A!8s`P;MI?F7(D^>=bDaOR*+o z(GtF?Y;r92T4*7(y7DmU9L1R3)kAauEujAlE$^3KR%^G?*pu}#tGV6b&vwpBLO%+6 z!M3CqjTmhAv|76%2KNzg!M4FH*sF;JyK-XT)&3LbI|3($M~h_)?{B|&mD$ghcjvkW zccV|;K{n5!MawzTG*hPdQW7xHoDX=or+>EU+4E<7FhKZVZgk2D%KJD8WMQIuTJ&UaYezL5zyQFtJ@{2 z@2+jJYo!0|Xt7{31^=M6%KYP2|DY$Nkh&WK9nn4s4w`u;>%@qx;q%COi1}6?5f#_O zBeZgXJx{=%*D@oJ_0%L{Kn9m$j2=sC{+j_4f$_ZogZoIhm`K4odIVj;JhGnVajBO!;Q-@iaQ)`aP z=mnM`qi@&^uonI|(Z#A8CyGK;Z zX8%UNskIqVgHUb6zs=A+dq=1dyhu+mf+N$8g&0vDc@uBPjy3Np1Skn3NKwH!iFhR=+m~@g0s~4C!iTn4(Et7*&jT2`~ zx&Lf!e3>pzaI>HxtZ+&Am@+ejz1!ksaL5(qxk26rCs)tU`w{W@m7wYX8cH+WxMe zvfZv78C*s0igiDNer|jl2?@X@hXk`lTu9)>20#K|uZG0W%d1;A{$84tTS{2-3k9)= ziHC~Z>7DRX4)#1Wb6Cbj#6jp!WT*h}3{=qNSFj*yrLe#ep394$x?sVx8yc`-Ar2|I zym*C{DX7bHSg?qa3yWb)fj-f&c)DC8t<{-D^Qusd>5SrXY(@pw8w*mRix~iJ+7MEo!xrOwLS``hEh;#YL_v00WWtpkHEB7+4w$bbbgr*Odr z7q}=;1yI!m3}GQ`V4(L~HjzdoXrK!QKm!k!L1TI2aK(>{ zH5P-pSb2O=E^X@;1uI6?^Kq7nX(94!Gj-;M;vzXQcl8l(ZT9wz2Q+=v-?LMQM=Y%%g|jv3XQJ=r5$k$yoQKlpigDE zD=Yz=4%4zwkV!xIs1*6^7Ew3WF+A~tq+$7l#4gd;HT0H-E@-XN?KC}}@!!yGRwx6% z86huVYU@x2nz}_$p9pD~Hm!*S>vk-zJQI@xrLiz2+2Q-HT#p?T{#k+ za0tVdHFHPW0Q%yejqb1wOwJ%=jFdqN<^>Pk#UFwk>W9>1Q8F(+eb~{Auw1!UHXYB> zn?1%>68z7LuYW8TSI)2;QVMmZM}rD~yw)u!RVaR3F{!f%3C3b1B+$`4A%Xr-k|V64 zk$5M&L4s#@xUB_9;3v=r_6RS-23si-;U0nl|hVKFn`9JqdciO*CBNYdy%zh5T z<*K72cO3DoxKY~{j%g&erQ%ylCK-_S!7zs65>8=F%}YH$$rh*pPnUhc^LLSnI7GTH z1v23{T}~!5Jd#T$xKVHWGBrnKe_Z|3#>LaitT40%Ie*TI?@E$dKQx0J?KHW~mY2#B^91>b78)t+1+Ll8G&)u9%8A{Ps=a;XGn_C+x zM!sne72>970PhZWg<8_DP#)~oegXuz?cxkvzKb&g`eT-%t!6~58SSi*}uY)%O=I&08 zSdl9O$;SnUVZFlH7;{_d0%f}ym(UPNiaMT2q6Ix9(h1o&CR)%Hihg!O3p8r@ejoqv zC_<#7NQoPdTHdMXOSYS}I>MXf<|SmkJLx`@JK+DSbjT}MmxU8ur67bz5NXQ+O-bldbqL4rh^EW|4AS!!k1uc>z0qw9CB%E>pB0~a} z{aU(}hTdg$P#Y)|hgG7w?Di=l0MI}2&l(G8r34G?P?4_zQ2+nP3V7{(U|}|6I6!0J z)%^TRA%h2HFWo`;LI8uAHKvJZB3K4BdtO z=cQ@jr!Gf?z0?s26^BSi1dQof3cPgA4{S#l6!nMGuiohiljSrh>OiPhitPQ}B3o_2 zcSbss$tl@{Kf-~=vcz;f+-7=Y3$I-@Ci72a9Q63eS?0W=qm+V-ZV5e*8pEj{!eq2` zJB?z@Y9rVvJ|QcKhP*%nuC}erWPVtSq~@mnB!m-lh;8(Keskw!d@;piy zComFPDT{o8B`nPF;7(t5IAjc#foJk%>4m;tELQ%q*1qN8Qch<7vc8e(;HW*uOm2xA z8&L|FB!*|>_$3YsIV#g4Xi>xP&}b!{3xg99*!vkCJjWgk4<4+wP15k7y*$H1`CFkwNFt``;am@oMT)h7Y#KpZY)vD0zG0?kLaNki1yKy-4=$X!*Dux! zz5Hb3&-gsjWC`~7X3(sXM9?w`hoJSOcEyg2L9owY5T2U!ESLj(QqLB5@{}u%+yjB& zDjtF0CyzkTS2=%#XN_S{MIX3UZ~B-pok>Gs^!|EzcZr8JhGRdh5gT>XhgZI%Q@rAd z1$SV(Lx4EcffJQ^+D(7pdP23L*{!&!L>4&%C+5O~S95}*_$l<66%)m8LWt}<1riq= zMi^6ab%v66tN(M4vUV0q-p&7*FC6>gXw*)SXaj>sDuV>bHtkE?zd!rJWPgDZ=Ik3l zv8K!7X>z1c+jvl+cjjV+WYOEP*)U5ne$G(iK(H`vGXu-nJ)&ici>#^IfM5SYD_0h0}FE@uKZw% zkhvGN+$%FS|qHHR5f;g!Z;b~6GhH>0p(xIrnA$UY&# zC2rJI#IhW9`-9pMJZCwV@Ykd0D>KI-{@u)nz&#AVnNED#zi^y+J>fayYM$$iL%UpO z+@SS~>*s8@>p&ayu-+9X^P@Wx*BM*w)LhPA%qfX`TL36d;-uwZ4|}XyRZTcaSpx&n z`Q?Z`!eB4@n;3YqYc7(Pw+$ z4&I;{Q2_*8F{R}E+m_U;lm{1=(Fu^cGYEf`|&!C`Rx8=Vh~eu2*)ZD4%tjJNz4j*C!-J`&7cr+ZPK-%5E6Du zA)zU4yMjx0QAogdobF&#h+vuC6cQ43g1`Kz?dcx~)nv0yWQh_KH|lFwCicQ+Z&!c* zxo~D=epp>xjD8)uR#itziRKSiY<(K@W*6A?$i7+dcVB-%PAUS)RO7& zUphy2&>zCs;EA>bOM@cU+yIygKgDGerw8b90ha3U&{+zvV(O&^4BjON44ohkFq`zu zrtd`T!?rhop9L7~?gWgi4$$1x4~9%^pZERw<@NI74(Kz+jIC(M1~cGxT}Gr6E)<#q zFU*rI1EwjftP>HMA6&rDR{V1I_RaGT&wt+#)ccqJ^Wpbj{s)m2T8U-@0g1;yOb-sx zG&3+A99YdJA|)?#x>28|;yR1Y6h)>xon+je=rtOq`fWsaFw zqgm$7{EN4A*@xw|=lB+@iQrvLBNF$wy&Q!}K9vHWO`=ht$ZP8#bqU@*{@Dc100Tp! z;K-B>=;h$j_y>kR=n|r5`(+<^cH`F8&)9C)a)kZV93mA8&Q3gE9N0QJ*vOq6^k3WM z%J0Rs0{Ws}!NuM@{_Xj@9}vag{Pk$l`IDKoL?Ywvm`ibei<=ny;ZNK2{F@GS54s@49y#(rGCm+U1X_-#jx)?7HC`I3OYy-ew&`}NW~r}SLqiDk&OPD5U3j5$ z*^*8lyqr*iWq5hBTzv8@&8uvjW>+!?0kJhZvRzaS;=~tiItB9^8c(hBSvF1`n7a=Is3Ze!c#Niy7v~ zD81W(nGS+M2MEOAO(ntbtP(HuP$Et%@^|Gaql0mcvN_ots$2uQV;muWyOw6Al_{RD4`rV^XNiARku#Ku&RXaODPKt7cHQ z6&Dc+$BZ|`jm`KT_9U>VI5HXfpve$m*Jh3mz+~V8e3gFqz=f{4%5~N}R|h|J0V4>k zaxj*C*wgx-6JId;>u50TzI)2MgT5g9T&#IV@O0$0dxqrFu;y zpPZ1YH7G_O=AY(Y9EIVlQD>q>hZyDi73*4sFD|KT)f6MpLInpZd?{B22dF(oZ+CKV zh=&v2?$M#+?ed{g=JAJq!49q%QwJc7cO!qa%Y$TTQ3@gf&>+->v7!7qBVh z+3Mb%##a)fr?D?}A2*$!tB#;4q;Wx8Cmh00Noo$zamFCHvte1VH|<_X4tfqLfrKy8 z<`3eoLyCd{T5_<>3ZA|_L@Ewo!{ZNo_?N*vmp||(0fjw`%JGyD8V!lj)7AQ74kdW*(h)*qvj0Iuw<{xB%!e~w-#quOoH7sKgBs)obKGBsYl%GcxpxEpDY)* zUf9g4UeZU7#Lm>c3^=Gxj?6)hi+^uiTFD}sL~#7}BqBM4ApNcbG~p|Vj*a9H{N!;6 zFr7;f0^KX~j%)R25T^cR4DyWYuzS<@o?l%r)@r=o4%mGHE|oJmIMgf8-~z(T;lee!Jh+4RYuSrN=MJ+P%`BKj#Y6lI%vzzX&pRwJp1qb`EIYep@ytoGlFldekBFM!9-?V2$eRj+U ztj5EOdt`}tMR<+0#D*n&JOtdCfCy_0IfyvolnRLmhDj5a2-lU6V5B+21RB`DL>L51 zvckX|#37PB@RLU#Hg}M$JnF!2 zcz|A^_1TPxE*cAOS6An&zc3{iH!)H@isNww@yU3PiHK%YKaZD=IVvI{L3^5Df}=7D z6F6mSQ0Ry$Ny~|L06$%*phfCHg*Zf_f{Mz81-{|jrpSt`N_*mr{rpYPoQnvXxNv1YqFtD#^^m{(`^EIHHRjofAB zcO8Z-!e>YcFIl6%A)(Ec>H_DcoWZZ&u_HLLi!;!yEg{&PfroDok#-~m+w2u0XkK&9 zpb>QCMcqohIOAkY>ePT3m1p$7LM_eAmc#&P9trIjxyREA%0NezDWe&@)nQOvNdZeS z1P)MpikvY$2$kB?ie`+&7`$Q4YJd&OiK95@5OAFnv-}%e(pYa_yCCe!Rx7x=L>& zQcIa0?wafLo>VtSVLImAlDM%MqXj4ud&BHRX>YJ26REh0AyU~nNe{!D9Z)O;m+Ue) zIHa{Ew!smHNP~m-@I)%Tx7^^cf!priUZkFk*{B*7SU=(qBzxc%hBu2PQ^7A@)VzcW zT1oTU2XA_yz}rhqNJ!v{5)o+Ro`}E)q!7VN0~ilBv67!~P zPvEQ795lk%|CiNYw+m4?&ZvI#_rZ>wo-(V zJwiRK5b{a=9?H*jJIyrXy6U0(tXf5(tuj9d!pou$+@zB}n5Z3v`KLlk;{p|_16^fya@^lqh{zZCCH`4c2X-XXfvxXG9k}*?40VKO-F?-XNmz(l z1K~Al?tZh8faS2Z0s^y=OeAnkarL>!{tu}iabsgg0NVr&9Kn%rz}0$M5`2Z}cMeOU zNAYki4-c?1U8aPFsBvuShitcNyCDvdrX=VcC;Ez)&TR?xK+6Hj83u_2HtI3&VNzME zvW|Yaxm*6VSg&qvSXfEy8(Z>Uj|CP)QCVs>@Fl04sTHZoF1i$tgyzI=V*A0zZBXtG zMov@5+4<+M2&UO6G2E5LjFhM3Mv}-}4GzyZ^l`)*u!=rZ3TjA5lmx;&oSqD#S62-b zf)DpWtp#zVoj^i|k7 zqDTyS*#y{kh;*=6gz!~pj*a0xh7?nb(LEveWNu0^=@4fGVPLr_Xa&QKbpbdS^Kgm23oWx#OM%>)eXk1|)vmBj&S zPtmzO%8O)H$L_k$r?7|`G<=&JH9XRjCCrO;Q9~f)u`zU`g&KZ|U9mBG4#QA0rCXp) z&*%vz=iXf{9KXg-I`Z`Ez!U-)7m1mWE|Ic~8&BOgW>AS24KGQ?oBbaWB@D6BWRg~#VstMf)6t#1V~s;%1o6?@GG}g zD(J5+rGn>g50PFlJ{nJ{aG*TbuWXE#ER<5I8>&|h(qzhonKBM}e7^W=aj~*d@#=o) zpj`)rpyh?#0TL92;2DKb;YB#~ zeI^Brfl!$o1TB%~AWdfuRIF8f@tlUSp#k%Z4Vb}|CpJ{T3|m7*#NGNp<%r3G8MS=G zVvIIUo~%B7(sd6zVrw15g4Y$qf+m(=K^F^NX~tx<>53rQ7g(lw(PkGc3{|AI%lClr zzAjj30KMGk&`y1o$;KX)Js>u0GXtj&SdN*-slkF05|QZa-eRf-vC!^dQ(6nrmfM?<7gz@{RF;?s@Q zldw996yb8M2C1L3-L5r>I7G@6bdf7l@L;)2IR;o-94MKB#|?<=6Hqy!;L-3Hp$^gQ z`TTC>#bIBqE}f7HnqL=<=-?7qS-~Y}rZShXq|SpF9#m+?!w5I$)Jq!SGM_2Lf}4X* z$}JDQG>=S(%FuOII3+YxMkeUKZ7t8>q>;%9SGUN}@@DnbiP!s?(tC1>(I#cEpg~Qn zplcKt?Iu#VByaeNla<2VReEvr3@a8?ctJ;_;3pR-`2Gzwi@nFU5$+gW7u28`nD+nbH5Pnpp&#*X+#>L< znL@<1;RNlrRwPp;AR-oyIC>B_YGAO0PKt}L#I~Rd0xiP@dfUK-PEQ6dkjyNBQF3R# zJ1`_;psQNOI5EAMim_N18EDPElrhEnZaFf}=AYd4Lvlx^4=&u=Rjl3bz+%D_bO{Sy zCV>S}Qh$XJ2{fYd?~WDD`29#DqU#6nYyYlufyw)gosmW)NF6&HjU$>lS1>imMFg6y zFCvb{>Z4I;>(^?PQ=|Rtcq8uX)Sd{ zIB_=JPV*FLXG8B!6Xdyt_=IuJ5=#BCw74ZYx% zGRVA}fBw95QU#EUlr3epD`wEr0yEeyV+N!hkf8|%CzcTd@cD=#GUX}(h^?}wxL_Fc zwm~pqkVp{>o+d|2$ZU2(tNhJHOK7Al2#;sp7A<(OVf->X!Mkg)yuskFH}|TPH1F%# z_}S~ujh&tcMA&Xp%c5E8rblyz6>-DA&9Y|rITNzjXSlrBo00%LUS1mi#3}>m7SFfA zkzG2LXE!3Xe#UmYc4TlBy(`8+@RO%w(IUBw1sAtHuEJxuR)AiQD9G_CIa>o^^yL2b z(>L6~QnAY~+Og^2j1EnLHxZ11*DW&!!xAQC;LN>J#`FNsCJ2Mxu)!ECN}588_6470 z&KS%+_UsFP3*BbP*U(5AW8jQ!8KYjMEl%C*Azx=RhKDp#KnrWk$FiaF{A#IE=9{pH z8(C+Xmf5*X2vofrd16fEA2;e)S%`fHMHs$BPBW=L$;cJV&WwLIK@ohy25-=fR_;tfjMcsx-kqsZqqA;ASY z9Hubl@**6R$cqpb7cq4JF9QU+#sCD(t^@=ilHvi);^6`3cHsdZdfWdt9t310O)$KZ z@C1%*d4gEUc|zu|D?IRBdyc@JjG1So@i2P2Ufthby2(NBP{LV(Vk0xq_5+iH@UTK* zpedxnKtBmT){V^LN(x!Z1;GJoPccS@L_>g8NQ@&S4A~+>*&$jt5>Bvksi-JTwPKt+ zbettJa04UmfqQH#G6p9w11orl3@dL|t3SQ`Wje9m)Vw|A2cbVCUeJgVFAfv!8!m8R z6Il!{!tZD4iiPGG`ip&!|O11yo}it+>DSh!v0ELTXQuS&s9Xn+$ICbY5&;!OOZ} z!5J>EpmPnh;A$3Hc<|n*Rsk+}dCe<83no7}&dXbr24DJOwIEFB4VrySI#lK(dMy

50BB=FnM3$Ntnll8ZO`Gp;o}Dkgy5n}Q zRysh|xVYt653vVz&w?6M&qk(TAR_)%~`G--wmIax01WA@e3jSfe0*_pL&R zJD3l+0O;qY zLze1RK)}WN+YW20_UP5}8r5{2Fo&_AT}R-&Q+=W*g!;rAlx;_35{)ffNHNTAwqtz2 zsVu^LjJ@0sYq$+BlKU$706Y@{>k_)IzpsKjY->XB94(m`zDiTcnSVU{>DQm$|Mbhx zKm6k#BmDPBZKu0_4~3gQE^hDI)9QV=5X(Jpiav+>&yUy3f30rj_-3t=(9f3h>($N0 zr}_Dp+sz?PRp@GLW~IDcqPB@Q6%?_VFFFx+1LbY!SfD$GAsKc&AXXz87uqY5St>~2 z*eg+iPbeiy3Lz84;A#)WB_R|VC>sF6!)=RVP(cyJI+4DbkUTTF{PYr)$#2|pTKk_h z92z3g%K$=|^Oy^1LM1||HK@Cho-xyJ>X9A?5;a^J>6YSyvHEm7O?cy6v4OOQdu$6B zx{SjRGbOcWi22AB`aEg@UiKWJ!BY!n!FO853wp+cOWeONxXkjoTg>@4e5{apl!!2L z3qYM)NJ8BM3rEcdf@KGGp-3DCmLrbM&R0dRAZ~1;c+H23HpNm#I3WDdMDghFWiB~*)9}UEF$oRO7sJV^FBT|z z>ncgOYnLyEo^B73z8DQvJKzkqNd_26!U*beYFHC|`GM(p$!2ZYV$KqtC4ew)<8p5v z|MvVHbabXnP7iT64T;e&SGV30(l6GF&!25OgN7eCx`?+B1_tdQ4GhQJ#BdSExJ(m+ z0~NA%QzVlEX1)sEI6iR}-U_qH5_HkX5MBINrd89kb3%E=943MwuaAU|-j+K!x3L9~ z;fdu)VWv=q6im#xUtHl@hLw4O1SiMW@dk}L%uHy8M8KlcC0;aKnwc5j+t-;LFm*z} z1zEG`AVZTdZ7?Mx!UWxs!-W*(!G&+;ni?A}Xq;`~!Y3I7m+_%B?Q8V-Vt(yX3xdCT z!SOx2#%QxLUeFCDyTtwb60l$rU_-!#-@gGCJqg(b3#;*XVBv2rSON)W?(r^Q!QF;| zWx{#Ynp>W%7Z=WOG=9n!K51l$A0OtVDzkeFX-~#VXV-0d7*Cp^RGg*Rj3|T@` zofrvnsAD#BEDdLSr}%QoBh-R)gb8PTy5s@08o~tD!m>0# zd*>h{tECMik0WN=Y4Uh6U!N}#;-l==)-Cbb1Cyj2`e1n&)yL73ndG$y_=X96 z7>m~9OUw8J?`ZG`yGHT{Kf#ngg3hzu8a(wLhL~_a&#l3|6Sqe22m35{Yp5C8&fk8F zX^drsb4-jL-&`O;!Siga;w3(RbPh0u%+NHyiSQ|NNVAfTR;x`la#w%g(^CFUlHx>r zEN6604~R=aBI83ez9m3{ct}7-5}BNiaM4_`9)|s%3;_aMB0+x{M1qI3hy=9ONhCx- zi=`eO&iItyu_lty^XpG*uU3J-%BM4!)8XHQMo9b{^r}P*dorCh>gM*NsTCtBc;EoF zCjkt8%m9;DSM%nKTJA7a4S^%vVZz$OO3=MtSx+=Wf3Vpl9O^b{ZynqA-i?3yR1kG;%BeYdw zE8xh48aw?Q8eDS^z##88X)zKQco7d6wjg1eqYDf`sW&bryuQZ8=>7Hl$_bu5p0Bet z3Vb!vVOI!xAw{%ZA>pTXg)XiMEgYygM_CaZp!O8)%H)7x6+&den{HCKQm^sYG3yx`ApW@v6#LdoqG z0Lrk`7P!~Mz{%tk&!urOdV-RBw|HJdahPSaHUzS#=sRAytWN1;JP?aywOps3I%Nf} z)MuvU!<*zRQVOVU(n6arTBuQ5o#M8ZE5n1k=4=ceteH&d%w=%-u((=W&%Ie2kFV$V zdmgXB=O*nAtb>G%HardcXcw6FbI+EC*?YqBR8wz+FH|VisNo~!u;I*Z&#B?Ws~{Py zWJ>@fnU`gnF_D7l@e661K!MR=52Zyw##kM}Y zL1Un#c*Reic;$DNdPiWd-pQG1nK)I%;QMRqTQF;~4ruHNXmi;3`Y2yQaG?GV-$4)@L;y0N>jMj(E zoAU@43G;Y_Zi&kycz%ONa5alZ@IIY9B8$WJ;gMrjQq_ogvbceabvWcVWE$zg-yS-a z=WZjPierr#Vl>`WtEwf;OKZ{P!qe>N0?gA59xsFMyb2}wv;ewKD*1l{_5 zIV4y-d%hr@rkq5i4N)Y5GkTNAj8p41B;MaGZ*JUN`LKw+B9aCZZt@w|Q!ybiL)U~K z3`lSgMnZzI_{6HrGDxt!dSZg9Li(5Rf;p-X#)DM+)ij6>oB3g!aDtj&dz#YC6#W_k_ z@az8N?CqQ9AD;id0ck5R$O=+zU@bm0rBEAV1k7-izC(B8BYWtss<_`%9O#kIf0pBd zFWwLw0MNFAL-I=>!Es7Er6KcTdG*CxE%*rwYOS}KV@fhN*`!(6Bqx&z9Y0AXAuOIm z-En~A5{4KuE&(76F5$V7N+^F*^McOss02rLQ3(!dd15e?06HF%*rPEVfpCeWZl``h zvr{W&H3Y_xg@}6?RAQ6@ZX`#k$Xex6K;T8i0TBvEoEH{1YAHHHa|t6;+tb9I#Q+qt+0wS6* z-mNfY=FZI!Im_#JHsJVixxT$aK=u|Xi_OlBGwM{V>RODDPmn;s$^M_bZtqiuyHJ4a7DsE2js+o zvpwFxPcCiXS_W;Pjkcu?d@;jlBZNgVi2Qu-7F7H7YJRc2UOL0{_&5hngfUP998%G|#$(HtIXimYC`kA;V!}xVCS+$t@q&^S}Jn z1&Z-5M&r%yt(pSAxodmGl}4m(3Bk(5m!aF#X9|oZd1627v&IIe}oOM>?1Namb{n zlPL$qGoYMdy@~nN{0pvTI5JN!vF_r(*Ofv4nQXLgsG!(XR`bv^HfTs~Y|tG!R)8rF zD}>u#j7=Y`z+jiL5~^v|Wj51dn@?9)D=agCO7Uj3&3CJF+g6PpCvo#b{dmd*czsehj7xy(&!>Zjo6m>@zGDPfY{~3>E1l zu~#_Fvsd_O5B7>T*waW2nyCfQ_Qnc@WnlI+7M`D9F8)HfA#ck6 zh{h*ZG4!e>901+=v##Ng;Q_C+H6C!Y{{TDywYhvLiI->vd$`Br%9=41N>2fBhCO<< z$W}7xg9Oip_(iXeRKIk=DM?2| zV}$i;uGXt>xQ8Joo?hN(Zbpoq4i2KCXGD}8#3iHbp<$CrPAr#X%2T|DLYIno-~hF! z7z4qY6zr8G1cF+q=qO{9j>XARd@*{5PN?CVda~robUPiT3!%CPb8QD#3YtvK;+9Y5 zOJoAsKzX;qx>(8|?HKv*OjJiF3cO%j;srazkS?|#1TMIb01LJaq+qWG3U=i{;nked zbiBla!2~yJX{N_u?qtdZxida_v$*{-zgeE!I58FJ!!gQnv`bpR;N|zqDAQ~?&Y4#( zmlLWrP(VjHP`F5=hZO$iLIq4_4B(;jtN}b8+!a`r>a`zM@Rk`?&hBqkP6n!xi@Y^V zXjaqVU2IDtK~o4K;rl%Zlc3lo5GlyST^HmCPpNT&H!*O6y&g`AOFh^)!Cng|XrfM> z(Dxb0DV*Mz;pE-&&VkA6#b4eyjh72B*zRO-AycNQL^!}iOWfc3%1V^rLTXP@rZAEq zumYMe43AZR8)xP~3D4}Tj-C_4ZJEj1J;7N2~$TwVX^Pj8s7b;x{IiGo)yGmD0b zXMf`UeL-b*z_Dckma(@WG@Ug~EKs>QUMO2#V!@lbctKwo251Mo1Rre3(B9<8$}&RX zI&aS9t2uC_1Z}?{33=M21AMJYV?;MNpPtQlRNVn+RHXC3R|OtbdjFbR91~7E34h+17!G;tY_;Pm$m7qV{ zFi8EJ?RFg&p+fK?CBUf%4l1-99Vht2Ty&7-`i4rv#6^vcC&&XpF$bq+KN3~%wwV{f zaF~saW|<)a8+k4f=S`>3U-kP6UebKBhng^p_l`GFaR5PE{%&7_1HUa74ug2YDwRBXjTkC!V9eY!`I<^w(5M~E|!-9lx!RSU{TW+OMT*GN7j7I(XWfO=cgF6 zF@4N*q+08ga~ z1)J0=v|E{c#fN8z6-JKIzJ}Aw5PYguLN1U~tPYRTC0F<@H(pEqob7h)^x`UdS1eY? zn|peNubB%Lmgj8ALpe4?c3-D=#RdD^rHo-#%d)u!hQ;CGwMR z9H{)f;F`P2{PZLez|!pIxIbqRQmrHrd{zmEU`sLxc4P#CeFlN>)TDdCAJ~)92R6U2 zv%?!S1S6A+@RLU#_$lWOa+Mbo!3n*o-oSwbx;TLZc%_Z7vqV^q3C@9XL4g+N4+_=*)S!5`SU43SN{QH~?@astN0PuNuz)xLxz#F%nZR0^4A}K=R z0dJ7t0l(#Vp!e{_!PApx_b) zQ1mzs#VGnB2MW!d7Yhk4ljqdpJVT7)X|{z5P8|vryio=frh>ScAg@-JT8i!p7MjvD z$BbWY2U_q2+Hi?KxOhRMD+DFMRyLpvwodbDyA|CO127ds3aRx9u~ro^4jPY5Gs_3cn|@qh3A;-Dsf( z#-kH8bQlIQ4c!EdnwJZIX3dAyg&I-jYv+bN>A->|4-yAh;KFu* zId1FX0vPR!i`lU?m4EbVq+kpUT$B}p^PtMS#;jc8d%S(L#x zCM1)l12kBf)3GsOBthF4eshK>!8^O61ZRZ4vs@kk(J(K-m1SONi@HP!;}0#08l&c~ zP6xQ)cG|p*o~&12Z(JpN_Wg63@imlD=?37-$(w`zzfZFQ=I^4>V$Mat@dR zUTRWodQQND2*`(k(Nq>(u)7m3c(`F`nQ?M`hL&fG|Gk(y)c|}K8N5EQARY~>aCbCF z_&%w8rA@FEO=h?hv!M0Vo}#leIbc0$fs&b9&>C?)|A!T6(ly~#HOI*pwL;7*O(4w`&IqrzK5&kJwu)Q6j;6$Zy6fkgr z+Eau|l`qcim9sMF1UJipg8ue!f}dRLgKPbJ>EvY0S;876PqCiv0%^@QP@Z3W-g1c{ zhYIins(+(w&ByZ^G7o}~-6Px@G-xJY(E)~h#gQomz?tEt6Q2eLcs>n&>Qb*fyHTd~ zGq&5ccom1pU_IgFOpm}*<}MBRtR)BK(cm@#e?5v00S#zTjfNM?Pp%GmhD?C1Dw4s7 znPvg^cd)=IAQ2a}JJV2hXkSpZ!~%}buz<$+e+UbB!yF63P$(?mgaKHvgvN{2&A;YX zo`;K+_hnX)?18Na-qA!Bx?Ukm*$f<@_7trM4i{_WVjbQw&o08`fp-C@F<@Bz)v}6K$y0@t~diH^yZA|F<4mJ zkJ0C2Q^y&Yy_if8<9LtvP^A-344jUjrGD6`9k&<<> zWc!fXcI+nyM8YDDNWue>(w4t|)~aTARc2*YUkGVypBX<`>yU^zrFWEskD1F{PIrz>pmZ)-jc!(tILXwk6Ywi? zp3j|_*}Jutxh)=CZ-+zlCOdEl&7lXDl9Xt`MG`QOJ|q~hqbC^f2`Vt)w<#8If`cRpL82jGF0&ma;)g%^hDB%ZOD`o+p1#&3J;)n_g^%v6*3WuDBqX}Yiigk!?Za;eJ z^Bi--ZTX4MaNv*(FfEiPwx#k6bx+eVnYQ?d<{31i!j%e$up0+twoZ4L&MBnlyxKwQ z8g38IOyT9GERv8tS;5afe8kpJFWA)(pLiupeBx;cnxf~!(eX$pJ%q*di0OA3SdQ11 zm_Or;-W_vtjg6IY3Y4FN@Ko?f5-@zN6faC;H;IVNT?P#LdLUr%cmgohzK58KNYIdp zH7`<(Fij@iPN)2q=WgN_255i|3p8lBPSD7-`W~Q}?V1GzUfkbd1sccP7}2$cRtED6 zx=`Y!3oh>bPc%;%EjVEaS};9NfrUj-3~pifYoMSDbC0DsHJoVk;;^BSGM@!|$HGL~ zr#>*Ln8l;H<<0#^7b2MnY9)y;QNxOSCKnW{7EP&Sq$**=CXpqfgTz1{(|mgBl8~n^ zRcg3=;*u@_Fnp>yk zk$w%X71;xD%KaKTc(LD=UxS|q0QMn|eO^hE$3NHi-n8E5$OBL^wZ%fYUIbat#shoR z0mL3tU{P2J4Z1QJG;o#yjZk4xVq>4sOR#|-deA^GbRmPkZo9pH%(z{{fRxmp zB9VbdNq~WGD>8-+jL?=YGH|E<$lyn*IpgdSE0ZBB#IYjA087go1s&=aNu-ANkdX^f zU1Ww|i9`nTZ8K!xECU&O+9eQih%YNeMwr#$OqIcnwg3&mD-s%b>I52qS%d~>Mdh4P z449H4dj?Erb5glR#^l)sGh$Hh5MUlCN74K`Ox8K~4$mtcSu{skjJot=U`#o#_-xdk zB$tq8`}oAhf`(5r6f_pe{EDeExROCIkR>;h zm)oq3M-okAtKi}?T-s@!THv$?T)3cV6@U_k$>B0HRtA{vI>Gqr2v4r@QqAqzy#>Uu zu!YYvIA}lyNVOn?*X@k0GSOuyWYA@qUHawr{>CW=|MUjy>8<)>reltT<`*=H1Poe3 z#Nrv@H2DQRu2hdg2To9XI`FcKdkdp9-*e!yk85kRnCw!FucC)ttg!cUc0p@dP{Gr5 zLPdH*Eu|HrX>+<98B~5-TwQoIT3_8>FNS7AcUq)+LOiVglLw&CTuaPghrkRKtcjWS zz&MgYrzo2=+c`qN3($nh61|lSo;0^f9Mtw~(!&(ywgUx9Y}DWjvQWd5yGHy5Pwt$} z>2he)aJ`9*Q!wxKeKy?J`a@^_9C0pzVD>RxQ$-AoZ#?AyX0uU8)tIB2ILeH9;G0Ym zr13VHGlZ*H&XD8;_(JpnQeTKTbzC7F>T-qf!qEk;A2M#&eh&fac**=~JYR?!F877l zkc#(r*!%hxcQABHDYRBBC*cp=at6GEiv3-@s6<7W*iTP`V-fH-0^bW* z5J?#>K+hH~h$$I9Im8`PV1$;o3kRS}yMRGhwY_cwgEweTkv=k=DGwNoT<5?z!V9^; zs6U`ysU)*G3)*B{vAkG%#Wk(fH(RnO1W$GWaFZ9AH!|nka$6BHRfu_}NHcKv`zEfN?O`0#{OGL8?h{f!}$!z>mAg z0zDf$-NuFB6NwAl+v5s+NjWU)F@q*wUBLqQ!vplB3NCJ~Vexi-11-A38yQCAbaB7k zwZ7Y7yeJ+U6zns;)NItkBE&%x6Ow18(l^LJ0ccsEz~vPxjua~DB`9qhLfhYk3!Ku0 z3j(jD6p4%AIi3D;3X6#lE3Op-tqfe}z^G_;wqryXA=Jp26si)PU~C2xaqxY=`tAA8 ze}qXkPafiCPKMbCCk2gFc2e4a$M8r9=*9PJ*{(9yinJkooWe<>3A|DN_y{rAQ-! zr(vm)+)d#{ojEaF5F~utR=gD~CA^lp!Wmx5@>1}eL{BjlQ{aVD%YhqH!{GZ;mP&yGC#XHeK*)5bmV`-k zo@8dA#ahuBkDVhW)If5eLi6TAg}tALLGTL;D%jl#6}jqBFQ_o4r=jvH6pPj%?#opEwOinL&VLN!f;RSNb8ed3Y|9>*XRVtg#C3^zarH2)ek$B4AM ztFl@(^IAqJuFA^f57TnA^RHrGHzYFSC)I4!iZ&E?kBXKHRY^h_;QjK{w3|UB%+*NU z93sgHmZ5XI_$1)H%qQRlk55Qu9-mM-<$OYgMDj`#EnWBlk;)B~wf;&@7FE9P2*pK$3Vt+_EYMvUD$w=@D)>_wJ`HnT z5>)U@cBle9-ieB|$LojG2oSiDpul^$pfHsOSD|-7fgAM(#Sv%1YEYbP*4JJMns=Mk zUw)-^!#Xs9S8N!gwjB};85nS-3=G)U4-C?5;AfA(0)H(zBO?fLio^mA zd04=q+-Q(%+7gFKgW?B$kuX1?!qP~b6!uq8DBUIBg{gZk**^ZzI@D`3iej?DKa47Ju)ERCx&6*ck&epfmA)-FyPmoVZfmr z2r%p}5b*lF3E_yrH4TKx>-!5QH{GwC>@6SjkOy>d9?Q|*$u07+~oJ=;*%GKFtYm^WDJG~ z)l)kE0PRtR$hXoLfC$(~K?F~0jG{s=Ylxr&zcNJ7P-7t?*rgXltP#}X_4(rba^-oZ zFISsFjG`ni@MZ(W1%4r|3U+)Wae+I1CsqYd(qUD?vpzmbaX8yG#Kp5E_+q)Z{}&p> zh*%kpNh>h?U^Yas356jFLt=q+{%?p>fDgK&*zBG04k~`_2~9LsE;NHE-Ub%TX|kuw zFWwZBXUm(L)s~aD@bbfq2!3GVs@Wd{p-NTY1huEEGn56v&kg&5z1n_YS8hMB)ALPv z3C}gfF+*Zt?^yeRpGGI8kNucg65|-lQEnW^^y!6BL$2kE(pWhKui3S2qHyfn#%5HU zO5SWdxFwKLi3Q=1uA~hxQ&0aC50v&UBAd)NNh6&2Ks_7MgE#mkkLQh2RStgIIdc?e zmEctZ4voAUzpDBf<_|qvEk9s%91hysKXgs?Ul$jfw&1#3Ql0aOw8#4tFuFzi7zXLZ za=rQBMn6grV@S?~SYxengmK7F0t`)?;ENdPv`WB&5&pnl1utIuJBgnIn}e)UodfNd zrTQD}m2F<_TMOV$PVR4Rknm>%;&){3s37{|GDHLK4)K2R@tC-3@D_2UY!FURdx};` zyS})uVU?(ql;aDm}7^Nuz?jY4Xu;aAH?3VRtXO`$SR>ww!s)Ds9pN!W_^C{ zr0P7~z~@mgX$@0pY8xjbnY~NQhXb*D8>%3Po4;$jBxwWBFKGik+0!cJv;iBK(ne@b zdo}{$){&@#2Y-Z7!fSZ6fv3z(QeC3Pp$)!i&+*sl&>DxI{O;EuT;JSi8}c~P;QOWV z5OlbVhp=DLQnxI4W8*4=mEs6?_CniI+czVem z(7YLYphXSQforF>0cwLYkbBYxe%wVLKt&gQ;KAZ4GK&Y@?a_ywOJ0<;gZP8>JTgdJ zE|+(=cw9q;KE1ini?y`Yq>#wNG!YXKg>@GyCJRez!iCTh6`Id_M5p#dgyH4DI;Duf z8J?HZ0}(hSc6SZixLtQYUPkX~rWfE*Ttwh!xkX_TY*Z6!AZ`F6rc>6I)`)nqT7N?Q z^eblw=DE8PN69VW2XA76y&Mi0EWwm;i3zf9f{DDk9NwoVCfH7if&@z_NZ^D=Kmy>5 z{cS@+@Q5S{yoL)2v}6v6qR7CF`ihL{0qv-UM7?@C-pC+{pO&9iPCR#$rx%n+M$cI^JnA)ZEH#-E1*i5I#;82vD#ET}qqQhGV74!P6xs2PekyI%oO` za2?4)sSvzLZqfJ6Dch6 zRXC=w=NT;U;Tf>dK?-`YLH(w{C?Eq@?Lr3GrX>R#8MsP3Me+p@p6~^(75U--UrY{+ zu)fe%kWsOeqDIE#c)bKM++cxQJhVX@g_WNSq1ur=1|SQ?fz~K14uZkvo`?OBd6c|B z)2cm1!LhePG82O1VD3c1tO^S>Xt~}9!q*9;Ge8l(YD=X8L6%6h^#*QD+!RA$B=Lx% z9vXF4Su@!tFIRWxm*?wqr@+D)mRiU9q*{3Rsbrm=4L^!Bc#2k&km1*y22b5`bC0Yu z$@@r=iaL{^Gl7}P1x>|8$x|Uil^SN!hVWggN;!!;-A)tMi-fl)|HBDyVlL zY^<8uZqupVL<#vZ-Te0VrPHm{mND0W+f2#OFYO`~?LS~s0 zDm!H=v*)?^iiDAS6B5O$alny;go>ZiDo9A+-5YeY1zK!W$5SL#0N|cPwS5>Jvz=G6hEM2NBXx`H8RB7;7=s8QD z;+UaN0ZmJ2@Z>Ii3eZIe^^vFhmOKSf;5vpE!-b(HzX2`F&2Y<7aZ;k$$#gAMFE=C- z)-!50>O>D!ej_(64}=&>FWkF#hX)b@eG*Ng>9?N|PF@R%~Y0#VFX-U}2we!Dgcd20i~ohKB1( z>te1Mhn6&~ORhR}5vO`pS6g=ANxGmR`n`Tkje|k}1UK-YaY(NhM8Q~FtiZtq^^(n+ zq8cdF>a3sa*ysE(4UM^CEH&gKI zkt!7UOSMM$^lUWXvAWQJ%g5fg(I5cRn<(&6LV_>?&WfWT+$0Sf^OBQ zZK=ZbRpTmw8BOe>Umg1~W4Q>436f+A3jSK6f`&gcAh?b}69A)v1@(Q33v7N}4ayrh z1_Yg-umtU5^iDX~m?MKxXBQcK*}mS%A+5cpiOJj5`SQvw9jV<_Ki<`)Khar2fTBaC z01b_n2vE0R9Im9GrQ{|~P7D1KoHP@dc+Kp7p3`01lS53LnQCr@uL-+#fS3=wK1hWSA042*{1#WXSs z*h;9NgJonCpOI;p$c1IB0ERwR$jLe~O0E)EnNH!sI7uJ4y+%tc-k-vSoS1H>{tDHD z3l|!byks%???Aye%;E5n6KwH|DUJ(>oKWYA#-YJf8nHN1N# zZop6jVPmCzV7I;b8HVn3GC6$>dsNRKluYQr8MS37@i_eaYEzt z?9m}ruqm#9t1i_c$fJY~AR(f&-!5d~PZ znLAkkjVTtM`IzHNlCWS6{S+2dj}jJ~eV4%kO=`gcXi-$b6rU6pp_y&-foJPN1#TO= z+(reSKc3P%vdWh-Wpe-P(?EI5X!uB4VK((7I0^2&I!+zG0FO#j4~M1XgWs z!S2*<5ij_0&Mk}*yLiF7yLhqL1y{51!ba^Hqp4+f%d0Doua5H~M6Kv8MokJ6uh4gjXBwru&&YhUQAbUfrJhI~ z6jBPEmPSpmr>EM1wo6e1nG@D!Pvj2n=E)r#>XJMBb*p6nEhdAs+qJ|6tLa^hFGN?+ z(>wP38YbIE5>hb}fHI)FN6e(Emn z3*ln52vTRy;0a=%xW2~4_Fj*IY;FQP?Hi6*D;wdtmf>(3-16R)ZgL4RC_ba|M66U5#NT`BT9Nd$Tz>c1XU?a)63Qbsw2%O<5 z4RmD}AozW@%EAT&Z_u978xRDl0SMS;7Z43G^$;5P4)ItT5N9{*&$zy!GG6QP-IgGV zqcVPOGkd`S#2!P75)eA(%Cvon2y@8JCJ7x60ZWA3=|p|<8~&;@XV{GnCU9?$BuK0| zCYZCd6-*qFoHZsU$Y;I1#Db(78yUZBvIS!NERjxi5m28-+$bl@A-6V(NDGYiKr};- zQy6Vzg+K4#1#&Oe2K`Qk3*Kyv%l(d?4`R z2HrT=L*oTt%Y|u(%{a=mm|Mi|&$Ac7C}Xk0VY8uF0aaxLbGmk+qKAJwkkUynp$-1O z^WtPG+6ckku?WFWhXMqKF$2iE#ixZEntgqb(8$10{$m-1@*u4i!D>0*h*9v(2|_vC zcd}bKPe~IBs4^#%VAC8zd6-?;`kjJ6C`NKZLEBhx!R}7u!@CW_%dBFqmB!0!j1qfl z1q`9F(kfZst*P32x>{pYg~>O2`5Zm^M~SZ+Tpy`pY11>;2K{C@yWrBagCSpau&mDh zw4GgCv&*-ku53lB?c3mK<0;a&!68q_QqJc(mW>m;Upc$LPtZXfPo!1XmM-Y!rY z05>q4o8t|Jlf4+Kn?1m)x#maDL2IQW8gFLGsz1#}9sXeMVgiX!7Dyx`4w#OG22_Y( zF|_SDySP~FZ5t5c6bT5tdIAW%c1$s18KWE!EIYg%KrlM10Wo=gd*_8e&Mp_9HBUTf z4`;NmQ+n_Xg=2%3D(h4?VaW)lD4~kiReO?ISvaU>+F-?8bce7j)tRyk2Xy+QYKY z_?jfrCbUQAwc(A5^ww@q6s zhz_G7?2hCR+&keATq|-2x+|v;n$y_t%BJAwy=}^j6&{HYILTCifn)3&X=6n3X zl_x{*St&`ln!z8ymBk-;*-mLnWfgJMi$6G`odM<;E66xi@2rq}hriq5$MD5clY{ND z#c6}GTIS-HTLlb?-YL9H4;;7zE=wZ#M2iU)KD-B(5rGlB@>s9{wZpKo$BYAwm1p-i zejRTLl(kkUt-UFX`_rnKq=1K3UEG)$e%sC859S7qlb9PkytFo;%AVE+eU?%Id>HUt zYHV;P&)DEDUB(7a-eqifgVr8NV}l!b#suz+j&> z*P4x*BWUFlOjP+AGG;|){K^6ajeTMSIAq7`r$cu9)*V@jUh9Gdt`_^e25;Q1X#q5+ zcQqMf__fCr_LyB&=OH*;0e#jV7p#D;xnlAH=^kz+M1K(oH2&iY0-!O>l+Dun5jTEHB!BSR+`3$pac9LO23FHyTM4^VLe ztR0*}{HMMys8hlPCx|gS^vgE^3&!Wr--1#AWrI7(xsDo8bkfdn{f!VPqI!4rs76t)P0y9WG>eJg=93{;ALso#1i_Ry>+X z21TETBX`p0;gDeJ^W>}0aG)!5gwdOZ&!Yoh@O3J>OsUfY1baRY9;M6Y0Y;-kWBWXm zH1Ql29>Fos>tVXF5g-YdYtzeAgoKu|Z-nH40SXO`my0jnfZoX^%5M6lMT#@1=&G6M zXaz@>qt;LV_8z z;eY`NCM_#SNWP$b?1lu|%7TP434jFKI)wy3WhX#_&g_B&E*AT{e#*FA6N5mFDCn_kWx3U9g88#*?HYa6Am`0o$8TPJcWhh&^%nZNYR#wzc z8nhrVnHh zpkOSE;ARm(J?zcilxZ!RL|(2x7iQ>YHH}&lIV(A~O;2|BaHC7{i{{VTYr&3$Kd>d) z13NPEz&?XK_*2Op*po5`Ha+6Np)TIwuRC}{oFe5Z4tczRLpf{kQmw{Nq=93-Nn?+z z%xfr2ULwoz!)Ecxb7r3{vn2LMQ3qc!jf-KsOda1!q#o_CEVN(_Pku16km4&Or8d}2 zi2~nVx|!B3f?*j*8z9}Wv_XHWg-{RLm>=%qZW=3ZSDVFql#Ovp?LJ%I>s3@+=5U=` zHl?X9n{Lm2+kB=22sPmiysYF69PY^*cC$%kc-{aO z45#*VwRU9aQx3#j=<-!95(+{TZ(%~Afk+Mod|%Rn;0rDx0axmY1U!pE0xu0fV4DBUI${Fop z9o*s3mML=cKrdMWp>^*96VU$?jDYid+J~GGc^H9CBJ|1$Ef7z@Mf4_Y=>Q&=CAYp5)PeR&&&3Z;yf2dJW@pFG zqlbl@hMX{smKU4l(o3E1$UXAoaDai?$!I)>6PZLTIT1S&POO{ue>x}Po#VJ4@Sbxb zyo}iCa^H1VJ6ZhNKRPX_5HcPVMU+6;ebo zkUC~Ntj;5-WA2ITu!v4FJQTz=?w3KaT|pSBeur;aXva561KhZD8;%=O`&o zy1{$7C;@jE=}36Iu_$4+l*l#xP=ed#D1lqPx^XJyyo)TkjX=CEP;OHY@ugp3Ot zJih@OHrZI*urkb0+~8BlaC3TzBJm!*Z0;`aoi%>PO~VQ8kWB5vw!{o}2+X+4Q=sX# zOfp%G5{G0sqf*1gCK)`Vhn5jD`_SPQSjb3SWm_4E9S9amMGcmhYh=YbSh32Jv@v2= z!!p&yq=ZAx zeqCH$FIO99Eb(;l=OQcr+@his0IWC=U?QX9Yybxu)+(Ltrx_4YaVZvw4VFB~uz`CR zNlJVCvBXBG(U^jxp1E%qRRnOm&;jT^0v*BX+4U#vm1#H*6q!Mx1CQGu9b`0(j@KJp z-N<`ASzJv{SATu<7iBEYxM>lfU6dj=m>(l)Lzj(bg-MAGCLH&~2B&cN2n~A4L-oR&&@CKeF@&*lG&KpcUbCEHKHyDl5$aosoi}z>UEH3mMiln+ z<$h3b;KwOG3dDg!TQDgF;=m0&c(C_!c!a_WUKSg0h<5tpxEDI;<7jkXXnGx1*R$-( z>*ejOj-45g>+ z!^j!sEdPCydMBLil5VFlcODDfW{patT@2qATsYQSspp6^gOt5OBIXDU;;c8zyF2uf zAwf`%^-A$bM@Xb2(uyV$aKW~O3w8*&;N$rwXu%MT0t@bJ1kbQn0|k$q0|ou<;RJ_V zn1uGpk{7Vof(cLVbZS#8)B`26T{?~#pFGDf(bVYn#L4T{T+1?H{vf!2Je&vMCLgJ7H%yec~nV9aE3Sx!%>TdlS zw2;9sVCS*GAx}sB(hE79jjxe&x_rOBzgl|yQmQs_x!&BNnATrV;lG@{ddHFT2Yo^< zYO+ErZDVgYiloXbO^~n?c@kAnDTeYfIY>@Y?ljYXh<9{&p&$gze@|T`Yw%&hjx0Nfd`UG z1CBaC7Q@-0iSrfn(8fLSfNLu}kY-XqkXRfy7qILC1b^LnDmEZ+m3WGDX7DH;AlL?9 z5fGtdqXxv}CoCI|;a=dvnSxU=FkU`orC;)eX~v#u#f__*D)wEtNUu`_u9Z zE@a8p)%CTGj6CR(aU(K#V1boRJ<_f&k;Pc>?QPY~3AhkYIq(*p;80D z%6sZH%wP9ph^<2K$`(|xyHmcBg9k!|icd4l+x0!Fa@t^d^W{o=!>!)%O%8p_zGTo? z88&P4cWrUn)(;f^pzx9r8GK|t4GzvsyB6UQlEC+$e@k-7vMtfJvD20OXqc);Y-Cnh z1+I?Qn+rHIHW-W?fEEmeg76hxYa0&C#8V%9gMmGU)|-t7Kma0%BsJEy;`maA3EbTX znT3FCVj37T!*p4=4)!8wss`uMXmA}Yhki~GObf~>Fay-YI?BAqIg&mg1+3viZoYU>1R z?a>}^fdVTsM5m(QW)?N07&g#im)mp#Ayp{D>7jhk+VC(^%LX31id?H`Z+8 zr&sIi#l|)|sQA3x+_^K$%X2=sB!I~Z702Nn%d^j+7-@M}bT73$b8H7y@}#jg15yBt zVP2gcQZIaKn;w9w%K@e??65qzN<2k5zyO42dgyxPria-wkv^2^X%N}R^6at0GFCae zUfo^V==c?-eYYBe$I}W%K~t6q#a%-$mo|z$&}eE;l99+3l`9Q|;Td6inAv8q35Wi) z@xiX#_%NpD@yJ8Gul=M2aRl1U0`d4JEFQs4I?WGxI|8V^X<_laJ!|o+(i{mCB`B0< z^?eyGWnfbXUd&+-Mhp~zH0>W)BV!L-sv(lLLK?1N$yV97RIRvl;>zGsj|iO+b{)(i zPEpL^69?sXP8>AgvELPO;8-waeTZY9WqdSolvj0L-rnn3$gMZy!vUtjL<4T`fPwZ^ zf&mhR+c%zO*+j9-!EdRy=lRm%o&N$p!o$$d3%$r zf;n;dqm+B-_Or_%Xdg)+XoG}6&`?r~;&4v_!I`O2fEnSX6KjIaf15SIy*&cKp`1X# zDzVcaVN2!=1#2u!&hGBL5(Uo@7eSWDFdf?I-0~gMAsB3z83ZSorm5NQI<`1r2!G&o zK_1vPR4DeQ(E6Vj0Xbk{R}GWjm+!IW_XbEbMqb_^n@Eqmw35S};1K3Kn;qMZ?8csL zcmjjVnlv~zk+6o5ZwRB+lQ%**Q3g+$CAK~0pcv?o9-zH=iqxUt@`O3i5;0SmQQ4e2 zXymrR9jLr&fMW7y`FV9~V}fJ)*{Ts8X=K}{4m`ApFto8!wTdVPQb)DF!@!H64{z)i zU7`sDkC}V1IL}ob!7{e?Ux*Amx!8WR=m4-HgxY)hnq$vg|NZQKbGiQWPa7=9|E+9H z%O1YNQkVwQM#pmg*ylPaFj<%HBG{pROwAU-`D{r>GDrZaJqw2OLURV;Y_r(?N`2s1Ur%Le+f0re6?=L8cfNOuiZIPypklz79Am=@6`Wa{q5_TM2~rrKE>r-r z5d}6XcmoF%c-6$L;9AkF(0#~35lpv>3J^wrRB#@JMg@lC{`zZm^TE>_r&tMI>x~Z6 zvh7hh^ofnKEgKfBfrpM0b_KQujAAHL1{+h9QJUg=&n*j}=UEmU3Vmm#1lg7ad&j!7 z0rLZr+2==G2uP#lJPKABxtzi&%v=#cI)@=&7u)4?kE@(v`Js;IM}zAC;ADral7yqj(C3m0kwukSQQTr zcwP?<8V$Haf`#!01CuR7gw$@6se%~I2Q$0BeNtN$hQoMHPXhU@!XVzIz>*R2Zgn3Ou%4< zn*z0A8IuPhdH$?*Kce_(cJ09^=6j)K-aP&1^HY}Jd-&_g156H%m~WI}=0{YEaf?1% zr5rkxM<-_ZU8M0aysX3wPLP;+Kt>i&-!ftjclc2W?Q!Hu4?J{AN;G@emE#31{#{59 zJjXz=Sb_AntDE%(S2AK^r(d$Hpw_7^MbYtKrmzIM6&*BUgJ~Dd-?i{)MPCpcRE5HE zLC5qI9zb5&t56*5)bB)i;88rUg`d>d8izYLvT`|oTyNfc^H^WalF5b*5Sq%GEBgNV0)r-y8_l* z<{dB4OPLTXPN=N@K3bI^ln9QA?Rr2Td&eph80rYZ_MU#AteP_KTU(+DW=7t7v*wOB zt1SHSfQxZ+u|p$pEakzsL7SJooMxa#8#G)<8MMcNWPgXH5rw0JpqZgxA-FV3T8o*& z0=<)NrpczY!fJa=pdvI-Hgy4e$D%~YnD(9pl*}v(HTiA1S=<6}Mu?2wu%7?4T_5MW zK}of{!-)btU=pT-C&N3gLkv85I3d(LKPRwlh7%xhEKcyD4P%qJHCO+|VspM){I%F@ zoI344b0(cPa#xz9-A;(pOi-FCH)kzGDQH3wlSI-~N14OJOvEgn1~xGAggEk&(g8!O zp%P6JTqP9#Z8;$<*WpPUPASd(9+oAJNKjH2tq>sDBdk`LOK~D6WOM5jig~@dR>*i9 zjqvr-5gEK#+1PN-);EzP)4WCur)(PLW}hvtcm!wG08WX6#WvFI)ZlQ;Zy=N4YGavX zaM_RQz?v*K`F(MFvGP3R_2tc0v4S=gSOIU8vC<|dYCb}TZCRNxB}JGV7UnXr5~xjs zg~LiYTEdc6j)>)CUl%R-hJ;>ubXqD$vm@Q~5h#b2jP+M`eSYB=S}w*?8i}bpxY?hF z)^J1`8|+9-4Yts(qathkX~F`qF1&PNWU%Sk7##Y#ku}`FGch=nTbS^yO}>~iFgVuR zz#MUHh0LFMySQ9@L2DRF^)y@c`%%tJ&|oEJ1}BISf2Nv#6Z?YIVeku8j6y>cQ#Nd~@i6WWSSPf>`J$3H$my9t#XduzS~FqT zzX^4~x_H!qLtCH@+|LWj12`Ua*u@O#21IkA$fItko~5_B;+Dq5FIUUO#vzY)o3&0e z>j8<1*3`sAwT}mIKrzoJWz$VbKv9J?dW1DGOMKXFOQ?=1?Fq-gQ&h0ioD!Efv;|ZU zc1|KY9>qfiUoy8STT5i096}W7n;m@0ECo*V(QSG8_9AwNW zl_gz-nQbvf&-J_|U9h(^y1=($(FOW8H3y&xesL)A|5PHn{jVYL0p1c&Ez`AF|1FEi!tn<=`$6)RwHvMQBCuKq7=S zLK5!CXGYNm^+X8TQvrmzuiq3PxSt0Qz&8O1pc?VPaqk=-^d($;;1|8|amX31nm91} zvb@29;kbt(N4dVTa$;L>lyVCz787x7N^>N~g;6#ttoo7wqhiG#$sJ6W?1>D+sBo5E z3JqM$Gb*@qmrfv%0>?xoKc@+gGtdYHfombr!@{~&U0+s zf4Fpz@m9yvTU?ZKL879_EJ{gh`IRcH0vzzr_UM(0abG-#eA%4ymF)72SE7%y1 zih=NEqjpW=?12D>FdvRccHa z*^{KfKxR*Lussz#!UXTcZ!O>hJ-Y>b;Q2j#;07K(Xf<+tR7HT)E!B%MW)-I}Yk*9i zGr#cG#mF<1_En~i_3JHIBWJt&GNXh{JkBn zbS9!-GjGOA|HY%&!ZOHBf??Zcy_iTB@*8QQWLm6UA-_(4@d%059E6ffCT>rG5nB%~C(W3S)sX zc*^B$Y7N*PZx$b%2pLj@HvUSuen34~O}meohod^Vkt2J#M-xit#~4`^Y8*@93cO|7 z1otqMD+Rs`p6b;s!-JfSlbeKj~@w)W_t^thsk@me(I0 zlk;}5(H1AVzsvLW&Bf$+!F1Mk?}PgEXXrQmAFV}0BwRYY`TN2D1$t5+2W?ey3R~L!klt>yjiGU5_DUw8RZ;wRqjT`Qb&>D(E_@+Gr zBC|b)ATtg*J4cc-v<4p7pb@{wx|5{b%_53KzyxIyX(fk_yuns%l|!&<1G^@Xtc*n& zLL`K86DvjfAiy#mLg&R}kD_CP-x-1xpc=~_WVd1Lu}?=YN6Y6G)?W9jp`r-;2V1i` zXzB2SfmdO0@I(?T;WJD8V3s5^4GzAOtzu=CXBi$Lf!Tx$UBr}*g0%tEa=b8{>*0l? zv#zzl)eO8~?^wK$(ud(?ZpBWX-d?(KlVeoncSrsE1lRC-W?uv9s;?VELnpEM`yn5s z>4re#naRN|dYT-Znf5VaY5h$OdcDi!;FM@jY?C8SktPS7>6sj0(=a)r_a9+$=H{Zw zFPAr-Y(2ieJHOl}g`iagg#$~;c70*b=cKq?ugM)^ae4m^m# z96oX2MlNrlpAFtXlZ@t#!L(t{{f89A(5^SR?$_nTMvvn8v5{^?f~FT3!IxXcNIS5j zB}se0MG1I9;U-!61zoFFewpsDgqr}$bT;$+no!tT2@_@^=6ERlsS+7Ya%n&s}wDZQnA2! zfMcKF>k*^)8XU*VoBzi0?2ff5B@Ps4#$YD71B+@GjpN{A#>(Up7dmy7ohzgA=&_!_ zAf_ZR7}-q&Y6W26sXKwepSRs!KW5ynVSp~syPEVf9P->70yswohe2IrG(^=48O-<8 z$S5rNFnX|tqgEza#1)pKd}x5Me~4c-8#P<#RQFKERKfzk$zXx@>NQxyal;fAwW+p8 zYnU6>1q{*YHZJhE9xm`O9xk|SWsZv@0Nup}c(gBL?CqMNR?gt|Dj;$(5LB zPk7KcgpTWqgI~SNA9&v0{IO44uQ7pH+aH&Iu9jEM%({2p9RE=cE+thu z!ym6N`2$ZV2Yc}g(?oR#Mc9=CRx#Y$;nWjBAYrPX!5~wnho=}Lh0Jy*Er?Urn1*Gy z(>-b8NSN;DC5~`>#c*NpWsM~gresDe_3(1{4z2Otw<`ox-2t1QmRMfdV-i1iq#eH3 zrlNsBG!+eb5CqNNwG=^%B|L(+4SdqNEx<%i76A~`9tMlhM&O?(57dFq_DBT3?b09t z=O?GHetdQE{Ezw>HDicVB#GcqLL%6T9KtUwD~(uU5d69?gX|w#rG|c9{mV^D+^ltY ztTib<5ZX*34FtS~XHxLm5(uPxewO^$j731ud!vDX7E6JE&F>Kic*7hBG)(PGY~{QT z0K)u`L1Ar5&K4I}-qJzO*2rge0~m9%V|0DRsK9)iOw0YTA`AR?a$0qV#gfqjO; zKr1S4&>;={?h%;a=h~q#$Uf~Uk{s}Bj~qZ@IVRW=E-3h_JySsr<`m2t6qA!nWclC@ zM)dr2{eH!TLX@p|7JcVYERKeIohEr7ZL+& zC!0=+sW2J)EwSa{4u zOK7JIE#QW+Xc?S2bZ}^8FrTikmm7y%-Y#zZIzp~L>(24dxu7vGlq=2O57G0c@q_lL zDKzzE4MPKX7={LWwV}bK6A_9N9${zr>(+>bXB`qc$FDs*gICV&40gI^hA-OJ%$Oo{ za=IQ-6V;=QP7|Yv1GG>k4o|~0`?qCc<~v7dXUQ9$jbWjP6kg;F*r!?~r98nx#p!nH zz=nB$o&$^D8U8Ce``hux5eG4}D0#a?A>|vVs^OdUy%Hp?&r;N>eqK7LRT+ndFY?k4 z3<#t$6^>2)Wj5Lr8m{@XBVidJO}iJgp=B!0cm#d$*W>Afw#b?GVEV8el63h4K{whh=}Z^~&jR@jxJj#qPR6))ilT^th_ zeDqooxL^@;dQnxB5;9fI{l&$~o0+%zr~|Sck++U$3O-{&Pxuha5OK4T8f&;Me!^lA znxOV7C-P<$n7{{^LxrA?hm}yJ)dLIo!~hE((9$&zMyG4m?TWg`{-2Xtblhl^X6ey zOkL2_g2v79!g4AeURa#mB^O-Hzzh1^!V8(GD^|;-oPl_;W{Ur~`tZR4ODR*)kG**v zob_)be9`mGpS1=?U(^jBv@d%`q@m%x|A%YfhwewTF?KK)4^Fn~=uMM|h#v&tHQN|V z_7?80=A2)xmB`tr<^45oV7NoiP)*SbrIhO8xk+Q~$Z-7BoBro0Ve@y51^A~47Vr^m z4-5GAzNtS1a*UA{IogL$zzd_$9%ud#-S%yOVLG*nxxZfCxg)?v?U@)@yIgPXu%g;u zc^!a?0X&EJ^XO>>aBogncm_WbLd>54g-zj*#_c{rUEz6PACy*0$tv-pXK%G-RW*lF)Ev z`H2&TU}T;T{d&dltSkZCEk_Ezcn>Kh<%w%;7b)oWNHKkoa%e5i%$anzxRN0`PA*qh zD^GLum_Nm_bdw2aPgFNI=B1(`X*M2g4bYXC8d?HrY0&CDl_kiyz3y8o~JVfA` zJw$N2QLZ~E?_5OG?bNHz>eQT2H~AfD%D1?K!4-uGxoCJ@HnTe?0L?B`2gt5NbpXO9 zxLOA&`Z>yi;9hD^qC3#eT6gTQnT7P2R(no9QN6-q0BT3h`pISYA6KWZVpHw2(jyUbKvuZ&s4G z+Tl!M0hpPWM-qaa3NCoF94(}54=p7b0$6MpEI@6foE|b{wrk0dlY7(?ak%B=G8=*E zV{Y1ILbS-NXj2?}v&Z0^)E}YrcKBL@nW+~V6rnI)ucyIbw<(LmJEj(gSMx*&*r?0o z@Yk)vs10M>t`!HntUX1F5Zu5sIjm5To18mzIwleM9I?X~%1XLuXR0u!zL6hz{1}Mx7Cfzx@WNI0jsWa`a3i~~4 zgZFTyNocPu*%>c3Hax5BX zCxbSBccrS{>k*nlmtpc158g*y0oxOo#SV>vnL#&!rzQ@MaMh&Ik>aoQn)J zLw{sg$wI#a7pukft&NT2<)$!!vw)7a@KlD5_DC*T_gLdzqNC!CMD+BXE#f1FbtJKgLB0cG|*@nG;kdQ8pg;j2Q&pVLSx(h3?p+LCChM1B!&7p z<92OO_@%ctN4mST^d2@y#W^;JY#$qFh2G$pT4iM>KQFJ`B~`iNsotyrd6e-l%crvU1nz5d0Dxc z*=JxvQVG>pN+pb&D=NWDWmJOe7*wK@-62WCIz%a#gjThw1Q+e15~8r>37bk7zKiEb zE(sA?XBL*dlqRG45d}(})-t}=a1E9u0zx%7&es3!OsIHspDl;i0;8~urt6Wiq^0`$ z8z5BF<;}*!c&;=sc|FYjm_To&A3h<0M@bQZZue9ve%yr!{<;eh=_wKrcn{CE;3;#{ zQcLL5f1LdE$1neQr{KU>B|Y_VR%R6J8V-|_^^FrG+w$NYXBP^y0tiU6f)hlngVbS~ zmG(g78)Ze%$ZAiAUE#>7uq)GDZ!o5!7D@p|%M#1Erluqm0e9jZoCO&A&PpI+ryI5f zHy&$SDA*#a^sy~7s}#`$gS0nyUNs^kDcO&!cErZ`F^Hh5h6@@)z{M@h3tTB4QrN%= zYEKa@`#YpBffmXVBh8F0sFA`rS8e`Mq;Oa&-A+YF2(Nh}Bs5bNOT`fR39f=@x}r!Xl2PRQ~^&D(b}3PEoZh37o%?Gz86WQoYR0o59fHH z3+3S@xK|!t&4%j$*6dyEF zWJ)p9Y}C4hwkA=l9Ji8Ul>w+stl~0;SXG@1VG^#Mgi#4nr*0-L*(FTy##<96FhDU{ zW@nnsct5Y!8Q!F~GC8#7UmoAzI^(-9S(8S2ye%GzbHH6kwJk~$6-5IBa(MIigH#24 z6G-HXdZV8+L;yDiB5>{Wa|aU8?x|7@WNiB`xJwrx0_TmcfrwvwN|o{?&pZ?oW4uz` z-?LU{KD*m2K4YyAZwVsWlrL^@y+>kd2!EQ4!TQo3V=$yH;laQ|0uL4~@4*;oYy%%g zjGd9|6d~awv=M?cyAT3cK7uj|2;n5p93sr-a4Dm1s9w&@e4m0_+mJWQi>2etpicKk zchxdKw$3QN2?^fbfdsXnL`0N)0SFKFIjL6#3}u9210p93Qdf!xw4!H1@Z+zG2i(A8 z3}7*bgG?zL&lrFMT0z6%?62j`xi?wr>~eAAj?VgkAlaLMpvUF_f{uv-1U}k?BA5y! zDFPo)1_&U|UA7 z7$dXR5X*Kw51NZ2o_M}ln=TFw5bO_aBa6V#rf|BBOv^NMZ5cg5aRnlQ+LT{UV` z&x2QULL9t==ZD~!;fG-FSU&_mjlk&<-#9qnfS%@z$@9&{`qqZXyYl#1_W6c^WajtL*jxfIG&igHP~;7N6j4XnKkzKNuz$1Q(zMub;u?^>Xv~ z^7{Sq+*zvIkC|?*k`P|Ej2b+*4Vs7j+p;{>bSE3Y^>R_sp&4W>`-AgR0HuLbUSgR2 zK}(Ir3%tIeaKTSBT;8ldyQ@AJNNK|!r6dr1HfHZOy&ikFQK=NO&EF3(kEZF-pl3he z!Nb4pst>SSJ@qQiOuH2_6?;?kb-1WQ9R zbxKRoeMsRIhUOaRrZiDtjgpnYyBSsnFJ!UC17wO;2G22!HD>0Ts4tN_h-%#4%FDlO zu5ffbJ7XhC}lw0PAuY5J7YV?w2X6vAyQvZN2e;CYKU@k?vITU`IO zK-Fd#AhWjSv*oAN)#P;X<@VtOue_$Ii=VCjTwbltop5bQ>bN6*tPW&aB+gM(cxlwl zl!NQdhB>^$c)Juk#7d@i(5Xgrj$NIg1!{-WF19CkaIvUm>Zgp`wYDKqIC2N=mB<~^ zU^3)~Hp>MMXO%=aDaC{DAmF-3G7@t1EX8wrzkI)RrcS(F-079B9@uEy%y51@X?_XX zscgJFGQw*sa4AEDJE=X1V~vB#bj^2&kzQm(E0yAla)x@9T9jmRy8O74i|pr9$t#sNTH>&Y$UwXSiscN-ka>lR$Tcd6sMKB zoZ|WU6)M-*kg-Dltz%tEiH3MLd%0k4@R3MsgBGjXKd?8pY8G$o3(jx;?3kp7Gt^`tZh#xmZ~Y=IpP{>gI#z zAU#`R&Ba@liu72x;K?OiI`P6`bd!h}Rw^R~*B)YSkopR86XvF>_8oguF-by$hJ>CQ z9Db&r!FfozYx__*($wN&N9|byYGZ-JN*%+HbKq2<05)zuxVh5)3If~Fvv`ehEQ24G zF@qC^UGEQ)UNiNNPH)MwswvQ8BR8tCey*)TRaWV9l~$a|Fgww+LZLKh5`>|2;hfJLGaO*{ zK*L^fQ3LyH7$LOSXd{Ft83-FjzcSc#j0<3S;N|Ad85cl6OIK(dID`ZYNyvn`C27jj zH=DDAYz4HBfXsnA34>j&0E1mQURcG+!wU;Jxp=|V47|X8jl~OxHiqHlXcrLEcqz{u zHe93DP4+V;neuDW0%r{=q0zX;DEp@UWIYc9C!l$g99nZ82JYQ2DvJ}-sBHNB?376; z)h!G{v%j3EEDm|TGV5vP&axZ}QbE}UhCPC^sPBsFYfSujwYUUHe8B@7;WjSSr~IE5 z6Kw*L>7W=z?%D7zz z>hbt`mtElq*Gj@Cbbzw>3A67bA1FuwV0eOFdG^yIb5XuG>%TlL^7QKcYG44R!>Zte z6G{ZHFRe=WmXeu$?u_(?iMFLfraNe6X;Zv9@?jQ;vMG2m!=~`_cM{+@E)aUonlg-I zhD`x9$J!KHu!y17{q{qzv)x_1lID-I^=7lglwn-V;Ez&4tUhygIE-Xkmrw=5;gUV_ zsN@pXy{bDSe4Grb*I#K9ePz00^LIxiHWUp>`Wm5FJ!vO2txYEA$1XA<6}85re$2St zNhbKUXJ7c`Cn2HeX z4+~DqPGC`Snu)>%-ao?yn$eIQxN|Bi2@6~}fYmp;!-|$Izu(0QH z5iBk$05Y^wFB>xp_{wNwb@gd^d$)0B1edgE2^sjVO}aqCIH=&Jig$k*Gcal)f#=FEWlD?yAHXdtwt-2zYwP?}_MfGep4uc# zQkUX{^CeSt0w(a(37UHgY87wjsa4|2Q74>Xl*^NfbQbAC2^2hUZ-FvnRWEIOegaQ? zv2jt#W%a|R6=&C(-#rby1Bz*0dw^CT0R;o@c+*6#nsHOKhoM)Ex#mPviU}r>dZ54y zb%6p1i#=XHXWXv+6>*AmVJK)EP|$J9K~Y?$1Xt^giG9Ae#>Ct6^UdP?*>I zE2$~#(o#kN8XQSBp;0R8JI@^%W?dxMsE8g34r=j?Fz_n_90Zo+3#M)gY+&2N2BFmj z4*tA#UTtvj8*fjM;6Mj?;Na*<4vr8ia2W%a>y3`NIlb$}dV@)!j(^K~B?=jj%`4(d z4oY~&E1D>4_HQeUy0=3$FQuxN!^#BO6lP&Cw?T2LGys-#Go7h}L!s-wiSQt>BZ&H_ z)HxUZ&|Gq|SZ$mT>f6~yqf20fCZXA|rzWM~ybP@_-jY!EL{OmFYfh{r^1eZ#U1Q>)VvMGaL(~}~& zaTkH`b8cnEaQ;BMc>ICq%ISj+qDvn*)|);KtlYA*yER1W2DdPRt3P3qpDR%dps1Lb zYoelJ;)w%`8BJe?%*31v0?(dp; z%ulZ_5dN|`#F}4K4x)B|1y~SZ!FCxeZ5&O@T>Mf|(PMQ;mYF*9(hM0o(P#k$VIZ>_k9T21AR_8?a_f47++=mtZF(9uY{7#4?L&-N@1`=djM2(6+V83N0( zI4qBzTbxR|v*SG(RO|#i)5N-;}uxEBk@;}^o}R7X>1`OPXUCMa(zIYlh6e02(7>N2Upx#8_VmO8&fEePaFr2Qz^`&B@KP-ij7P%3 z0iA-(ro3C+tsH}Ly133l9^Y1(a$Kq>sFeL+FAk`pR>6+}sY#PkIm7!Q(Kvw@?Fb$; zw^1F4A8P`Vi6H7>O%4vtSn7)f7HP#wB)FOp4taiYF*Liq zQ-|=ANXKQcU6vtu({BQq{T;5WE)e6bi3PEk(h>0jZJFbxAU|-(MN4R+OnqRlg%+ON zr9JR)1Hr;IATny<>=%Ds&V@TEb9BNPK0<*NG+G%e^bbw2v^L zh508&g5rSz7SKoz7P7fVF2(RW-q|1*wB1;^F!_HNUXHA(5Xbj-S6-kM5w*+VI<*b4 z>%ayp_=J9s-PdbrDa(iwp~-E7?TI8WDKrD!LbUr-1g%`%5D zT4{1fdmw^LF$R9A_9PsFgR1n|Y$sHnN`tLa24D;psjO;NaYkUu8_eWOx6@%fhR|KJ zLv>>i3GihR2_;RJ-Qhx45l%f+%Hggx0T1)GSH9u-iHjr+=Mt}%(LxTBpdloW@Jo+0 zQa5dqYK9&EFSNqri8)*&cb;lxGsOUPeuqD-!m&D7$=U( z;hfy8%(2(S1Yfu3Y#CqzZ>KRad3OJ4apl#~Tz%AY>b|YZ3)V2*+kHy8K+?N#M8))x08_uC%*0}8$g;Er}e zeehJhxD#&BhrKWd(dbDR92$r@5X;xXoG#p2%^2*~#B_GI{JhxQ0l!9Ug)6e%WZ}3r zEb9mBD~{QjJzN>3hZ-xj&#-i}QJXB<*F?XB>q?tNHqVR}K7Pw+sV=t0{(>^F=le2x z)@8Nu0?{JX4;r^?$A@=td?7;1F5?rB1-*&j8LvQaoe?I~I<0IY|}Li6>a4}2INebDN8bM^4d z-_~Sl`j{S=s}-DFEw7h1uum3$oUT!)Q@K69k?9P}GHf9nWwd!2BTVQpG14AzL4uFR z2MI2#qzwv>XlXN@?K#2xiegivg@QL%IHdw;;VK?r$QOAyA1yZ4-q7ofTw{ow3^8U7 z+sWeQc6sh3&c9yYY9-uSn?AKi{CRurk=WslrP*hB>Sp7??BHEfv-1$!OdMx3c^>#Q z%nnoK)6Rz32@_d_w-me&$L^pjyX+3`9F2-?cfc<36d8jH$UXh#6z_*JMz~fiW<*ry zcBkTKd<26sJbd4t-qQ4tR#n@bHz?w}cFfMN3sj)UX5zFc@ihsIzexr7?85#W7-%sG zi}ni*Elf2b=IvoerbKK=V6Q?1_M~{grWcXLp)H_+H*i!44tXjBhYT13rJ38V2Ek6Am;UUTm-)iMMd$$!4+op!7zIhVry4##Sk|Dz6+Zb=9K9?N9Fj~=MHx;{Mgs`*^H|e7QVsQYkEFo3A~5{iG4ho zL`3+KBO)mL1WVvr86I#E0}p6$Wi;^Hlq5KS?BM~O-Gv8SKX$f_2Rv9jMQR8& zWdt=>Oiov~?lPXo$Tm_o#P?Y{0;6~h;jjWCWrx*`o8c`KxRiH_6V#q!Py=C5e3l}v zvG2L7%&<+Fhirx)=gumJ=DpwsHCO00YZ49G#c)>9dX}?_?k8f3nauncBY=81t5|CU z510YvCF*2;@XAHN73mF>&fo@bFR((Gm%&0aYjRP0Ao?^48-vk8-6GIZ9e9~%Q%)z; zc$pqD13ib#?h*HUpebR7#E}O$@R(yUBS!#xVTM_m8D>!X|7x|kz6AmeE%N(%os9we z{L&2@^rV0dFi}R0>-@9_T-2ba6pj*(!9f)h+uLF2L!f55>y_iJj%WeZpxJRD{f4{rm#uIa}=!o(iX^J;@ohIw=Vj{M-{PoK2VumWp%xBMh{hMKc6M z_85to&G;Q^c${pOw|_b4_z|nN{^f>EegB4YV@+sK87Q&QfT7Bf5#!dO?eew6}dRNWdJ zzb@USiQcWR7p^TSK*MDV7Z#vI2?~7QDJC=`0GSLB*k>3J>{WOm+!8oo^AXkq&l!8V zMs3`#Nr5+L@gjKwhg_bppg>%0AsG>d0}ZZ$fOx6rz975fvmU~4Il?y^l}{0SK1Iw8 z!@+urDIOjcbV^3q7*5R~0UkCW!I_r|6V`Wl!UqFuubF5Fa^F!Cdz`4F0D(OTBEqjFCU~a|6lg{R6vUHM znR?L&cAAsAj6od_I>3aEvICGaIgRM2nCVNvuvfSKMTabQL>pd{S+boF9& z3Ko7aV_>ipy(c3X`~cE|L#>~p2WZeq3R@~{!Co9NQ}-ZWs&01Zlzw19JB1g6r9fPb z-Zaj|Ir8atItPZvGW44jEXCJr1itVKD};gFgiJA5${Ej*S^5ON4!E+HrjW^hVVDic z8D5&Zv9QqAu!65&VkLZGN!`PCX}hnDmFW(1V?@x4Gn$Mf_fXvtX+bCh3NzEsCYy%~DI~OIqh=&p!Gf?sfK4BX7AU@fjT3KKdggyVl_=_h>PA)gz zO7F1ydZE zt6*Fv2Mn7xM-208Jj5K*jCqK`F#|E!JC;}GXpcdNL5Uf|0zLnP6?lul`V znNkfiw2-M^ffc1N8o|o6^ycr{o57B{?a`iR%&rXowv+C#BNMLJX9!pRG_e;5cVN%= z;?1B36Wxjji@KEr0}X;NEA}yqmn2)Y_-Vr9e9@k+?fk%6736rc{_NzB7R#r)o=lrQ ztUq;Q(5&hGP?@dSsG$%hNycO#dcG12z*`0eG@k(mw4VZl9zj7XKEjK^=M`O)`XS?X z?Xm>xd2;&d$5$uM|Bzk`o-RQHm?5G8@0_E7V%UWPU$rM3P-7WqXUg>X?fLq>HxmvE zg?w6Rc6hKKqsxMzBg@i*y(63%Jhw?T?K~IPnBY>rG2wmWEKFz*N`=&z9C)L$%p*!+ z=4#Yp>IZHYI?(cF@O*|X!QQd9qz>{mf%P8V%z?R7$mDc=v0)v5XMpABZ9n@G;s?HF*GN6;s^GQMGRRv zl20GR%sF9xei?4}TnS=qp3Y z)2r(R#`HHfQp_Zv4zLWgICzf^sCaaQ-d1d+kbx7_o+4KEcbG>aG|JSg@g6cx6kPD* z94_>NJ-pBcxEdvVQd!a%UTLhw!GjC}%;7HQ;bnk1Tiu*rVHqMDFqqo^Nv8!p;N{Ko z&pRVV(g7E|un;KSa6vbi)Us{3%y#fS3AE4|GMI&uNmC0RCdUfk;6Vkqa*-06Xq!mE zYYasS*i$3r_thQx+Y%>5ces>U*q;d+DS{?#{H4r91!sYVX#)4h$qr=2>##Q-}gNmsF2;YV@ z6lI{{4aVq87?)N80ho}pZ1>q>LG_SX6+#{_RY3DOgL z$4XHAG=irWPnhLW&+b;s_qI8Cw(#Vq7gg=hpWsnpNizLO?n`5;tQKlg4@f=icU@bw z&Jc)Ty;NLMpk#m736L>;QX>VDGDk`&L4>I!;=Rihtz9J8J%G{*Sn9-N{q&e;SYRgxKW*wh+z+bWO^%Q z&)XeqZV$Lv#y3>iSdIe;lB>KRb>tQ-c*w*U*=F(O9}LW5;J&zr-QIV>s` zbcw8>BsChQH!k+gl}vtjS1~!>e920bJStX)7ZDZ)pq2)PZ<=Z)PE#@g1}U^vtu))A z$&&U3yx2B3ieuNg(r8Jm7!K;ddRUeZB-^`E6mZd z`NJ;zN}cAc7v`*;s6M5yy}UCpc;4=wx(T(pF5 zDvQ-%?^v`9PQ0ETnoij9;_Bk_$}u>vVU6^BzsG1-;6pGAXi?$?=n!xr*P3v7WSS)J zBS#o9YJ?sacnOsX4Q{E}kQrW_A{7OCc_^aaI?#en$US05?X|<2T8JUMy7+~kUNlhc zU44ZzJ){QE1oP~EbGdM$z^4m0yjL!bqEI)1*H*UGbJ_TA`(^0iYav8hNWMqXAE{Tz1B>17Cu$qYxysv?g zkcTNT64u;EK@u#xtz_{{c435{Z3jj|(oB1f1W8~u+rbH0rn!`5s>PNtf-7l^{Dz3l z{liu$~Yi9NFemULD21a`%2gaNnA%RhG{69g)lrzMoX<-rC3>h|UghkmTGAccl zv|w5Qw=QV(p$!^?7Cw?T@G2fOXj5}&@WXLgV+b<~5u{bcGEoTosg><3Ozdx{o? z19U+MKK*Y&}o(xAr<74Gw?QJjR;w*t&Y^g zhU`~Xt((b@Fd}ZqBcB}lD3sul1xnB+Wt7nWG+~04-YQJ21^566116mCsBr?mtLZHT zMo=;iP`e>WE!Q&xhO z)`i0+iFskoJ!xOeNjzv~!@w{yo^~(A2F^@P4DRNM4IJt+F}QxTAN6C#?b=HLxb?0l z4~7Nr?wJ_0Wv(?CwvQ~KJX(-X0AUZ0cD`>ea`pVn#_?#+&Vdz$h{jhL;k2M(1#{rF z%FNMbe`Knqkuwom%AO3gFo><=X{67>LQMvDR0Lv*t}uZI`*QXuE^o!F3`+>X^Nh7K zbKEULs1JMWv(SsyBX1Yymu?+jvtCjg|8OUtbApS>A8qf)PHre81bLc`+RU?868?Co z9FtVr16%jhr@(32zhIAVKKUH*?Z+N?);I$Y05RBuWYubnN3h3%<*7h=++Tfk_~WM~ z^6_o{7&)A)DI^mgpm`xtIdE#C3oX1=@v*l<7gsRIeBVow#8ajSBZJ^;%L$~EBw4JY zg|{|@Djv`x5Ijw1w1W#kMHKZQkb?t82Q-1up}9j1Ehpab6=nhAvlm7OEmJl+E@FVS zts(~KK+6lnpr)q-nJ^NYIL(9%Gu(2_P%4;fL5YoXat) zVVE(8bAQ=ftZ%$R1#cH0K3rv`3R_Ug zrUz)Ulq;G$Wd_(DPG3lsD~Q5VuISw@C|5j!XM6A{o^rJdK^K?0!JG7zt_MdP?$B)V zc6op0F3*a^O}Gq>GCw_#(O3p8I-C7DDsV)K5$s5`2)6K^i4?((3=i06;DJ9)zIwm` zdr~xD(?bIeb)kX3ZX=@gGsf-Oj}fQH0T>+e(11g^65*w`AVi=;K-S1zY?n{)ni4F0r8K$yn=Pw1+hHI?fCbCy0Y3&>m=~+Tf?c`4LeJ8}%7|H0 zk71S(SV4Pa)N;D|`)|j=eYd{2u)*R`OUI$tyT7?B5ox zRI?oomAR7wqNDY`1uC%0h=~x7KniGTD5)5db#iv|1-TzKt5DC{NbzLKU@L?MD1(Jx zyNMREei<$x1!b3tBfOYmtcc}s?hp%;TQsrQR*2m1F$>r3bFB~>#)yPG0xxt7h5`l; zubBlCXRg;?4dCMqG6fW3dU(5D7w0|Pcs3;E=I=W8j&l<)hr~b9-9;0pUCZc4Z!d?4 z>&OubWLVA-5~n=Q-NnnqJ|7=Q93+D;e3HM>ATXS5>*!mqbI zMS_Bw*b9P?mGUSEql+#s>Tjpl(rB1lXHdlCc(b_p6X_thiQ)X7VE9gP#N)&&+Fp1z z=z3{WutNxt7D^gq+g2d69U@bR)tNWFE)<;Tl7nWm#-JNbQ zkP=bU!wo*%<9DNJ1PP{Y(ClQhSh-XDi#+0~2E3P24GoL1U_tY@J)TR`Agl`Bre_cp zFi;diwU`Fa@L-RKqLP1FdZ-^VZr60d8?>iL2En6v1cC?8`2z@y9Y5Zz01p;@6!U|& z9Tq_s3i3cxNbfi3_Pc9deW(i{uWkNQe~`0U~lompS?_ci@=I z9i)TpW*ri-Z6w|xkGA4XR-@)8tU-YVHcwV}Utq3ThbEM+sjJw8dN-)~ySTc2I2iUA zl_hq~Aj|Iyq!MiKY=-5+MBkM&fnqzZGg`R}K6H@@J~XCcH3hYS;5*tm1RDzQRc<|U zTs!49`p!UZ4!$i%S#Vp+aA9|+;R2}@v6odLsW{O~8?I*yctQ?R;5pxn822kp&;vMB zA2-%&&cw`S7p;<^fSE^;_A$L8U@^5{oKx%>FosvUG%)+5 zHBYt$6Q@WEhPU-B7(?&52`f&-!qoy3_DE=~szbcEh6V=v?pAKX0M`>Avv#!W9JTB8 zMdQ=Jfy=Trf7j5!hn;#xfR)4rcI2=yQ30$dQ1H7?+@kQE*VTE z18YPic;G_05)bIpo_N5S3J-LDQa}KV9!K<`2+`X%MF`ADK;Q-*Am}#b6v6o1RuC~~ z4xOfm$=Us1e|ht2j5PSi`N(LxVKNsDX(|o8iLyeEQyRNFoOC3WhBr|YUucT2kbl;I zH&wN~pfh0thXS+rA0|f2jg=X=S;UVaakb1q6+nD1IZ&3HPpfl>KZfPUcMRnK@B$?O zfz+en^GX)|Wc>^iCTO*-!eq9?Y2yMWhaOCr4X1I^8od-4A#mmZg>`ORqyUJ9H$?4Z zK?N}AR-|m%k~@@PWyVr)8CHHp9eSpm;cAux<9W|8?|S}s2UPe{X~qe*B~;KD!f)m( z$|h9M&|8JdJYOGn?yV(ktq>b47^GOCQBAkgbb<$)*?!>y8+bjjT~{c=)hwu>(Ym1m zEDwXqA@e~sRNmchZahcVTKKb7BytR)TKjUgsYyFeaR zJ=~B5Q@WeKYo``1lMu&4CPtF(j34x54zzsQwO|hXHf0Xn%3}`v_TR=Fc&wZ`fYblW z-n%xpktAD!_j8-w|A4vsrDwN|%LMo|)94y0vGgdtjAT`9@7h`!M508rC_)30s#g8_ zIVTd48Sdd8nMgwFzGGbR2 zH1tuHy@;bji31DuqqPj}cc@k+O#& z1)VRDg5S~}!2w~9K%tx33fC!R^)N~SoeOUlQZ`UxfCN37(+NT6unAt|l1b>D3?eu> zluX#0>v*|92s%7N$m_+;-G>Egz<;$F#SfpY@90vmrj9Q7P$DE0H%)V)oA|3%JDbMH z&K7$|z=ZEMaAIwVf?Exs;PX7BFpB9yC3I6?sNg}JZtnVQGG=l43@h*Es270cn>JFO z&p$7&=XZ3TO8Jd$faDN^#8e*p7(Y+bx z6L<_w*8Lx%BkcI_$D`=s5Hs=bj*T60Wh!2JJ7`N!cLy)-a(B?wu0Tl`3Na&A21;n! zw3VCjtF1FZe>V4biiwSAOQ5eFv$~TpT}X`xY&3o6h0M6MqdSRL$W)`TY+`~wu7?Ru z?ni*E=KBHQ1QT3Xt}ubG?}-WalZ>5_F?wPGFYdwwn%ad40UL=43MK&sezl-rp23=+ z0Q%Mdh2hzp-QT$z6j|l*dW1wmX&Dm~EcoewV#1D*prC7zfFcy+7x{~v4bDJ;J~RTV z1e64Y-jf~}@!}#qhypF?0tK4TULrw(HhY+0C*+tY&h4VPYhc1q9Y4>pX`fqRYw93Q;zTJTYS|$Mjgd}*Vp`QfFa5_T+ju~jcQH2IRI?4RNPY(<@)x{9J`U?jUoBHCwQJjcL<-X$JrsF+lf!UE=o2MauO1F)c`YN4Y@0xGxf?l5_$IpIds(Jd+t19 z@yFZBi(xcXS(UfQjMB66gJ@tGo+^zfGakkU5IiSOo3MF{ns|t~Tq`2B%XpLa zOh(w5^>%hMy}7f2^Ln~Q5zUU^I){XSjYbVeM;Z@fK8}f+c-#dIF7QEvlL|D5hbhqP zd8@5hp)92&cOr&01l)ia222?17Fux501FyuVxV3uf@79pB1H|Y6C9(BX&(Zy z!&(hP!j5s^l&5_-iI{62Sx)2u_Uf*c7W8CsgKT7ckzubIVf6+Y{sccaN-`0_Ib{bC zlYD0xE>}nl65=dCqltQP0YXniD18AW=zUegShF+N(T*;Aqz59<*cLR>P6ai2h~PKo zh+v`WmKhWxfUo}c2qTo5FrH8EF0MQmrkE+_V*(8}ApuqDAcA8-QiQ}qt+*5O6sLQl z0avD~0BFF&Qsb23ban;7sSSXEFK8G4l15V6Ogy@W)??e-Fq2=mK|B>=q)7qom0#ewH5VCY3pRIna2`Yd`v zb6dgJ4Q+qdKqwJKFn8=->sS&$M+Xa+xJ-d>=?@PpIqdB7>}rV@7^>qu%RFv0hdW)u zc_x!3niH+z7;(UCviNr$8OMPH5%_+E2LlcmCk7hGFw%)7*b*y(qj)$?;PI{KZ7A>s z>(c}Pp94YIl8PqqS-o*^z!Vd$F@BtTIf5k%q8Qeoqikb=4R9D@hm$K34cv}BfkU;* zkV*80Sr-IvHxR)&C-0QNoQ!RUPs3(h6F{IC`-xXqWw z#R%y;*q;!uFc@M$7FcJ9gWs6Cs*qPXz!9;eXW`@VmT;6`imz2ilJEvGAj2ELr3h|N z8b;bin8r&uhhA#KXQ@|87^lB`CxHRjhL?(d8tSEn0PH#ejh$Tv5_71`u6?UB*Cwkh zFbq?@j-?YMp+ruK5*{eS#NRIhw<4u$ME+*2D`C)oHy>Zff&irV) zYksigm8o**ZYd#0B{OD_J^ol_rlSiIjT4p*42@*q#Y*2$FA9C*s~b{OM{zO=qERKb z4FdeOmq=FyrxI<0U&+KgUY=_kCabyHhCSKW4B6Y^qck8!uP-jK&ja3JsGB3~p{q;P zt+-<|wLI{}WfH+SGU0_A6}l8*P>^q5zk79Ze)8+;39Wj;$@T|!dpdetZRce$O#)#= zMJXFvGg)pr3CT(H!l@uLDaU2?kJxlGFR!iz^!>&wvqZ z{GcY{$}s8FH$bW|N*LLYNWqvE0d$5dO{mZSEn@|b^0C7ANV{Zriz_09x~WiJ7d>q# zoZ7oqh#8OVd6(@lALHQ$Epc&!Z!~a&JBH$hQ=97o$Iv&gzs{DoHf~NAsHuvza|$?( z%j*Zm&@@xh@px;1MlFLg@y}Y};6S2p@T(^B81%q__-99MAa6~i3s0Ai-k8IShqaod zZe#nq)-2)@scdj6Q8xJ1DjWRBbqycYkcD!1aW0^*$CT~fE{AK*A@hX;g{RZJ5YUQ1 zydt_C^Kbk-VXkWW;HReZZ)k8&Oew0sgVd3tt%_>77^{LjyqZyls=Ma_BYJEflU6%e&jV#b@x{+p`zvFW$ac3I4I~ zu0%sK*B#?+09%5Ax#NlY$jbpS-jpd27zjpuj9OhP5RzxALGbed(J|iL;jW+w?Ilu! zuq~bnu@i~8RL{i(KelJy5B6KfyEP_8r;A&6W6PI|Y;Gk+ItGM1AY-XQ(DrIoyLUh$s3tN}ov2Q?r@$6uG1Q%94y zGcDgKNE+dz%?`*{!AkK1E>K5`jIp}~1WTjCiF%<2Q)H1f8hw}21P-P@(;2xavhNZl z9l#n;L>eSN%-&ynRietIQS@G$Ob#tyG(pKXQbkvW2mKReZWz`EMpQ4>AR{FauP_j8 z)__pErdKG64r4=HB3T2cJa+}y%~?YjnTj;(KI*kd0ixBbcm*N0FJ);w+{$V)4yK@|bwl)^1 zG&ymuxANFw2Cn9XSF`#3t&Ib#GHGK<6qmQffQ2T7hAmoOGXj9ilj7es7&vmn)KwsZ z%m=}@38JN|8x1dK%h@M4u2rxsAvxQ! z^i@P|S>oCR3^Yomia~Adio3uQg-7OdMJE&J{ zgO>C8hcI~>PI?=(*!6slm-qfA>)1<5qpCbxzvf&U;8Koocx;seRCy=3KpiRiE4W;E zD}cE%L$Aonjd1X3PDTe#1q-3SuL39yjc^DQ_K0v)6DH(q-WmNkn|`=<)-XL=&h&<2 z9n-O$2_5Jn5lsh4l|kZ#S)<+4nl}m(c#}F(1j%k*343zjxV2$@cQRQ+tl&e8Xc&6j z14~GV_Db49#~Sf4Ky)ZtxSvo=5{+deH&#w--e~PA# zv-#!a)JqvG)NOISS~qozl`$NXg{2}QE^1!}?4DwQv@Rx8aK5McN*GBj&@@gW9AQii z#sXUQ04$(!i8F(bYIAx(jX4$=3Tg>pI2OjJEs$e@wMJ)-7Wr|GqApV}#!;|eBgJcG zD8PX=9wi(|d?_9PC5Z?0ONIw@t`X@#Co4SYDf-|d*nq%my2ycW>L3TfAJU%zoDx9b zSuG#{+Z+(AUAGZH>};cBG&zi3FJ@QQ1Jz-`^pcu9VzG|p#AydFN5-LT>CA=jg z6ZE%1CU|iwXc`(Akg0aTv%I(|9=QQBAptaeBgq8s_Q=Fe_u(QFmrZ~z^i!`0`uM;~ z_c@vR`}AOE$$6G`-o#Feu&A24k(r8NjVlKjyNCE52@G8uEHA;qoS>e_;36gkjcOZE zOR+)4>#SZGAMW7onmR`3SBp=d-JP*t-rr(} zNo9dVH8Vutpb5f@r8qkD4KQOeP_sPF)i;#q8cSGqwm5Mu0v==5hqgi<8fsKGV@KUo z#YG7<6*k;26bxQ2e%HG?JwV~Go=HnK%}f17a6k?CR-@+0^>pb?sGeEYez*~B7GOUzyk$s$6T4P=AsLVx}SRaE|ZD50n3vqtWUSVwdK@4z%)g_2+I_r zl7=aVYr5c#CZI_kO3JX7gKK=xTlNzHD_i!Chx;&pvACa3lP8L4SsCO8&okpVWmg{ zH>x8=+8A#!Uq;Zzp0}H|Rn-MwQe!t|30g?$@02x2d_i{BG(Hu`i)`yADA%E^!8L>} ziu$m|E|(r@(_{3~-U4Vj$#zFNbb+Grm zO=+z?zHoC66xv@NP&k9)0tK)$fPy=Q0!1!|>ID>Q$G~Tcn?I)4-h$d!_w(B!t}S3- z%D5`UQ8`+rp3zZ{kZ2)iZ_$I9uE8-w*Kmw0WwQ_XrrL&)Oi$gQaT`!K9NmpZlIkX~ znq82RF^OE?>;Y3PRj8N;UesUV9HPfGgFL^#aW`zL>&?9yfDGnp*vMe~DxAsfk6pG; zB7-FY5)-7NLyYfDd|GA?WXM1t8^{1!6)-rsQed#z9xxb@=|To?Z^T}=v;AGefY56$ zk;uT;dxi)bnj?cV5-u{(mfpzNVaP&- zP4U5k59#mJ%b_V0)Pm*ZK%E$PL3a)1mR;PtCb#UO5gA^vXdl(WZt*Z90BHy5z z`O2LX1EHDr4sP5iV5K}26$c*FU+~n zctPkD2}NI=nEdJVUN3-XCTrXz)h6t&RS4%rl!T)ROB#uP*VDVec4E!qW0fsyn6l_% z!<1EAfPfmBky^62%`;?is>_b$-HnlI+p*#jX~&`|o*fJAn47V*cUq|{2Pg7XtN?-p zN4Mr}a1S&X&KI9xt6tBYWmZRpQc^vp?wVb_GF(9KIk;dTN8$n+D!~P9h@rp$0m(_-X~Khgl$dTimKoI5)e8j&l<>6jI&|V4hnb4DomhWi>6` z7>~l@064|(2j>hi8q{)2^HQHg_I9mpJC3Ixo$SFOJ&6O4>{2PDoeq_PLDF6#RSNnmArA0o zo8?Xk=gh$^;aj;Yoq`YR%N%=qR&B!36Rg|C3k>2gHmrRZb>KsU_k!P&I;cI<$h)hO zaG}E1!>9u(?hu)vj-6e{7GwMP+8vN7cx_jeSMR*r8_?f7JZJ<|h16 zF`_03gwn_e1Ov_>5G7y`2*X4vf#5#R@<7wN2m}vn*`aP_`@3cfG*%x%dG`=&Cle+N z0$8*RLK8k`5F)k}zhV&F(U(Dh3S6yGadLApUEWMDki(AG7(DXw-rxAz&o*TFO%oYR z&{l&#Vxij0fT1=LByuD!@U#pTxX6G7EP-^~0~X9UN@0O39smn`a4T~g7Q|hndZb=K zQ#@3lSve?Zmbsw7xivt6?$V$bVdeGw7CS?ER_WR7?n)8HO1Yr#briKT$N_#6ngv2F zYZgk*L|1d7rJaYV7KGQsfK39a6+vkR#&Bf{u@L5f^m5%o`_YJtehn$1R0P}9~`^;O5`z>~h!Y2#J;4{jQaVKe-fXb#Iv$sXH zD6El#iDQjaOc(Bl8O}g@m|=J`jCd=U3RG?uGizr805gzJhMCj*o3CDR%d=Ugbt13i z1k>U3qKgGqFo+~paDgyf*;EffOP*|idc*k!o(uu#Aiu7!fOcqR(THMdb%8==)L%5bHw z?3tjre`sw`_~Z1mJIPxDg>SQBG?|Eih#f#MEGY2+F(h!HL`9MaM38|2LN&mED-{@U zB*g+n5VhV@_M?Q-{>_l4-15# z#X_Ipx6~&%AoPg`EL^96^)O>~vc(vYP$%A`KL&FOFLy=`c=n7AoN{drV0yDQ2fnLC zP%llwkdTJT57_4k4>G)5%bVivj+^mny^F_8{DHo9#zAe*`Q+Sat8daAP1nr^pcbf9Sb1q{L2YF&Ujt(_G zWrU<3WdL^>TAojrOIAR%dE@EqTW=YLZtj4KW4560HmY6rwzvpfXcxvt477yBeF+)4 z+9kwRoMML5P+o`~1Hk|bjt&LOy6WX%n*&8-<(I|c(uT>)DfmXQN~2(k3;H2`(H$$OT#Dyb)_r-?SdnzlEH1BQHm?St&+jJJtc!q%#{ql zJVeR(WmKPB&%Vr1K?oQKwNjtFIRES8N8!&p&VNtk8*yG$NF6NBpx!6)sI&9sH7}3l8)|6>m-00;uAn6VnAhJyhY;KMYlP zcaAD_(-2hUB@eAk_*aWB_wGvG^F^TmX0Z+t_`w7NMu-OnU{s<3-<_ZV6eQ7r(}U5# zM{f)blut7UQ@64GUC+r7Zp|gVr)luu91Wn>O`u_HZwB)NwmkL>ljrx@mbNQ1)>#$N zIZH#hr+b;=ciAw(1tzTG@m6&`%f7oMFo0*3&@s$*o#PBe=k4(eyzGb17mg)K{ zV+JmFKx+kIperO{@Ke)-AA_Msf$;UIPfJoag>}Hp4$v{uu}n= zlcxrLMpV_Ugy|cd?z~!|0M&SmuaVS20!pZZ zStGR~k9@%$?&9Aa`=;aJiaYe23wq`O_hdkSBXixu6WD5es>Kf9X!SoX-C4DU+h@i3ax^@(~2MBmVg>N5tO~yMzRs z>y3m%rif^6_+k0=lQ(sLzMOvXSMitTWGhaenrNuV*N$zmOQ=djFoz~#h$^p?%f)RO zMPR5KK@XZ$iX)gdEewp%*>=4t+`Rz{10Up(1hWP`mY@xgvqZJ2G39c+R?|ewqhw?pTIE1SZ>Ar<;z`Do+>@*6 z68l&xt>WXP(}86~ML>n?y(CsRj#G=Ysm89?M>Ho~Clbi*nm^ZsNvW!)mS|4f(H0o4>VkLDnLVaGZ-dhl`4k8l{s3dF+8jkS4@G~46HnW zVWg7k$1o68hLy9c``K5UVthU1fu$qWuoEf(vmh9DsRu;1>H8C-*K&oxD zK}k3ou{+$wH~y%7oUnu?2mu{?_IS0yMiMTgsnOG*#g(au5ig_EQ+t+b&pbNJe^OSRZ}PP zn<0q6*f0k?gBBhh@ZAy)I8YDC(7nttt@}A500-iq9RU!k|3ok3!P5d3aHceXT?Nq4-H&f zrQp!0@){2e4oN^TikTq-U1IpVT;8G}p-Zv@ULHT-<_%y1t!-hWrr_`f4;0K3&QVdZ zVvUOn0BH?e7!2|A{Ez>6KfA^&3@7GoL2Hy&R=SmKofY-VDZ8n8jKyg$iTV& z$Y2ykGsd$y{8zldz=gFYs!0g)Bj{lx8&iF-WT69#38f@qp~|r{S5~5e=3RygJjOsp zn7WfHgeXhxiMp_g9UnuxJ5iBJ+y(`De*h?0`YZC9^UqdIuJJ+|)_JA}mJ}bdf6(%e1iRB20QIU;t<-FxYUHFVMU$WC*d_ z$SBYSl)Oh5T;!M|g9Xi8WRM+u8yNHpG%`-HlbE~C2^0DLmOurk3S?gr0#)Ih*^O}se4YV9DmVTMS zcT>H-p;0mJQWsajG;0-F`9|Idvu^Q69mSwVNYqPMRUp>K;qx;F0TvAg!O>K-@LO?j zN*^QuCx(GmbkPSc>7y^pmz*-kc|~S*fS|W@GuV`o6#?P zfumj^?)B6QclL1sf~WNc$PPWrOuHb9{G-1jYW~sDF6eq)?4u(kwNJ6vFJ+y-4z()!$>Xin(sU8VCl{(3!tDs%O@ggXWr-|Qnoey-DHSvbl zkWq>$VRTPh{4_81>wI8yAV5w7N{cx_^v!MK0Yk_kkgz4K)Gky2ZHRnT z>L3Y`qf;~XK$jZqfupH>Hysqk<6B~=ZRqxQtx3csQn>_>+WR)FkbrM)WgW^KB|mH1 zCT5VXMIF{M{L$%b<}7;JyjcpI2=>5l$sSarDS0$uRPFuXLIo@}?}`i5k>d3I-WFpX zf;`4v2|7;Ir10|T1MpAjM3|*G7f9clw+DI*`oJey^nt^j^dYoQANJU{9NZBXE4eXr zV~N~`S1h1g%5cF2HV&ZW5v?zzC}2wl7|Dwg8~d^VI=Xx+8SsRy&{RII3~?UE;pr2`jieEL;|MjG#? zXp~em?`Za{K=h)!x zF3-2m6bwhX;YnY}U9V8$iSP~gE^67}@=`G+eVSzRH;i^6xB zC+I;zDG*VaQrwJ^=ESCqlgSn*LIhwcW|2USUKc#|)K$hZ#;m>Z^=Jim42219T^A*=7&J=GW`B2M_NViU ztGSM+G!Akpd9_I&(PY46a^<~5@3`VJl=!2b66Wg4S^(%)ta_`5VG^|`O=Z3@$ieW9 z!<0&*IOrUc2=H%@Du{Symv2mYG=6VrZ;Rg@x6s9Ai*Yz+n%^ z7~BE6(Wv-ke(j>-5mC! zZfHytGtiE%{U!c>0vhCKDgeCQAqjk;!h&Fk`Xdv9fYf^;0aq#{;7H02_~}K?aH_o(M!ziS&Ege?CHfN8*d0fAkR1vr2s@O4?6Oti za!hl=HAK;roZh1`Ix`ee~;b9|PlX-!MBzr_+z_UddKL)Q1Mk&|<+O_+6%v zh#zgK;6{ZjMH#q29Vv$2v&hGToD;O#b53x|WsT4~o7|R8D4<@PVb0u5-!EtWs?xREI^8seX|5sEJR7inY_>d>*9AdQ2H>AzZqjNUEhvs=gTIJvyceARM5 zW79Kskr}= zUe@x9p;!C@+^^?1K*iz?sUV^1wl2VZ{`32p6nl$L!?$?2g@ZBqJ&hiy`-eiaSQ&j_(IN`kh7OWktFV z7;((Na=u)AdVlTJoO!*$YNFpAAC8_0m1r$(3M$x&2(lA8>~PO< zn;HGnUNcdVV}%R(J&t0G)5QuproYFGmvkn?%8~*GOKGf(Cxn;A%GnjN8Esm5nWddH zc5l&F_Q-&*S6)rDcJsA3fJtKc@>Xi>6GCBUmtHNb*YHRvXxYB;fHq9qd4rc}+|7Aw>Ww6KDMffd%s z&{(19b43%jWp!qNS5uTWSRkj(f`v0*Yl6i}ar|X^jbNOOl=H=nf@g zN89WY8Bz7rftFBTh$ELPT+SX>+%7;W9O~OlrBa5B7xJ1TpJkXGP>h*QAVCqP z?psjwEb|UY8<@x;LA{Ye0t&}7Ljbc4AR_Q%tOPF+f&TK$5Ykf)2zE_N3d#<_NC|du zkHGzS-%Oc5U0h=N@7~*T`3Fq%=<5F+5th!xB6NXJFAztEdWl%lp*U(+1u|uO;7)a< zXpQV`G4WLD7d*nyFI-`sG7rIvTh~-CQGV9cAFa~f>u>O&PCtfOh%J&Do#xN{O}5EK znJsek;U2}Vyf{9xj2yri6UH5eh(-!{Ln4J?Wnqc1QjN(+5ki}?M3B%LhH^4lyzkW} zp^2igf_COu;aK&#`d|s; zCCOLB*GsSv{vtqv27f24>_RCCR@w6w42K0{NnRUy#kO0SfbW6@o#BE7{a}CvEgT9K z8hLAig(<%oSe{RR{}`gSR_OKB^lrJ(%llTU$N)>H7Sc`*a}UdSY3jghHPoEg6uAH# zWP&s$x;k_V#B)WVrkC-)oryBuD;0zw-dh(A^NGOApZC7L#PU)0fcUzU{)UlX>(zTNv~LGvmsQ*jt-Ck0&-} zwu{+I_&%3uK(Gd6&`b+4_@Yk4&=aLKbu005S-ySqpMRj!sKblD|MXlF3)ksBfA`Dj@^!&S-zmTq`%{gZzLKJ zQ)&es3f(6^hu6Z{W)(sWi`O$`#@^bfC{*1m)q!pVuMtt)5sQ>4A)yNXhLYG46Xpg2 zc#Z)Ij0gpaeLyS)1;)Dv3Q)-gKmibXps0N?dz}T(<*49ngNq8B3-r_@oHE&Fgju5k zR_E2?%Y6FL2FBUN;?B)*D=ol*6ij`9PgMFKLV-n{U*G<8uA*zW zVGTHdnl&6ys^oI<#R<4_v9R`M8r|V9vZ&l0huRHDa?P zFoC-)Ot7yKx0Qi-ATUJ*>E9{RiE+09RB&WBz>%oHDGwDanw-O;+78WifDz4@K0FbM z*J)t9n*Qz<27f*EIidgyiqqtXQVAPlKSeH_O<}>(I1&~t^_jr}#|&5)r8zkpp2C9J zA094%*$vw0u5`+`jSd>WYBkT4GVs2-+J+3;j9M6v+12@Z5}UYH?C1pOdsMBCKDH2vhKiQ z!eEz#1uFt1jB$VqBq}&AGE_jg24V1CsYbYVF$G27WqUdor@BDFyBi%*x3c|R>kM&; zR3i*_C8)r!1QjMg4?7*VQB1m>0=G2f&D15do`q4xC(~Fv?P>g+M#* zuaTTaFu0T@{hgNP^NWz0DR2e>Smw^xJWT|Sj$AV{h_6M5Q z1rM~XmAwrQ4x9EGi4UA|@qrJ@@gdWc6+Y^Y>ZfYkmqW!@P zC7U%krKNO3M8Ymb9Dmf2jbJyn*+XHM0tE}mD}x34#DGOG;w3J)EIj26{Cof|gv@PR zp#MEsfCM~P(3sC*!N$9^fwrIpyaKddll`%k(4ShWF#aY{R|Q44hs<(QILj`co$yv06g^I2>}^N6Of6E2dJ1F5A@zQ zfrq^Vvbx5@^ZT1Sk0#C+%lTZ{qYY8`#s%JD!T~~XD1uW^5)J6U1PqYZP%Lo3NOFJx zQY=tZPLvCR?$Q}T&^9-SOC%O>%EJO$nL}YiNZ8+DuuLQ2=jk>4Qm-1~OYB55TYryb zw@!1Dot)bd74se?9GK^j;DK{!5)c4TMh>{gAO{+#5)h=}o``_Jx)6cqx2nKKgdh-! z2#k>g5%{XsUu7TUkdQfo%7{R7(SlyuVt;QJZ_}9gadG?6n{`L5JgZ>WutU*|z=xOw zfd)7Pk(&@4#jORA5d^+G!vl^PctB$m9{A`)S>QI0AaJS+4!pY~ejzTAa6sb{IN-t- z4rpbLhG3sIr7j>sjxr$)hBl+1N@T`z7|R+=tljUy8Zg}~;R zb!{XXSo=SL17|KI9?S_c^p?R3j1x&fFf^DV0)zA05(CZDhC3uNpeY_AxYjm@MA55& zl>6$5gKg>|jR~%fbmzxTmUE}(13F%BQPvsIq|cNttNx}LefZRM0-C#9oNkf<&Hb?# z(6m;(jd$&ozAB{pc|lF|Nsylv)1;orW19G|p)pNrhL&A=#551KIpL5&=4|@uvs>iy zcy{wQrBGH-aaKrn;s#9+xWQMH$Abfb!)Z598v5_IZ*P1fP#`oYEr(3+|mJnGx38obX_HF&qDYB;8{ zG(O0$r3Yn#;)ec;#*BK;;iJMO`F(!j*&vJi4~jV&G!#t`&jN7)!SGfI5VclK0D-R` z3<&gU3JA*UpD=2H(USuL1H%Obz_kV-(4(3fM#t07m;$@QI}Bp@_JtWBh_V{tiel5W z#iuK#QCb5Rs3S!^0^Jfa&WxFdXF}Cag&CEDv;pVR@i|LlHxpvxu0cR2lzx`r_voXD{BqdHkn8 zjqu+i^*cTE>zkw3Cy###g%IOq^)Wn%rOh6P|Nj~aoE_avuV!=1x9~)Ls6HDWa&+@y zc73}#8IIr)?_M4K_2l&NqZM|mZEF+8XUiFi>^Y=YC>h~L?HOV-amV>JMO0DiarQs{ zU7JJzZ{mQHKfn=*3Iu3+sz5wCDlvhM39gz@Z!xgEHr35 z>ChtHEsrEu)V{X3M7rNN<=I3S;JFHvc_7Lt!o&LNz{-rvy`J7)AsB0;;VFW417dSc zd_Hr3AVLQY`C^xLAF0sk++{%G0iI7!JfL3{9>SzR5}^wN-cAld7k2@IcYhN=nBf?? z>6rpV-Alc~_SJ#4;{VO;i#Lz+?rQFq`SzWqNC6rQe7OS#l+n`UKyxHvhkQE`&_#v@ z$i;}xkyvDOE<8Lz14pl8A@@K7@9sbY0NGw5=>ffx&;tj58@H$3loJF^w^myzJm7J? z3F3h1;M(j#-Ciu8oZa2wC5Dw;ocAyJOwHC(8W+uyhdhLTfZmWgBkcIm&gJAJfu8OO zi<~3?lvHU@Xx|hS0yq*C5KMv!0Sa(xj)^+a-dWLr+t8Np42Of#a%63HP^kwzpY^v*+>;=p*xFahBjRKcE7Q~^n(pupQbP~dC2 z6bDCc2UQe6fm0qR$QuSK(0G?A_^mxBd{LhOovKkWdJYF^ev4NaJn?S!d857&-Yjud@q_(p-llb9^)V;R15SZ$NQi#!HI7@iT( z>3K%9y@TAW06yMpct&{IP|t`2+ajokFFoGp?2!SL8#jKBIUlE}kXLxQGk_oZAz74< zuPCF1_Mu5d&qB;1A6)yTz;6}xOJDGT-~oxgsKxNl)PXTd_oy3jds$OOmm`DTZ%~VE_EcK z7MxUz9wu9ypcP;l`vp`94hy%I2PMG31v3DHF=7E`o70n>kQsm&i(p<( zKY44Y{6gIxzihnZ2Rg!J7f5HL>{4xUyK4a$&UqW`Vu8iP3ZLh3OMzPOECVSxY9gfv zPG~wc*-CvB(SfyD;^^IMc`^Soy~fiFs|6L`^%BgOb&#hYbf5$%6}*BEki5dhZ!(;V zOTG(CP}H3&Lxgj^qV8N5s#yhxa#Ep#=OKj-x2uJKFas($Iut65Caj4S#uu~}dOE+w z4kYs%XNmu_slQyeR0)K+lctobCIPn9AJNm<qp6S)8wrI*k{Ag+Z_o3=7j=0);r0&K2aRaGBjpbVyf#aA!6R*ygzM0A z-$!oyuA(G$SM`dTjSo1>qOoywbBQg{yoGjvM!vIR4~Mv7r#sk)DkR6oUcQ(+)Jbl` zBvrm_*Q~<~IU3lo7R>-zsaK5t^ss@>>%xYRdktZY196FjMo5Tpkb!3A$Y4~dr3Zx! zbaNo89;^I;47^(->VN;s3FV)qaqFBzkvBvZ;* z*?Q@m3r2ZH8R!!O7EEbSoDs@PB%Bd=*p9N}<}O&^wJi*6Scpp`Ebsy^SV9KQVNuai z+6*qNgy=KG&^*34_hHdz1twSh*!ZAx<=|t7v?lREIHd5P{*W4lp|-y8K%*2qxXL|c zjW7;vctEgS@ZjAY3WY{)dyT{gfbQXgb@vT?Y=`zbF;;YDf8OBG)V#rpLEc2no5hmb zC%PflVllcVEn2&)ERN%Vqhh|P1V`&h21w9&RoF0~fj1b;0pTiSgt@-+rYGkaDVYhmQ*}3Ao_5gbU@EW-Q?|O-+UC z6cs6K-~x4|h?Z)1i`quGxL#FmMgTQjKp{C?c%cWE5Kr+Eme6vkED>fkNo7MzHZw#p>=2IJMwQg9Y=76PMO-4a4ZbWfL(qGzUrTk6s~c7pYneiMFC@O6b9$4?OnQ8M zak=;%+q>f>mj5#KLhoKEwKM*Xz7Z6HkyoY=n&~EoG$%Hta(1`4hp6C?eW1yRrXDJg zg}Isr!zpJloRZ5Wh%in!IyZ#N+|2AHXyijy zZ^}0wZ^auK1ZU!(b!-d=68gX|$sRb6kq3?$a`o%URsf>tfjDASWx!CJNJm_8%) zNQlT9ilkzt$rNEhli*lxjk0jLr%71V&ebGA<7Tz^adfCAp>5ZqsE;Puwcv7gHUI3- z`Y)$*cdb^bf`q$Mvav@;q>WHjH~v_y-vi?$DupJi)G09JG7dq%7&?WrkvfHWo;{Tc zhIp4!!6jWvg#v4AR>>g&sdhM>z{!{J;d5HyE3JY%dTW(EYYy%g?4&!Ne!9gw40H5! z_ThuCR{X?6cl`q7;HVYGSS2DTvt)_|{U!l{KF$CE5;g#VD-|HnZz&$|Y%d&!Q(brv z!mit>g-^@C!2?ctMh1Ey#{->Y7Z3c>zW&KRmArOYM=$R$JuBno?6=?S;KpF*1nm_{ z1-)HXD$y>{ItA!az)}hX-K>rj;~RThG;@SXp*}M-N~p4#&Y30L z4T7_RVYttTZh(b`>Jyq&Erj~$6Eo@X`2yT=bBkvg)PeFNGX<8$9xbVpY8#j^ZBnji zQ8JE%G0Epzm?E0cN7!*v*rd$p&iDgeZSe<(RU%?|DV(}V8KfSS_FPL33L(cg{9sdv zxJ0THsyB~A>~g3q*4ZLJDHS}eKZ$ToOOptRVg_%;fdK3D~Oy}0$LqB!&BBLXdU5W!q>i3k8VAqt$X$6w?IjQ#j$ zvjr3$ZvcYbBv}GSQao@7561%<@&G*GZVwOOryLI(GcyjDo2CKra&hJ7t7E2C=c_lW zqqG8w!_U+l=ywMV3~5R)9fGwfikh#Nhf zUEjO>;7_Sx5b7vK$8;KiAk0mCmNYjp6vT8IH`z2Hg}zmgQr0CdP)CaPCU!&)N<3NQ zvhNiDU=mwOG$Dk6_tm&fO-wpf>2G0j0{Vf~Et8X!kQb_gp0!L)^bz@^SbSe@<AG91=TQ!V+nfg=pN7pE^tIUq_Tz%(M z8+ug$245lRMFWO85GK8JPqkstt-~*STkIeKm_xT19`ju@g26EZG9)c0!6Rgecr*4k z!Jq>z$n+r?t_@lfGG;o^&(j;Ow#BBUZfW*a%*@8lV7B47NQ=|pcNsjmz{E~G?nc*f z!8*{vaK{F`T{NsUd8IuYxXli#lNfUMCaewe=4$bx0K^_S>N`8u#tuI& zXZLO;e@xcuUDX>@xO4VQfG`o!tnqJ$dj!FlyBfwHb@U??=4gZJ44WxLU|jS>gi;a= zis5l7A}Fxo{w{`FEAG0L?eALr1w;VriNA|ywNtfhrhy3D(K3PxiPinoYdy=}j+K3U zvb?>UI!PQ)r^~svL>fdC;$i#_6BAW4|B{1ZH!lUF)6#b{o>*g{6K!Bc6r3zgVZlsi z4;O%W7gGqK+qgi-XsA3|p#i57T%aW_E|?ya>E-ujAzD%*V&(M<&gp8ifFFv~I zq%Ws9wP8CWBr2qa1`m`i4YGy}v}nJ!I8&Ado~@1)-C&rh=p?=KE!KS9XLAF>X6$^^>A!}8Z1>fX%f&e#Qa=ZK*~^$wM#dt}DOs|HlfPWvYEEH--&sae_x{Q^8^(Ut7J``3hpn zFu|kLk)o-xvqd-yGNCVKSSnntnxZTiDrinlB}_4KsRUpM^0Sl=`o*9UJk6pK9PXqN zntbb032xC;^6Ywge~Tv>@p_{aNW&tf1QG6gYf6}D$4bJHH;kfo^iyg9gi*fiw1EMb646ByKAm{f4Ufu1nnt%=J4qk@l4YzzGSXM_Qom7^fE z)mfX5bA3tSU_vFSQ7}6DjGPZgQJme)vQ)xNPwk>j(nsOvrgjB`5GJr_FDhgyD*_j& zBOP`Hod#i7RAX-?b`=sw6OaYl0{7>(1=p5&wncHOj!GiW1;EpW+7`?^ZW*PAV|%b; zhR!e#=SI%Xm$NV4?h(V{AP}Q)Yw^1b8P^ouG-4)O4D<`c>^m#>X+tWspfx#Ih(KqS z4kR81yJdz_f8JwQj|+p%-0b1*>UwrJzX0Wg^WJ_B6DynX z?DKr~$J{B_e0H(8yYtjm(K`N@mP9Mxs<;dPjGLRHZ3H<2O@c5T0S&;>r{; z&|dVnQ!R%3JS_&SbQ!nwGaAdaZe{zs)41gXrf1FAna@m~YRN~LGZ;pF6KZiuG!Ht+1VvOxZ8SBa0C&VcYs0n~;-P;bFNFpJ47kMr1CA;%;9)5i zaDj&foa({?@9w~YxI`MZXo?2~+?882fQ}0U+|U;YWn$hoe~lut_+y1=n5 zV0NR09FD{;y;XE?>HwnFv_%pK=!_BZ% z&|4`nz-*;goti3W^8-v3-rZrU(7tFck){gWt2QQ!ft|!;@r{P9B2a_D%{Y*UJ` z5O|6BO(}!ckK;WbI69O-2*)Pjv5&?%G@LoX(d$6Jl1%vW5X zjwA*xPAY>IM+BeXw_%pxs3sE}%3;E%Ib^~|c%Bc=8J-W04kZ)Zx)zxLBOtpBBd-?- z!!2z#F)WddkqOA9OeXFg0{HL^Yn+g9mYS|ihcX~yWf7oHbBu(YgdLDDj_#ra#AKiZ zePy8pjqLPtm_)KBOh}j+OwOlYuNMwXje+mmHkc{RMC=dfDIO>r}aI8Hx!FvsyIBe1jClLC&I6>nzPD*phFaG#L zhf%uy8o;-waHAuG4~c))J`Fy>(0HU*gfA(|S5IxH6UMzb2EhE%zjN`5Fe# zP|xc9+*T@9LVbgSVr>C8tGQ~Zk=vxughZ4Aa``D?a!3%_nK1>n6bFI4k~j$CLk2$hO|7nY z06xY{B-HpgzD9B8rflt+%0kRa!HxC1;F^n`^Kd}k+Cq&Dh>6}snI?VNpHPCwi` zWdu(2nu>l(rzyXiDIKH!a?~?BEDRWb)Fw%=7(%=<=$eXkM3mNnd^4?s-ZQiggNvz% zp>62b_S8BUy*q-OToPtzv}9BDD^C#?v}HmunqvW#yV9VUD1$*4btG9Sw}p z(*@RIVaJ)Tc#UC_dRH&|nd&rxap@F0Ra3gfrNl z#^E_vBS_Y7L9X8&z(( zQdWuOx(Y)kUJXc~Cxs{_L6?_GBC%V;khDEs+`It>!BaG;#1(r50%0?A0%6az!dLzb zz9gWqkH5P|$3NqbC#+?Jj8rEca=y^}Wj@4g4Fe;IpVE#k(?^p(q?J^2qJ@_-I?$Wy zNMg4V0?&Us|Ml%p%paifv9m=hQkWrQI@<<=jIH`tld#)*&LZWsBEZ_E5m2W=BeW=n z@(9bhv{>pxBzsn3{qX{8Jm(+0J-kn+zfEU4=H8G=F~ym*WwJ?OXXbIvCah?eu*nV? zUnYt`u*2yDf=THFAa)}p-E_jc8y~lBW&69+AOVg&dxXQ?pc8UaGtIB)1XR2!q2m}ycgewX>T zB4O95B+^2I1UKhY!lCaPB=~ffNkTVexgz+ap+tfQwjk=mBL~cz$>Xupp0&3U<4IMF!V z^0(&lsAcGN#J|^H9ZWh%3|I=>Q#rV_%Yua{TNP6`v;AGG9BPS1D@h+1U+SqGCWPfG zhXIXN{7U7ZxdDiMEZ76qwAH8>VV}&a+3&Y_hXIT8zbRDs=4%IiKz@Qg&~Ql~q05po zN1Qn^k%h6hQC+iRO#OShh9P`SA0~TnTFPS&7PN8M1NT{0>vv?2-HN+9Xq>>7n7b1< zQ-4WbpZ(uiIMWn$)wstwi|o=$jg(S{(wOz1ut>#)dXh%y_GB~ynOdPV$Tbx;9bsyZ zlt{wJv^`JJRYI3kn_kPn&k2bKelHiZFkHSiW(oIci52R z&RMj1gmas=UC{HBy#m-_HiJvCW*c>|AQAiRnw%cSRG@%jRELjA@D7r%`#8jhy=iEb|r<&F%W=1k`C z5ndJ|&KZUXjt+HWaO--+vBzAcOwar>znk8=C5~QBojd_=!ln}?7(N0eKzSJ@7&sWDU|GOaWo2`aL^h^3B|WmgajOxLnZjp4pzeQ;q|}*@eKtF6C2h-%YN`GHCmp* z@c81)`xeR{`6{MrBB@%(6!R1hASr399zv@mN~U5sP-+?0vdz>Cy3|rLyg3mt^bBkA zrlu-x^wbPab(yLhyNy4)wwl2UJT*hxAXhVCPnA~u%IC$G_0==`dyLs@R6L*GTwLLc z3^jA49o{Ak;;zoDni54BBp6y(V?Lr_lGh0jD=6Dc0B%w)5kXraLj;Z)h+rQ{K+u&+ z0fBz;0D(t#0fLilU5Ee~wUFLnV#c&s*42aiYQPe1A{SpD!=r=5Xr7j^~!U$!&c46}xhTuTGk@I6|b zRCBwqG|-d6(x@tLNhZ!n?gAzV z_o>r0ksM<>28`e!##rdxkVY}V8I3cE1ik9e2-6-Ui{KdviQt#y5FE%T1hisM2#zY% z!bc|rf>(P4f_u9N1edg0!zK`Mi6ju5N)!uzC5pwO57L##9(5n}QY#1BEG?|jz}<_G z9^xpLmy6{G%^wXM3VH=JxF#6zkrE4NYY7WDJs1mobbyYYYfo)AVv>Sc5Y9OLu!bn;Cu`)8CM-xnNicS&) zlXfyx;F!S>yjR&6cwY(&T;ai@hm8?pW*zQ`utOwA;N>1J=qVe(VBFt@#(Fe?muqOe znYo*8mX^f$PAXb!LIMiQQx|BK!~-OozyZA}(SVQ6zyKK=VBpA+U{F}VPY(+?^$$}Q z=ztsvC3V3M?Ky>VFtLn>(UZl^4Gf5{=n%sZ8BnLUV@?(=7A7Ztmrc$HAnoYlMg=To zUg83Eq&$VP#mOpRaH1-R8QUTkr1)~fpg^n_28|=wGBt5-vtdvHp&o|j!47A_HD_Rx zyDu|mhoY0^{6bCJH+DtQ%=CDJPNR}yni3%7aL z!l^E7p;y0+6N4|zu@!=Go)=R9*Q0l@j{b^u{*P7&81FErLc?Kn^!`0E+-;@wZhEg{ zV-KB{%~{E=FrROr@3}{9}qHQz(+u(GiKH-*VPtG zbHW7+1nPqY{h(rIC@6!25-bz9&?;Y_!3Fo%nl!}==grgK={jD<(}E%~4Vkq71E7;< zisgCb9>0A6b6_op7`;Lo^Ay_3W|*U!%jN8MrF2%H8CD-LbVRa@2ET_93~p4qOCbao zs3S$Qcf3V&N?>N=Qgb z6)+7H+}+bcAy*y!^z8XL8jkeD7jK@S=!6WIK$hxAaz!04p~>3ZBtt0F6#It#6G8gIy*(`5p>X;W~TUXA&Pf-ytD@kb2_f+T@N zo{viHpCJM_8Hk{U!jW_A0!mOck@bBP25 zx+TE_*r|=Xb3Q1ySvguMe!@PjCu6Lj~S$pn?cfu+ZCq2kNz90Xb~|78olYENIsxxIklD zT<~0ui;9O`U{LKjp0ajkCSwa2qoc3WYiB~{c=7(dlRbvM*A?A6VrvkWFipvr(#qff zNVmWuLz?))oeEvbsK5p4NYTtd#Sn;0m>K)KUV`#A9-ab>jtmtM{}mN>8Kv?}4VK&t zax)E6)7AhAhT0lTB4`oR$Jl`5YdA7z_kY}fbQX4_&buO!2#_2pom2lBD`<>_3S=Om z;vfYYd&5X!d`2K;yyqara$zuBbD%KB-~pvr^B=un7^OHm6ew#a8BWGr)R=j^zrZZL zg@_Do15UAV;^m`u&)Ft1ARRS&qRCX)Jc?ot010z8>%e%me?dTA!-vanjz-; zOWP#Dqf&)I&jnpUU8;mZA2(<`iwPrD?D1$zZ;8=S>uEGPo zpa%dVXvZ=j_yPwIIF$eb&uRf-SCk=4TJu7AvUpm5Kv?VOp54#B01|kGp-rC67r={B zC=DjcVGgv+hK0)${Co)t$}JdvfgEsJk_5(ePe`Df6-VGmN)eQ)_p(+3*ED_F?2-UR98TB~o zm1hXmZ77k`#pRM&>JCVbu4jLDIirxjPOr_zMsp@tC3{4I#70=nk$?j`I%?+Nyg)`B zIA(x@uA1VGijM z^{~nu9^YRf833;^bV|Nru2CojBLhM(p}{Q_$>0U#F-blu#pZ8x!c1_7aH+obq%EeViN;G9mpTR)!TXf9Ze$g_cKH zGxytXgQ~PQw4i5KtLBVB7zBZYMOaal;3AZom%!jn88UFpKn9hSCJYip3Jr!)J!}9) zU3|g2JFrpU3%tO?#y;AaW1~zL7&GXre(GexRh`=87@d7yT>lLTL>KrhgE9P^5A^?< zVN!I9h-ct?%b~AkHwcX}+^L91#Y7<_b)*<2-P__6g$QUIpg6aggcz1&CFBzV8u+vt z!)ZW+#jn%f>Gb`8s!~)9L!)0@$=d3iK4H=che?JO6dJqs)<_xo%Ml@KLj|K-Kn1@g zRET8}CUxsU;4?OgmGKtyND&wsnKvEpw_<^|R} z!NMq%Knv)k11-41gbRn~M)3l;XoA7cHy&WPxFEH7sE)>!y$3Tc_4E;C;*tyjWz5E) z4=sHJK+udSrjv1dg7r|dw`ZZ|hw1I@Z0S^@dO7>;H{}bj6q1um)(x9Lx10@KfrT>} zW!yB8SZ%{JCt5REnJl1Hb);y-OtwhGLNSf~oK?=uY1~XEHg4!|xxm4=SQ|L#H48a} zSr>BZO_tWtPkS^CGQN2?y_sX52q*aZa&fB@iB{0@C}+*NJTyt-h65ta5NML+HEwX{ zM(Jj}#o7f5HnwuYbs1zVskP)xC+h%B=*TW+qVjdjLxnX7x$mu;~j?M z9JXb?xJG!zRzd5#&yc+^S%7XyA(Tx@A>VQ1T`Ds{A$v$XGD(E>Mm34xOL7aBL(x+@ zJm=B~U}707xMir$VKB4BQA1w;_i9NE)@`Z9HIqZ`#G`f2b9d?Qv+EfOB;tPxD}AD~ zXq#!5HV$!XUqtE{lO^JhRZ;*K(w!~`ImKmxxcBJijT5CDk<2;3^w3i>(KDwLKJt;eYiXce@}g9E_o!2xp5(ExSo zf&s7T3x-|PO3f+a^~D9|fWP7uhDs?~8~*$aUSnb*_gka*X?w=MYkGhb5-4QkU~xlV zD4_p)LIF?wwot$e)`9|1ya_1m?a;2%yntkYKc+XHYrD8|b%mQh#uOA22~?*JD^$je z4Ja=G1Gz>L15`;y47kW326UhzhL9SVC9rkr2+uyZu$rm)!IVF1& zpl`ErB3czW;GPh9_E*u~d6AcA0Xj;If2@>#20s#jyAmEq* z1l}q&1dgP%fS(V*0q<^vyfzMSw?_*&m1~I5))=8s<^s<3hXFT4)U+_FRm7E}Bc3g1 zACwU}coGHx5e^4NlXN)vp{c{cr3O7Ar2<=ck~&fhb?q9fv%UE`Dzg+@43I+g)woR^ z494Em-|2MSDvPBt{e^>p`^4`@!OuYN4fQvefZHmT=B2*BvA4q-DjF&;-cQ|RvJzwX zri5GCfzd=qgK#4fWlT`zx2c89K9#)DUxDCBscfM*Hmg7a!prHV;*ydA!G)~?F-=RF z0j=C0z#Io0i5d>4)5}jVD83-m7_Twx%VAZ?C_Z!0HxbO-0X25U6_Ci6)522dsY(n; zpyyLvf|2Rz5}f+BHU*mE=@NWmu1navn}EdrK9`(nNIbp2m^q3BJ4h)#(pVHeKX3xV z#KJCK?(oByY$pkV>sS(k;LVaC2(1hbxXQ3BNG{T{AXlYu2v+>yHIry+muW$~HP(k@ z2)x^a1Dl!Sfyq@a9x&+p8W;Nq4326%jGkOC?k}C<49pjU6kq&&^5)k!N3TyFKkAqg z>6}i36orAodB(DZ;jVCMPPAmB3<^A39Vwa^Rvp}3cje>(Fr30H%)%ZzvuS%g?phjX zXAt6pXGri8;-@$dDQF6&Dp zfCJvF;qV&U{akswXP@DWo?GpqTHF=W%`8Vd@*@Pslz#lN3S?oRCU_uVBpxiyf{P41 z0G=8TxH;7O&0~50S!r4M~|DT1fxU=H_}CBlr}vdh9Yj1yn)l1MT%nvZ{TPuRrstR zgPyd3d%I`@mpp(raJNSr_+*1NQjrh$SPv>|!t~iaW=vqW1x3xn8>r{O&tP*&82jbkL^X*lJXFLuQb z*6C;|M;S1He_wS~9aTcXmeiKb&x`qoJDWUSPA@(xpfu!B zN-}ODBU-$)Rh62FrY+Mm9C_gJ~!qH<;2*$xs3Sp9ctAu(4PR9HD7?zqso=ikKuqhQhxww-~bX zNCc<8t^H9T5uzrGy2?zChmZ)(J`ISY>-ojZS>g2Rem*pKf{!$<(n8?}BjcQ*_XHEH zfgv%$B!CPQ=taXS%|W3P&cg_^jZ!?a3lzM&u{dqbf~T~XNTU?2x?DOT6lc-onno!;t*l&dfr+koymfh$BEbdf zNTOPBQmGak5qbr`4TBU%Q@Z)zE`l;XK%+_g>0CGHiovC9 zhNwJPDJlNFQY3;iaHpaSgE1gMgE5p`p*~=Id{-@k7I=h#QyyWUM{>RhX7j@nPA3&@ zL5+pcFSEP5u(_$t7%%fw^k{_U)@TP=fJCJ!s>vd3Lz1SPvf$Bop$jaN#j6Bka7x;s z42BdmXFx&Kx-Ruusr=L5sSmr_ORcd5g~17Yh~E!SI1TDCBB*((Kg&C@qw{BX*gWcb z_UY@=nZG+As2N)X^K7xW{M{=mbiVlHg+2@lDNA){i`$V18W~q!Y5p_vhZ0R~78WTqk%Qh- z+@xTF3)GRKiv!@GF#;_6wz zz%fx%rr5xkPJgCQ8Ne>Xk%D2cF1J7dWI#FlG{18e&lGs&UxAa_c?UP-<}40+2q|$1 z&Kdd#M~89=Ze0^0R&1@KI`@k&*_3Y+IdNm~6-nIG7gbY5s5LdB0lUk}K=DW2{|6Y2 zsWHpZUfYP#fMwt@{@ujkK>pcI7;F=xw^eRHQJUVZSh2&w1-?aD5)lf! z^o`<=8W&ulnc{-0^As*1*$fvrX5oTeC2_G%J^#%B1KizWM+W0Ld*=n$IQg9egNxcW z1PpYv2FB|P#7h=lyaYucw3E|dqnzT}Owi>nFnYiN#}21(Bsg?|u~lFyNdPeCX>4#{ zU?^;G*+I$~T%hBT27uc|8sYZgtPu(wo;I{0SmOYrA_oV}W)~dzmfox}vG%}ywz&CY zdhPA?@bcc*CWC?_O}*`0o$PIcQ6(}evTDj3T!SuogB3?I-axMyyg?ar4&cB`jIbFhxZB<= z`d;S_bvUv_{fm+#8~XL<6TqN0Q%IKeU-f~8W^kcGmkM*>0(GPq_?T?5!kzGB54^Rz z!3s_c7Qz++s^OcuCNz%I->EYjHpKK|Wq6w5tDyIW`YPN(rv*@-xW|O!O(W&W>~iK6 zn0mVSgvjecL8T#)Gk+_gCJRIx9)-|*#Q>+$E)ZZsA^DzkIG+&+o?;LP)o>zU&h_HL zp7en$yA%s9X}Q5sEa@eZK7a#{KB!=G`k?#LN?$oDxT80H>~Ok4gJSf2w!E3%TzW>w z|AXDOHyjYtJc8dcfEqeD;SuWF@5LivSivJs;xaw)6l7!(jMAJ%!g>sE`i>9rSOlG5 zum~7+FpB^yEspxI$PSaVG+e5Z@3%!+MH0at7{*f2vqh7?1U+ zNPHF?$jAf74D#Tul00xEw?$M?&R$Q8jm47+2kG?C8fI*`9KJuq@4XM|o$IHM`Iuhmj>qOnFEDN_TFQb&r$ z$L<#A@&##B3<()=^l_LkN#I4=sGPbXld!9UXMr$J+T{+k$KVd!F_b&Vz%7#caK|3E zMbwb_5fi*$ZM>lV&}@0<8X!J-uynD>9?f!yR?v|%!cL3vM@=BC&YIXCxd($0kWmPF z)SwWr_Pr4#B77wg-(A zk3YiNiSK56tUq&$x*`TxUM;5cKb#ErliRx)Dj?cm@#gwFH4nN_U?wwGwc=sG)W%Fa z;erOJQZVXJJQ_3Z#$*+f88gtB1}kaP1I?;3Tf_fC$$+@T?>b8n?;WaiWO>BqrG+fT zL;CERVgBFQ+*?{@q$f7!>iN}tc{w`98XL?dy*+zz{^ISM7}#ja+iByXInl~Es+q9S zW&Ba=90m*%of8sCMbwN~%9IXJVkjLvDwQvtUTU-?W=5^y6kR7<9sTs|`8k@6MH4UH zJj3c5X{PdSc_g`_wo)PL_7Z8hGEVO)9kw&qIz_J+jS03-AFXr9NVCQSrsuxic_~Au zi?0`M#i~Liq!IpFsDR)kDrh4ns9=!3b`&BK*qQO~X4ck51wLJ)0+32kfh#;z;8YhX zgvf1oMqDCM5qxcX_9ZO3%S@3?fno=HZRemE{jj*b^%lyUFT6CeVd)-_m(VNdrm`*4 zq#0^-0WKT1MbH*uid0?=2{LqwuV9Kq^rlLL6D{fQG?vP9t|xa6xor4AX%)<&!tw}6+!%(OM zMY_h4nmuzRUZ_=cb8FMetC?$-Je}RnKLAFv%hB^W6QXmYHSD}@N|_#|QX$bKnXoEr z{83x2G;_ zkcXV~5aKwuT8mS?{8lno536;5kD&)`wZa>LE8~FU>;i##n?g3DbkO`ViF~(d8f@Pl zOvjW(+;vujr;4~@sOCIE`^qDd0qd;rw4qEAvQnC`eGk7E)gdqxG+NHCX4jXVk7J>w z@p2134*K1s5WL@E5Jp2KfdEJef8b6@A2^+PH#lan2acwq1$jW&J-Gw-Zh$-Rl$Kt4 zbB9SC=%&`vP*GNXXkX^A4Ul)U<^1-_*2me8!l9BUgO5OSG2B7U^J8jrW}6cr(( z9N)fv_v+;Q4 zJd`!owN&>HnChuXgIQ)yL76jbRrh|eT-<-SQebI_!=I(Whnc()kGJxTT!Az3&wowv zhDZo{B)2$la1*p~u)T*TX*8UL?bh)I!^bUb7*ftuUK2*IxSR>-n`-A3Yesvm_vG$CAhl3A8gYB=Acb5}??gh6LW(a{nV*5PWnJI}fAG zE+>Rj-`0@83p^*pG$cY>qa!-pkT}@cM~7=1oG)je=1!{VlPlO5o@#KHJ4FqzH9tPX z!2x6=`JmCvLH-n`oJHx41H4n=07w3-;sE_HL^CvHbpQ0H5&uKyz4-aX8E3zE1cuzn z(X*dU@B`03IXVl+|MaJS``3T}=pMMeo8H}yxBmX=`rEan+TPyZ-yQuMfPp%e;@{DD zyf+z7_Hd@!-4(qiVdfxBG9S?Qhql_Q_ZVFE)w*O;<9=W-kUN3HD|KmTt&TjwXKOg`5qyP5w?UVDr zzB?J!!*}%m{X2&Be>-_~^7`b>`RK0*f{gzBf4qD0^yvKPzdsh|>Y+dSbNz29p#7Io zqzCIS^Zsz3_lo$mUKfr2EH3}ch;_aHFZI&ZSN@qF_LouJ^!Ts3l(&!m%>Rcv;q~)Y zpIPsK^M4w5gloO;2@hjgaqoiqxuZX?Z0x^`+8x*aGKzK-j^Yt#e;Gj>qx$dYG5ux4 zdBp$37ri|@JK-N8vj3ld374)EC63Zk!ar1v_-ADp;jdL(g{Pv!i@(Fo(0ur_j=tg# zW#;gd2(8B3+M^*Ng2T1_f4$sxZi)6Lv*F&TMeCpGk@^uQ)7!h&52%lY zJ9+mg?55G&+8n93+Bmwywg-1e*=)3`ab?(c<9Ip!!pW^hW6BFxJJ3Cfv8dP6>+8i= zJ2r)7^mjPbQt{(Qk#rxIgHv#t0{v=l)Sn?``kHh{89U%~b-ft24gx1@QSgevVF}#( zPw0*Y!eK$x`j2#*K^$z6-8UG~LJQ#1a(}hB_=qh=fyao1M~R$&oP~-#$&tR2XB{P>y0Lh z5IFn1#9~!19EsKNU=b4m{sQM(@>~RBVgT|9$6E-5PW)z?6Vlsh9(vDa%jxx{BV~V(wmd~alfI>xm;qnpwTE(sk26p8t#F}E1*~~I&445=&qn~r7vg< zqD;`V<%HQjW@Rmt(z%Y^6*C^=ikU+-FU^gQA1Y=lkfCBe`?R>ebJR8$K@1kNTi>ub zp~iEduvBpa_&vpp-uA_elbM)-3nAr7E#6M$%7QZK?{qk`R;MdxoHOL?3}>K>{N2oL zaJRH*p~Znq6_g+C=Ys#TxI#YAEgoQuXI8Ov1l?twHI}$m-WZeCV8&rFmhs(E$W&SN zN*5(k+^LQvq)i5p)>(xFb4o5@_jBvEG{axE)0jSY}>oCt4SINz@1Hs!W*jhWO>zZds8}|Tka(U^7 zU|jjT#p1YdUybi5@KE=55-95p9B*vDcy{oqW)9NkVVZV9SBUg@>@cH^Ro#8xP6Bt0*>ws02osS5kNdj{zOm8 z0|oJeTRIg1UfT+{Zf5(t9`%r@J`}xyToX`n=JMz0+AtG5SzNzg-rqQDLkkhy5)E)a zNfQA^>^2I$%BO0yJ6P+wBo@GN(-Q&cIgJ1uN)bS!cANrS*@b`}1RyTy&;(A&@X$ub zEx@?}qyKC+b?0MG?=QbP!@n!u*9cn2gCa6opCH8f(D-t^FAOkiHWm+DC*C*S3IX{v z+{cN2Gjz{lWohs)4e1uETLG3vNEe?lG}u?$sZsDT(AzJh`))bEbSC1S++()w))BF9 z?9qlYhc_pU9fGg8mq;AzD{~)LCj?JMmIM#EWTP8hYw$Y+rA9ZOon%`S1~&oJk!^vO zw$yF2ySPLSZoELcduUtE?KEpQ&F#piymreWzF922Y1ppO&4FP8AOb^;ZX6b)8^2{V zKlI)PMt8Nvy%0n&pKUcqH-@7(xN#~_sWtnGrfzm@(?iVWaQY4HH^}_$4fY-ixYEf)v~iL?A+PUoQqTVbmZEG1@23>I<&3{0W{G8KwkL?-Ia*H z{Tz7)8RN=ik2LnKL7&HXV+dj~!zs&nkzHK><8`<^? z&Ohi2cvQSf5ei3$l&mWy2FbC)=f%+?+hlK%-P%pIHS*V=wNQ;JB#;SI6{|Ixy+;o= zyY9fsm6=E+5-tudBIZC`r7)*0hT!WIQlhC1Ym9dkH8u6cP=MbrZaBB-GdgA{pYhHpY=R>p zpD8rP%r;JKcKD1J{7d+p$#DOO!g=PbXUR;{!u^zt#*RI(31UnIlH-s{YEGaS!)BZ^ zNsY!Uq{h*qr0(iyBW{H1eL>}*Y;mGrZx)#J?oylx?7x|DUlu+-E`X#}saaiqn_XZP zls1Ipo~DovZ3I{L@69a;dzSFOP|A(I3jHx)-V)~pAtR1^ZiQ5OrkgW9#f1|sF zDCwdF`5h`+%p4tTFea4o7dJf9<*VurkFmE$GT z!O4+6rUR@K&L9+lWH_ETiZiIhTn}v1!Kk8TJGh9$G#-#I!T37l3ZRBE{2@yqPp6ld zOf{nGnV74&1djEvRCHNY$f7sHmy+L95+T9ss7Nraf&(2HPAudKIL0r>qd0B%@Oc${ zigXOoO)14?;#{ zpr&g_|8*XwM;&75RF41MEp_T>f=Y8Yi@O<~tSoiU1oG@4PxTnzqUr#y<1}QTVt`!A z0MjS~8Q^yBQUo*!KvyaffM-V%;1ooR$q#5E z=sCL^lAqui(08D|;C20#G0x%=2|b!FnNKsBGM}4Ljtu=vXZX!sTXWuh(ZhqFs!{-Q zM)Y@{UF_lExAgFE-zXj)CS1W2F^1zVFluPPWzZTUIC@-;ZQZBT0qbD@?g1mXck{(^ ziQZFmJ9Fi}0}ySTZZjP1P@&}zEyC7wnwf$A@0#BLI8f|V#gg7QpRgOiDT*Duh}g}9 z+mPLq9fRFCRZ{Fu@_g9}o!xj}O!#I&Q_OB$o06ODH_452PSSds^~DO~w;LovYRDfg zCo@a8=j%sv^@0vRb6pMmcc{}M0RVvl33vj)r3BFSCnA8yC?ddnrFX|_8=(j|yQ$~N z;>wZ;Kt;Fs_7(wrvsCb`kd?{+r#(#>sC%dHApQk5aG&3OUaYKCi{H#JT}i>4E8WqZ zkq|fqkHcaJkH$&`?>IEV!iSwLRtt~{9?w=4Jlz0m0MB!dcAA2Btx0BS0n%Uvk0-X^ zH*vvEIa9AExAp+uZCM6LG!Q_td5XV_m9#a97_O4= zgO~~WxJW^tw@UCSTR|L#p1{A#UB|~sj(7d0tKe-ASg-zCKFt@<1gqH z6WDm)Mr6n1LQ?dwh8uF~ z)&|%5E8E|-zt+cvV-@Wb`)nC0NQ*NjJp*spN)9X5jGA)qS7yRpHtyEo9OqebPRtSj z`hJr44x-kNa|=yHSwY;9;18q~KMs--;qRQpz>y<7!|})STfBArGt^I9cn&&J{AJ=q zM1Hog|Ag9$)tsvpzvJ^2jeowl#0XDS0JaZbaq?fKe^X&BrT;D~4@vYX58)_|JHpjH z>oHfW6QC~@{Gm1@{5iK9s(z-39!GiI78hPDV)qU`AFd0E{D=__K-q6#&L#^Vy%)ApUgm+2DUE zRLAaBl|U*2&YNd@!zC2xE$P(*1bqmH2lnym_qJH1QF!%?E~!2}6|UveGv~qeclVGx znlRL-r>M8c_w(uBWB&)PI@d^!N)*ujs)jF*)R_jQV$7gdWab^X=uQDL`dxyIJ{<+* zr&}yTDL}?*bo8ML^Ap-%W9s}#4raVV8Kk3?Lt$oxq@ge$E*9KnS3~*j;tomCD#|%c zZo%vAduqlR*_KKi-52O^bd5COphTjnyO<95z_C03$N!%GH3xKcE z!tktnn_v%-Q?TbiF9M(QrlI+!J8!{{7nR`0drR;G()JPwKL`=QPtB0v&pt2(fK41= zz9(~RxMuC}G`(Bh8N4rRcQZ5Esy2c?6xtoSU^Ci%H{G$VGRqS%ThCPP0M4}Gx^llH z5!HorZ!|IO;e$6lG3MwFK{n^4Ztv%o=dN zT_GN8RR}9$j4n_$I{LE<#$f!WFiy$|q1X(%P)8Y`YoWZJ%Wpbdpo;s%?|P*gS~VDD z+&dKI)?_F{Vb^$zP3@nFS2%Sb?(?Tt4cTkb|a+5x#EIeJKt z9iVmOI(n-Q*ALR7kD07Ty2_SXU$2#GNGF&Lt0sQeY8|c8n7#%)I}l@Ry2ZbFX(y)t zx@d;xb`1N!a4oG>pY!r-G+w)^7|%4}%ppmq1dd!oF*<-*{hM&Hd8lE7%qLOB~U z2c8Pb4{*fg7_1-4uNL#aCpVTK{%Z2sg1O|PLG!2}1CvOjJ8)x=@q05=2FK(`%ka&h zTP$>O$~4waB_&G`(3+tX#sh~kIFIe~o9RD+g_`qny;$Dg-7mgksq)9OH|K9YepsvI zUpOvRab=EQ1gy8ttbyzQUgtTXRe`0ZlO`=S%wuAx(TPeNg||vW4gC<>Y20SnX`C9t zPV)tZoklaHnWlzMqNT2wX|w)d6f4c*%p5^$syjE`opaOB7!`S7X#kHqBHShHG3w~a zj;Q0x2z4A84|M@)EtxUI;*!Wai&3XiNKoekEYNYzX_$Vl*{9i!>6)KlzeKmwj!JlY znuLJ}9d&=AwbmYME~N8+*L)`>+w>;b`JrsblOx>0{}6XvV6h#iHp_OrK(ZZrFU6gs zMiY3P8vy*t#9C1LV!@sBRLn~Tyiw?}HxA*9_^w)TG-Wdud~b{PLKyL#9Wz@wy95d9 z*mbv-*?{j?+%!o2P)iOHw76%yG?=*_rR0v`rZ1-xEGXt(e;mCz|2?QehYE4-@Q3He zFMoOY&#%mQZVzPlVaL*k+sWKiC@*sHyYg6%X+Nr7p)u6Ni-SWk4B()!qk!ScKBGMd zQ!HQ=Gbhv-9@8c)M{ym2Xs{OYgja~)^2g{nh&N=ck~<3y^;N(nhOnxW2rvrhAT@3ghJVR z3iW5UziT%^Tp~ROIM>pGFuyo8Axt@FRY~M9Hj8cgIS-jle9kkK|84xurRaO;;)vlM z@NLdE_#d_FKz$JS4vbw&^zpO=d^|-NDxl*d@EH&c!LO@%>F5KABKmk-i?)tFn$un) z(Z^Ry^!4Oe8NQ9s$K&zdzRTPWC)j55-k5Veo?We`*3MleFQVD5DfS|$Vk0SV>K%uL znn#y*)beP0KP?ZnA-sr+Nu(?dB|r`$~8?Uah#NOBik9z~JAS}{v% zX{>TWfWVFcWY;-mmb#!f4H4LVoLFF3KGVr{Fn z*yZec^34hfl!tAQSXI^g>)UzekXVa^f)yF^bjBm`V@H$ zc#g(fhL57%j~B~JRDidNPTAWrX39e|QACFxJzy}uO4q<}_#R@9K9bnuKw{i+kAgiu zDB?XG$`E{FGaP(DYMu9#u0}OT-b1J*?N!#xiTMb8JRPs?J5qnW`fTt0HK;7S2RIp^ z`x7bYxp+%*9%u!)gQXI7JSu@5?N?yOl@aWGbfE1C%5c;Lq`gtc-4b;iNOe68;>eW$ z$MUmTmF6Fa&fHsz-b!P4E*?~QkCsKWcMctFZGP;|QS>Ma8>=k*r; z0l>q$;*VP+_&cMIZY`mYOIm2_Z)|_pmRwvSjk{Ak;lQ0*FB!Mf(H|N5PuH`1D|83g zUD@cK$aCCmyRz!DgJhl9gvdjw%FbSHym2 z-O)F~$^(2Ep~v$^G4eZUmBBr%t1zrQcfGc(d^78*S$TYJcPr0}OH=KPO0>}Tv+}PO zH}&XT1NoF|+*OKa25%e~sqmt|B6vNkY2f_dHN(;A0m1Q0(i_(&?8Y&L-8dSlcJye- zZCqec8>dE~wv$d@Wmc+ooRZW=LltI+=#DaML!r>w{E8WPhJHW0SeV74bM75?$xyMU zk~P4ul&KksJ`{k2A8#2BKSisXV+VLfz(1qfaZ2J3BB%IEdXFLi$5!zQE^BuicB-tm zT{=b-Rp7bDf(1O2l_c#sOA+vXy*wqDlY?RIh&(T3D)GC{po73_gZDMs`Q=jh<`(?srJa&1u%i8D3LG`7erOgKGbdxrY6bK~EqYUL zMf66C0(vvSMJjfU-smdDuG1z+r`?q;4HP@|?(onYtVHXO*B{ybuB|veEIwrV>;}CB z!V$gM!u}E2{FM7|HMW{h428Dgqp?2osB8V4~D6oYhJwi$>kUee8wzo~JMig0j^6$8@gJB=UCUJ` zA|FjYBU8@EdH%S(!L}dPM(UYZ42%1!5Hn{#!r0sCZI;XPKk8@<^Zo;yP5w!f%}n*g zWTPiFlZ|_&<3^A|gYDFzoJ)+s24wtLdAUcCzV*)A`&~>Byk?Zl+L_KsIO44Uj=_;hp36 zW?q;&vS_c(Jy|sPx+MB+e|GQb?0QzZ099)Q0IbFeyz8gRr z!HfeAE3)mCfIq;SmtsxSz5~#JB-4Cj;hTWS&PKL zG2Brj7^I;GB-7dWaY8L=!9i@vhyGJFIy>FsI@g@IPS-R#{qHm<4q~8bbiAaf(K)Hq z!rv&EP8$7qv9MONoi7=0SHr4hnC*xf8mhsJZq6(8v0&cYVt`g?bgFF?V+L+5jG2jP zVvGkW7~{P|F{UaQit)+fn@erd{Ia*x<@9zq%=jvm%-&^-Vrs5i0xl<3dn}L-wwPck zKxRs$3NlYvAZNoZ5C8>aG+{8v=%1k=|2FyCuJ`JT&k|@zC(-$U%cFhW;5f#_&l| zr*C{JQ~iyxGe%A}qj5RZ#@&&72DwiXSvQdFatIF{`A(LPjw=4vMh>NM7(`NKI+_}x z?+ETV5h!$Awi%wo>k}l0;Zc=2zTeX2xWyzlzPr0BXQNA#BY5CYYX81i&98J?pU-ZA zgErA!!qv=SG=Oy-@*{wawKsmNz(QqMz~&sWUcmC=60mq#Ct#t?dQ=(vF#Y!E_XSzY zyRucrj;Hr`>&sl#v22mC=QlUg`Q0~^8^sDH%}~JqyPdenI904NhT^g16f|ocTc^NV zhi#Vm@;_=*!4Yj>E7H9gsXmk8Jl?Gs3tSl)i+FENXL0qJ7zkczG>^Borl7aAz}?bX zaD8WLF66c!(O@3m?{rwdnV6HwcUPA}6#z3MI)gnrTD1~hBoyQm9{~pf+-v0=xCl_P z3F_ddf;x^ysN={FbrJBsRxS!I!rFMCrL7wIMJSaBayGC3ObiRu&L&n?#!c|NZ5utSh`e_RKT<*w<2l-sDU)&0F|gzTEC;Y9mg5uJ zOXQf%#Tvu<1WX=FSuVF&k49~^0qnaz9f8}=hEw+`lU!qKcr$kjEwX&N1bV676Vnc; zI>JuXCt=5tAFA6aR*m|XsN+;**)`B18zXD>m(%$*CgF``E!k3YnLKHLrYh#{ z;N2y3NB1aN9!Ce$9ru=8Ibzaqx&FrXcg<~aNrCRTHpLyvZe-lgFcEL9ia1?;o?5kI zZ5@vf{}0uVUj8u>ZnH_V<{lqaW`w?%mX~5}fIz2LX~|%^^NCtVPbzBN z} znAu8InfHWwmoq5>Wit}P$$SfAP869K$U2eHDh=c=M3!sV`Vkp>b$pr4 z?{pexrb^A36tr9g7a&E>m>*^{v&H|dO^u_!(3c}kpP`#W3ytozEHu@_^MAa0eGa>V zRGBv)ULjXO+GE}=j|5i~{cwDWw9pg`C~dT~rR*pkJzbCPrnuTwO`JfG(Bo-|dxjoT+%xoUH0pLW$bvdT8SHU$342^p!d_e=hut_8 zVGqrtfDct^qK{`d9n&uf>iA+gF^U5oE@vz&qx$FexI0rpi1-=c?{U&ivYs^!LhqcR zc8R{~okM#l@N+UG!rwhVz@Nfqk)LC#k_6CmF0dcp)?Okd09_*`fJO6C33%#0(pa62 z#}ANz*CZBJX`gpjiMB&!08>^0m&n2Ys7O^FP*lA4dWSIH=nfXnr*b z5;rLVK;e<2GpeFc_d9OsNsxGP34P$#LR*Vz`@4o(Tq4oO3ncnn*p#9#w`hw-XCTl3 z^iQYX=ZiZ-{Xb`6DZ3uYJ=V1&-~Q3b?Ed1Kp`2!d^;vE6Nt^nw`apwvbD}x&%z5o> zbvTgzLPFe-+yA}M$t*P9C&6cxqjECQh}@Dzym4Gx5HsVo*ksg42pif z-TCBxg+zIc=Wopn`dlD}qlvEYdS-K90RMN*a?m0mdEJ{K$2*&N56H zFTW}#Lvh?KDXv<0&O4YE9*-En@E6mIFIJrOEK^p>bm{RKD(e1t%V`fZ&iFt7Gl>>^ z^yr+C5kHwt{{Gv@naI?NGXby6O72z8rAz&JCP@Wcub6XM+e+`nGrh(C-DJc;`zjWl zsxam|?Mi6V@zZkY(Cs4-9vUz<^FAkz-y#{W@8&^A&lCW*v45KT$#N`*P#p&KES!zVzkrbrNNqz5_0z?FSxKxy?JS;RM%XfCNq^*jIcvb^+A};41IKTx7>`OgQmX)s4zm}cb)I}0FWQDfZ;9nYA^=lsOQZzICU`IVX2Bm zWTmvX!xy{|IODg(nW8+#SZp_Ca`kl29E?2$UKY-DFr4Pp24gta9cRE_Br~VsTg%-D zS0^$bP44cIxo!=--DWLTodVNPlX)tYmD~q!oRw7zXjW*cF({W#8URQvHH1O&(a?*L zixvf4C_}@i#2K<_tZ@B}?eE$uJ0bOhwt}Ii#3uoFSz;!Nxj=OeFx0PC^Gma8f5t(x zgRYcB6?le7-HCKiRxy9qOF`|TL3{$>84w)`JiaFap3}>riG!rD}e^|=Jcb{rN*TwO-GApu%S3}hk@fr?!q zWq_(AVLz}{*iY(2Qo!x~LMgyci~kUzk`&;zEyOzi@s#!wDFvXrbkz6hye0~OyPdRJ zK2iXb^Zw(r*Q>iP)-uJz6@saTmA4f@#ZZ^30l4FUf&WfANMq5o0Dn$^OZd5RB!OR7 z{`{l}ey-vV;dfoDkJG@&5#Wb7TkzwQgrBPdQuvvkV8TC42^`)`xHX41JjeX|&CFV^ z*ulv|qf{wic0wct>Tr#Ua8q91Q;3dm=BoqS%R|pvPTo3Aukp8pQ-=-^ziVe6tr|LF zV-%~!zj(t16d%rSQr{z~JQK_mBH$l48)NfOWrKx8Pzkul6J|iMJ*qabt z*Yh|U@jNc|PDMT3&BK?pB-YhDUtk#fG!93{Mj<*~MJu%eVl_a|e^}farvCNpa%vg+ zGR--_RAoT%^Bv6rzZlEW8S6-Mbbg=g$KABVAa)Ki2Xh>-Sv==nUar%%!JN)>njDSj zyt*{C#&Knfe?OvsgxfZ^sE#RCz@=q5XI3gzrc2+hT}+9tR0fV$DAMbfOeY;WGM)BL zPO))Wb7;}YRfFk(vt-d-QMiW3I@JkNbBXlms2d}jj_>J7^dL#($q>n*oc z>y2*j2r+b7KZuiz=jy<&>(2~2%XqJowHh-joO0)mBtT<`qSJtG zxlL%V+ZH6d;lK|-9dG%;sN;=4!X-POSow|9KmP2lsE-E^vCdfGfEsw zk&=L#Ta>1%a+avHbUDwN${Z3aYW#O1f;Vvzggg%~J zuB=V^jwUyM&8(F$Wqu-pW9lTytCqk@)s;7z7^a7{^uat6w4=Pzp> zWCnacm@%>Tx7*Q04apFE72lEzfYzw!`r`$3aoJ|j#e)U9_^pD=>;BD$ajzD)R$5%H z1Y8MORyOGaE%+pe3w}#z@w}08;v6GG?mpcyXHi_CX9MB&qS=N>!kGiJ#qS5tmOiAw zz^mniVRDVTxe;4&6&hE~sUSD5eMDgBLV+?OvJzzwY7~_B(h}WIADbw1CfefhI!{MO z8N&nxWjsx@#5i0Gw=rdOs432FmI^y;g=5T#5^z#Q7p->3TX6kQoXh$A-}SH@hzG-R zJaN2W3@G+PXG+kvb~2?fQ-YywIxHVQ81rIrXIA=pfdH$ub82RK6auJ%?vJOSHmf~5l+KttJ zgg3@|W2!qw57*M;E@hH)k=^IV@THm_2k=Eb&Sbu!di?*PD&rEjE0dpC!vK0}!M5yb zVz02{@M=1U@lEc)J*-6tcEsD0<5@539F-9qd9Lr`)6e}zAe?{kh2!u zD6C%hOy4>0v+17~Go#}AyQv){$;_oZhV4~Vc`kK5OwAoAXvBBxdh6*_)z#x>vFa=RRd1F-FFv-lVD=}eXG zsivuPoKswL$k$MnPEl=1-`J6EbEUTvD+D^M1}}FTR=I7f^y)E$Q|fda2y{9dD47lU zPneC16lN0%$!t!2gv@rmrzfQ2REgPmeoI@O*@FC%*?56uwyM-!irmPVOv~vv%a$Kv zw&lBF%}eCA&__eNa2vh38E#iefzHQ|9zA-zW%z4gqs8sD9@Ut=%SWp0#?hhd9?orG zHTiR4`Q$IAcgT=XUAD|1tYY$~%J1AR8oNxil?wu(G5;J9Oe0snN3iraU{ zJjat2E1oI#S7LgP4G^9A1|Y(WZ;a|swt%P@mT}CZ56d`!GlHlChGhoJ+452*IzG8Y zzMlzybu*d&1(j_;{9>`P3h`!Kbkp5t*L&pB~r`RVwQ6441m%U|l^eNP(j6Pc%L5QY3)VBi*?@01`?- zMgVZCBmlg-L01<5G_buy3IH87>CbVRCY6ASQHqxDqiFh*>C#$_!u_<(rDr8Q&zP2~ z383vFAy9{Fz;QD!9qRd^7lopysHuvcYQa+Uu8M!Z_#3)ps5?imrbWJ=ra$|-oLt^8 zEk(2C*udPJNhyKM;7T8m!8u{f@mqpS`5vQC$j%nSIs)V;c6Ry>zQcmq)ef^@hTJG) za_C|WX0)wn&mS}ia}{!^#({`s$MMLL8h7I7OMvKVTwEg6INmL_IM)~|TAY@^8kVz# z{o`%Vv90Y&hdx&dDCfxGLsVv0#4l3d9`9YQxAs5QQFb&o0AH&Tf#)Ug162)vJU+tz zATr~1PLMugvQ8BiV(P{p7iA^&J2WF0PnN308W); z0AE*<0g6(4jT8dNt`q`pHKYiEH0k3A!D5L~ow0|~`NFITRBxJ;UP z*Lx#ua_ju&=f6Dj&Ko+~|M@I5=)BGtuj7#|~)*#^uE0#%YUDZcenBFb-azC}8eU2?By+r$pdn zloGMG#a%!I&e$8m%x%HHCO6+7GJlGn*m_=KgwO?0c3;TQ$=eS@p!}=a|uqr$&m~J7hZ?llULC zi%#_)cxNm!T|2(sD<{DI?`DOm?wsMpk%R6gY=T@E%7an$14xhHn&D$D&n$7zIW$c@ zGa8+i)sbU{I|g{?Z!w={r$pG%p6W~uZV&IQcBj67yvha7#L^z)Yfk@$(Da#T3=<)&Zzxe*;4i!M~Wy)mQu$^<3v`^4!!aC4LFWUkm<-Mh z!2D#goX%mwHOMoQE2g$*kWxjQ5lVyYuGXUj9r`N(ot9Xlj<<9~9Yl*zr)>#gClw8@ zkE84HJ_~l7lCX2gp5mS*`#|6q%lp-pY3MWaYUTAQmc?g5%18#BJLei5X2AjPj&2<~ z(Dv#`7n#<)n~v40{e9Di&sKBuN>&?_Lp~KB9!H0Ibu#_2pAVngm^{Pr(ty10v}WH|5)#}@Sx4=SJNxh&90o%KueTrDLvjoeEM9X0#^g1=U*R>rbttbN zKz*%#SyR?c3}*`!gyT*xKX15 zQ{6!0X+tsIr_LX}yPz&MfxHTV25>Fgvo39&uI7sF={h5EyHBa^XS zjH2r6GRU5@afl6_?tqB6jKczA{O$-buIvMGqOS3wDvyDYg>4R6*N_BS939GH+&UE5 zmv@-?n%>+P@snJJgE^X}L#S%$%r=EpIuIK7VjI*-hSPvb37tV3M8j#$ zbce%f*M)j&15S+)I)gx1!)cIO`eAx*3#YnUm?KXN?H=HXoi1*y39+*aOo{?WZPGKl z2g*MBOGUrklDs9hkDu&6Mg#nhb(*q+sKvixIttnt2moWTi3H#(MFO1k$T(U>k1>>i z3b$G@6^Qf*LI8=ign%!XQotgxsXJbA8HFtd!{p$2ay2zH{HN(0^}ej7;`S(v=;>kr zF^<%`2cFex{ol0^;My>vRtGNv3Fxi@*fBBw5`KKJg+5M=fIeT)Br?mq7BozPPk$!a z11FoJjHeIK0H0>_JLjlrqp=s=EzQ(-NAvE+FG1N|{FdgOaz93#1%Wfy z-9c^(L$#`fXX?CV;1Nt^MzS>x9q<`;#7wF;~r&95}ETsT{K zbI&TNykXG-(~H%>|2jRUFf7W|Ez zH!p4diYvcZTpHPkrwiso4Ak4`**>ghW5X~S+9K694veC<4;V5Nytb4!y4vD(ErU&* zL-5$Oq^rtaW1O#{m{-}5bXR#5EO_(Vo3l3`Km784uv+=iGx;+-@`tOSdW5;pCFj8W zh4o=~GavBiS^Z%5cl{rCq%PHZbUnM6a9a#sl&#)+wEBFvzN}oi^<;K;F=HMj8m&yk zLls>N6-elbhe#yEW-}6r9dn@$g-CLQtJ)f_r?Z68WnGW53 za3CrT4hMXR#Q`vi#er^#h70r&Lz9D-StbXMEDaa%uy!m^e`Wi-wl8=}d3$Z=vy-y;Tw#l9pAsUb4Hxi1MV8AbL1XYb1@_-TS6|O-{i(ySTRNaKF#& zlx9bKn6RyCan|H>CsHlG!_`Gni*p0bK#TLKQj6nhi4wcnTds2tM#5q)4FDNNqAzj{L`Tiq3!_wuP>os|f*C@IicWAmC?=S0eNPH^* z{dj&nn_`n|+^_oNrY5&p&h>^W#MDLx%hRQ+(cDg2e*$kwNDi(lBnM0p$r;`YNuJ?2 zPHmRs6r$Q^dsh2rjNzu98e@qq#1Y5Y!v5=)vDw03*lR(D9`$}96lz?kWW+g>q6z@| zGQ!_co9zvaIKb)1aDvlgW`1{zGem;n+$vEu;v8~W?FM-4+Uem$9Ui)szwqwD4HVnO}XHCq{~Yy6fF_$D43eOzD+s4h@ubJi>Mf7kfq zKmb2itDtoO`arZJ`0N;&Lc?cwgC6&mpvNUGuJt#zzl$Mpv@05DsnByCre%2y{UGFD zFIIL6O|Gtl3H(zF_)H{in~R4e@^qmB=$Tw>-9o;<>S80)ds-RjPYl)tsNyUkRFt2mfZX8?AOeu zN)H@@8&Z(Ufb-Yc;2+|*SJ?45a_;Pr83%EO>yA?p$3DItnmE+AbJ96%rsR0Byt2wm zWGai4V=R`E_s3h~t^y45Zpd{{mojA*xa1NW==7TyP+uW5w zjio(4$i>p2T1}|&Lmm)>-#D^w?Y<>K+>eaI+p8K3aNg ztQDacY(eSBW2?KT@AgT#?M?0S5!qTnnZo?&W5Z$FqmwTMfcx}>0zfI30>J!|jvkxK zPL~3}`fVKr0Qi&yfZC+svg0Z3CDLVsEJ^{;SG$lCEzu(Z@G7UD$M$-BDFDrxQ~>tVtoe%J zxKIr|g^WZJfbK{n0LK(Nun!QUky}N56^a1eG%I9>MwMiM*c4=dYP`8b$^c#4deF0Jj%Po|gLgsrbD;Prj%>Y)P+J3xv`4Ssv+44$iM8cUZaK3VhizwCVm>p>9^vXqyu(Cs|6`q)jw1o34?x3+)3~4`sqx~7 z)T{&*vKmCPSdHE-X>&5JQPuTVw!do%^99W%lGP9|$!a~NkvX0%%#mAHYrMX^#x&}3 zcD1tHy4TaWxf4rnXb011cy+b}>~a7vB_J8wp`)Y-s2eV!n-smStYv4GY)2UYz$FfqYX=EhX)F$%de#X_~}`Sp-& zgDwb+pctxP?+KVL1%bnyKowM}F5u)|q6_eLMG)v1O7EUi96}CA=iw$^psWi5T4)IZ zWKs$O8=DG(4CX}20Pf)sIhZalrpDsPk8T@>0lpnRUX=jgW#G^3atS}45ugun>Q41| zYtZAE0zF6;LoboXZ5Hr2HG&493U8Ez)Bt#a1RmF>xKnbPbeafuJYfLrhtmsVS9Dkh zprplbOO72kH0ubv^X!1Bh~0uh(zA26Z;UzyJ6n&@g~Gdg^4Kg2%3!nQ;jvVM>u$}( z!#flg4@ZZ(cvNsL@cn$ep2GHAE zjNA#JD<*!ajy&T~7S8k(O`IX93eGq>6lVr2hvNJiqcq&DjL}}9V$lRcIIAp)?aVR$ zt>W8cw5RczisvE~j!q18IL*IQ;*bNc&zp+=5G_TCr}mrCfY5$Bs^~F#oEkxk^96<$ zN6RBEP7z6p(lA`r!aRx|e?3`V-dn3y-cDS-3ssqCBeWL$Ucsm6(7uW-72|aqV zBlNg3vfm)|&w@NYP9jemkN^+C9~t+z)A@Whzq-?zkE#}CF(RAwXp2gEe>_DUXC&x2 z5D*@}#si&??iP^4m7AqIULZloffRHc92s=X$4oD0MlAl<>CNn)KyZlX#)z>BJ*3$k zZ=qK`H>hH1&vD-<_8eO>fjh#U?_~;o=L|Z&(KP62zGBevX3e1U;!`#El} z7W2PRXwsT=gACka(5~WK&aj}K){%y|YEZ-96|Nh8OQxjA+p zY>VTtDZXBHqnZBCItYvI2(&n$8mh&4tBktv(W7Z`3Q^;c)!*3uuE*I0nlJk@avhg96osh?-##Orsw_h-(Q~AHL2F6q@JyhJ@YVsS}?HUe?1HI z_q)tUyn{xEnbM*6CnDqRO6G?7uefeADt%`QVktE`_F{pUsXCq8jhH&UmPTEtyLIB} zJ{^YnUc;R&{Pd6w6+g_dnEXyhI+HPzjj5x{REQyokDsTj$T*bkrFkrq= z;B0;|n`65UIOe=BTPyH%dI|O5E-Tjx{4}}$g1W=HIysr#Lk6?~pFPOuP710n$%XdP z3|#8mK%X#xSzBVSMIajf&nC2odz8=~lw0H^(lrS+iUUm#?V&ZLu|A&PdWH2zw!dpH z5h55L^7Z5J0_iTGVd+?(bN8)WMQ$QKe}KR6c5?B>ELZztay#Qn3l;p_iYW_y;1?+q zZ%H_m^?%pH1N2q^J;Un~`tDI(FgAi7epiS*u`}jmac>EET++g-BTvz4h7fXU4yPpY zz&J&oqSGjp2zWNH|IGZ^>gHzlm$i;2mz`6xB-}7ZbhJrCOD^z{Q=-mW5P+I7W|AgL7mxnIPh*c`|HvO z<((q=!r*oZVpfY%AqG?$#6Vs^jJ}l+I}VKiaaC<`nZ9rrw(X4>IaOkT%=uXpWSmn# zM(Z_@ak%U;Kx~GB{2KYui#g4`Htyr1*QJ?cW1Daq8x>j?5vIv8 zDDD#EB*&RQ5^$Wc>N3Z1kK(1{gCdS|9a`wC;pYQ5E(ol1oX()Zad;-uSEE0nIBU4B z)#VY#@p!zqZyx0#;KxIi^ov{ozZ}-{4tX5L_byySaR|Sl_{Qw`Zn`DRu3a0H0%)0W zo^u_p`D!}Fada@nop^fn-sqKdI~QBw#tZf?-&3Q8DC+^wu7rG5y)vM^}#(oP7U-G853LEBtKIonAa zZK*SaAyDC51u0c{k4pV?@uZ)|bjDDJGxG8PB@QFeYG_@FL$KOQ}N@^Kvl-d*>thn+Ea|92r4 z_W0F5lE+0;7i~)M!bUJ0x5={n97WRtAT(*2uvm#algxL^*iZN`d z1)^9D@s>&e4Ni0bj%oPgR;dEqCK5qBmf#pF0FNBO3g9W03P7(&@HtFO!Iv?)NCONr z0xT_S7nh!1U73D>?dVtBtV&k_jA0u1?{cl2#GiISI06uUDFBR6ClUZ3i;H#w!a9EVeK*f^urUu}MrvF&)_3CnJEPdooPk>DG1WLZX zS=b7_9La!l@*y=4FT~}bJB&{*9h0)l;hJ68PB0J z2LpUPyEkn3tI21a8HO%sqQVQk+=anlw1gMW8zmZxy;Bo-?KLp2N~HlXw_G(5#yb=a zqwzyIOl>k0;`7OJa(985GtE)U?8Kh2)0UY=pG5F_yUd}K|GOTk;TeI6#xH50Vek_B z3@xD9XWS~yGu{eYX7i;XMDkvq=(urvC?`NS;r?*Iqz%ETX(hn*X zZMx&lCLs~huzJ!=L$Es{jehEb^zIgm+z6zreaxpCSU&}37S`)&pr+dewrGI(U8hGd zHaHY)*6Q;44bWLsO=!d;U5F-(=E4Sd6=T?wq;Y% z9XQC+(S^+zs<$}=8Hg_2t;rR3X~Gi}XAA`_*K2ETLFNBe zQ&$<>tesr1uW^U}voKwfs&1&uP;4RP!Sm(=Bn=H?5UOXOq;?v6A!TAFHZ>hl)n zQC#udBF5wC&0hnmigx|5R%wxN7xfR zBT9}_C9BQ58(`~?Y=754;}haTe%%r7me_N~CdHo9Oe16e`(ib}!c!I6zqp;7zvg;9~H>G>Pk6eUc0}k01aX zL^m=b$)8S=F({f!z*BsQDFKx6{vq-c%!ZoPfw@;UbhQ>FFOS1Hg`w+zU>)gyXV)q8 zq3gD>!P(9h$IrrdduolsDbAMnw$8C`Tq!XU!+qlSdSo`-z%#DVg5SKflmCY8JIA0hSRga`$MH%FmAm#N zK02V+yf{3Fl+IQokR!8xAiqP}%kqm6NIjXXP&!bha&B<~_r*{FX6fb#VXwcpnUmmu z)RvrfKTzh>ZbMBuZ;EKLr9kfxjtYpTezJ9p4e?XSqs z7q=Gn%zC;vz%T3Z#%#w1y>akcXU>s3DLO*)CTUKH)a-Qv^zr!Q{mb*0f2`%M9e49C zISAj~a{4(OscoZH0p6#M)v7ADR~Kk&5}oEao%Mv^X+%dOibQ`%Of{3>_I$E5Gnli9 zEtauvHt=ad_SCM@#0E~I1U%j6X;EUv#YW;H$` zKIDCNd`d)gXv&o6Bu{I!qwyN<7(n#X>33`S!VA_Oxf{&%%oeMOvUf~J7YcLU-D3ll z`|+HW=7q<$e_%$_+3BV^&#|1uO<9ho4Q2VmavSCs*VDUuYw9Fd9iXh$?ZxA0Ze-Y1 ztZ~_9bo$;F^@_lnQ(>w?rx#)&y;eSr4<(R=6o}ssUsehW(GtCRX(#XeXmZD;;o2zt zY-+9?&be)zPg31B_GE;vSLZvd!r*_@ZX1njpw2m^CDl3ox8zl{mup*qKX z4^ZdMrSR4NbagH?z8rx&?GkR4%pJ&5bq^#d%xFZ_{Igh*~Nl=l7j#nhy zMqer1CKPf^M#T{X*KnIfZJZimNX8df!8Hn7NNs#=%4|BOCbRMM0nGjtyY3j-rp^~maf|zuKo!uf~hYXDeJ2}^2s%vy{iPY%$I0-wx zO@ST4Ff#19wI`Sg|9Wk1(vY+0XsOC~e>@$YL7*k_=#hZ#xKYv_rw8NCNB<;rPjJTp z1^0)+4lUTDWZ>(?>dqAv;7!?jfzMG_a&cEb{f~0Rz*oz^C-a$=MVC=BCEvkIRjg^y zmyx3RvAzT66olu14^;;Sp!Ilu16n=QZxHIG@l`sF3O-G)5YL62DwA`Ea2qgU7~~iB zUB2LkN*|CcwRQ)gSEOUe2)Q23!-;;LA#}8WcOOqs@nxvO0+>SL%(WJ=#{-3@SP%RF z(_nAekL`{(4!YCl4RT{`pAw~C}<*QmZp&!U6r~z+n+w^ual*P=H=>svnpKx*&za@)t zU=$K#<_(=CSbSjbzeb@%3IF7~%h1&$X<7(GKYx zwr(alRN?54jv(VzwjYN6i)K{;a)j~rj$JhdqGf4thM`T6@eT!Kc$*r?cu84A z_XGLZ#o=(j;%uKUkp=|>2YK5xD=N(Bc8$ttgJ3cmE~ zG6Ve<%2|IK_bDjj=unjFwd}(}v1b>PId)0aVLn`Z9*A+HmScU5Ue_?jVSzDvLt;#{ zM#7l>l}L}U>oB`4L_=M}pqNC~A%+eg3bBmE_9QY!&6BycX7~Xi5p$nZ1LSzv3_T!0 zrnVXbWGpp{LEb|FZe!q_b-3{y6J^()WJ(2=WhlyZS(;%)X7vVh!-P|m{;-BsW#@|- z_PWCMCt|xioX+S|TV>;9q_UN=>QC%7g*iI&oQbqPv_dQY6-d8c+}!K>I#;W+6yNlm z9jLHzyW}zsjKXC+M`&yORy}K8XJ~4iGLgkO#Teu0P-JoIP>&1s^1s@Zc4u5ns5)F_ z2KQe8?krdk;!Jck+o|em+-WlyCnE+UI0BbgK)lV}5)7E1;)I1WGdWD4^#K$#iUu@Z zQuM7klpBgP64vI68*68Tv+D_0o(;tsEmgI3vo<`|*623jd~I+P#SA2gb!HgF6$lpA zPw^lVYr>Z%tv$f~+JgZYiuKFY-SqR!$U)B(^Dq+~xa>y-xGBfi0_>w-6%&>GkM*oK z+7wKFrrXowHOcHJdR<9&AQp1go16}(KXJuovmMa9c4k~od!wCl+MA;@(i;;vEvUaR z#1#CLn_jYo{S(j+Cs&z3H}*dYbW&))Ozgy(*&ZPM7F}*&*Kgvu-F$UMjpauZr zE*(X{73`BTP|@jWd31WmwTAx+HKNkf9NFX+fn%pxgw4P;p?4Rt5@zDkBjD^o1q}xL3*n zMK6?tx`%Z+=%5bp+7{&cGuz)a=y*ze$S+n%<$&q*sT^=C8q+GEK?9@!`yyf8nHE-k zf!$QBdNB61Sk){%i^pzNW~s_Pa}*^0wZi@fy?)o9)zd@(xV|$H07Dbta3lfM<46_c zL;(1fQ~?!9!*g8(sLa}Hq%Oe6NL4^Rmx@3o!!nU7fP4&+0atNZ6N1b)C-gV*H_m)>tW|oM*nxAyaeHmL?F&^CyR|D0|28W0Jx+DR~G>C zw~-Nve_D{8K42;UwZZaXxd{JZF2VDg#R?y$OsyO*W_Mi7qN#hR0F? zVAf#E@KH4TOXNnl@&cL!98H#%1Z1pxlK~TR-;GlZ0odh^*IpXxpr6|aBk=H@eHKGJ z)wt^dz%f)L0d6ywKmoAIkf{MWhyVnt#kaQz;N_7B;7BS0&UH3r0G}{G2^`&EzZHDE z;^DnO;Tb~**b=}kQd9-dtWZybeihwWi9csK0|h{~*S+dFuhAcdQlUSNM&P4!1^YA5 zm&s2LqG7g}Vi^4IJNTW{deiqg;2Z?}~`<0mTe_6Wsp;7TtWKwH>6>>{=uqDguNn0SiA`fX+>|@f%aPl;61SnEe@brSy9BrKTje&;vbc>?&UkhY&Othc z2Il~4tIhkx=JMIHtGhdc*tsALd@dDMo7lAP(pcl#fYvxYp0P$phn$82u{ez@e+FX> zCDg(jF&bY#z)3rrd|pnMhKH6}x>O=Ga8i*+N4n$xDr5a*i*`bA+D?xCSy=4^(@(bb zWA({3!mG2dAg|_vzg-%n&JDg8A6BvDU^xQUQ<>!i{NHuh4SgHXn&w>c8qexztf3$x zV~sdx$ZI_6U&d?jK-y~6RC6yyli2wD0mS~~Trmyw7n9}p8!OzF10Im6xZAL!^%8Z! z9HKrPbasjydEv;<0y;iLf{tdV{@M7LzeI%9%$Cnpf-GrsyQoyccsJ%k)5O>d?U!RO zwp~q?N7wb6LS-JmQ0LqXO7+ols}p(7kxcW_h+djIjiZCTG_3t4if%*?~#IG^tqs59zA+?^8WenFHf;vYfbCd zt4njS``d*z-Ya?j9_Kj8#!E(PSAMLwP;u^FF#PeOM~@zF8NNSZG3H(}1mEW)sk0Kl z5Jxq?kQbNy!r>IpMrg4?(@=~(mOmTD_%4aC0WPmIz{M$3edFA?$l`%RwfD)*%BlkW zayc8ibt+eBsQMdU8fkCGUXWebXHZPqXWFMR#&kPd`z{yp`7h7>P<_W<;Lb!P3t*12 zOoW}kk#HA{7|LBvAr6K3{q!$u3%$3~JFaK!=kYUUsDkT<0&^^v zU?3|m zA5E;x=MKX!e2yvyO=f&622QQxdZFHdu2k>nl2Oz^$m}4mw<0kt(!kBY#0%VTGH-$1j zPB>`Q)66K_hIzf9vgvW*m5mRxQ1i`8I}Ngrr^}1!rM0nIX1ctLG<25=G<2vt9^nku z2$=DEGbrzGad%GvGpAfsE)QO$1{oEg*Y(yR%0syfbw8BKnVs{VTgtjLiso`z6-e#@XBI1727ElS1U|l`MYjIR_IK@b<0%zR$Ou`;4aud^BeO_#b9BcV>Jvv;HiX5MvjZ;No}-|8yj_1I7mU zKSApw{2?F-{=iYepB)+r02NUv0Jy>$hv8I70C;ypZCwDwC53Sq9obX>SaQP@fMM)E znk|tEXXLKGwJVS1@>?@?W-=uv_POj|YJjKYXs{S5tuaEI*Hr}Y_(%lyXo~SlDwT<8&hNqu7k7MqMUpenE&g1DDK!8n+!1}MM`-{Un-SpK*yE#1*bD96oBw!$gr4=7Qs}uV--Mnf zu0Qm@O>U-h?6!ahD-+?lv0J9j7|SE6@K@Bs2L8MB^(6oI0eHaw^_B=>C=T+T2mqka zECBmQN&q`qNI-`Eke`wW2>q^$0B~+KgfsyzMN90p$+^5#23#oyQwW9`f!}6xga5xy zmnaE;3HzYxfU|7jL9@h6HYA&dJ*y%Nenop-Dc1gNA+ zg3!SZ04GZjND-+F^n!D&l4r_6GxF@WFV80w7Bu(e8NSH_i%F>Z0ECOQfg9kXWn!QY zaAg>W%^M{qtL(}1#E#a{WLRT;#w4Bo)pBKVPKo3|2nLV!(IqYT{layq9zMOXG`y1q|oPFjaF&uDvmbcy7G z!Tq&;%yE6V@3x1uMtt}JCvg2J{$cv<&)M|Gn0Y};lr^nx&%K!?M>16n%xp4h7{Hx^*Q5{r{(_jY0|9i!p3)Q2uSza+`Pleb`V@4Qz zYVYwb|D#s&45bH3zG6$ImhUmc&LMk5brx$0sa`KR(A_!6K}pf$`%7FGmq@oh;W~@7 zq+CD1hnQT);|D1EUl;$>1>oq~EG|@LvBvOBZk$bTb50$S8A0pmoNg5$-8cM- zc-`56)4WB4bxCRp&PMa5oMigcD#>YloE)Hmpedzc5llwo;R6_b{M{@UnVF|4ahfSV zs#^EQ+le~IWi~e4N)x{&r_nv5aC$r0N|V!ns@cwNXoqs-@PF1m87>W6vbryWkrxhh9C`6pskr&*V2%YpAE3B-cVXZq zE|H2Gr=;S>fmCs$b6RYpVHa)~ptuhw^Dl<2&aGOtjkQ_n&x=B%C<|>fRSCyWbvaAmRU9hu8$_ zgia(kL%{#{FddB@ndxsQ7hf{@6w4*!uBvvo$zE}#hpOP4po>EHbT*x%jezhR%Sr6j zd>TlIGKqpOiYV`%9&jG_S$ZC)N`r19GhEL@3ran&#@E<;Cj2!q`c{H-@ii9v(}rbd z_t*>FK)*wP%@TrUIa4bH&WB5Lp3yCv^5U4eCqr!ed*;@h%;d2K;0LuGYFsMew-Ls4 z)ghiX)Q@ADO)CP;OFMb%XV@y+$eZD$`f|2-^kVwx{o>AmyyUZ!jVkNGjmWKYc;m1b zm*Mwj(B4jc_HBEa8t^o+b_%HJbpfqwO%iJXD>bNjadBWa9BFv$UuIUY<<)dyZqI1z zbndvIg3C($5xU-fZ!;ml|5&?lkdZ*8uLnVa<2y!B z>G-f#>r2-i{Uud8?M>>t%hf88^Nzk4;Jlwq?y=U?DA9koT+Hw4tvgk#ULv`@-*O@b zj@;-5L2~riW=Ou9u0=g~VrukV;BT> z)?Q0C&zpRw?jAe3Qj#3K5pWzmCcQQsNEnV|3d4D;WH^q5<21a<^4W0Z2n5GdEP~^d zBsjz-<+uAnBLf;w7(nl{+u1$p3TY18`4Sh$$ zL!YYQHeP2$+k97dr!?*uN@?6Wl+mY)|6al6*1$cT{5hFgE0@aPLd~e)`s0x^9_wxN z`KI(X^Mi#e&a812VA^*JVU8+Htxaln_QTga&c;(2oEnPp@d|+!(>t@3cF8mYA{AkX z<4;9j^skC8zS9C1r+Q+GJN^aq#mbR3wKC=3kQCXzlAsbJpL9wCiS=67Y_Hg;D+|7_;)h9zrXvuT3*2) zbA|kl-kksb%l|>SDgW=8JM8NFoy?Y#doEdR4~Z+|LSUa}H})VtllE9L6HMIvDec70 z7Pn9qosM6M#w5tVI`3-~q$8cd!;_a^u^HGb5@iZ%(YX+oH$n zq~eG-as#{;+2gJ#{zo0BVRBNSQz{k@fPxMJH7p;W4CC+)Wf;dXWi*cC^Ta?7PqQ=& zE-j7Z@Pc-jR)1&vyN=7ib&L=B)vSO}j^y;+(s{J?`Jzw_*Y!CI)-SpFJ@>@EVHtPT zU%*y8KusIXl~~Rt7=2joco8H}AgU+*eP*nTl0mVv#pDh_a~76VT|zqb7R$4>zIB?z zIA}Cq8zLcpB@?;r1GMl|l-)Q#m;giVrj>5vL-Q^>U)%XwBmh4}lwYlh=M zK=5^E2K)v_3BA!#3cYbOX164_{6c<{cKyjBxfZ0{FwZ-%ox)Q<9x;~wE*6{cN8vbPQ^(!N?n(KDHxV(Jy zo4LP3$$cYwD#6jy?s$RVfNWC)rza%5H!g=!y*Ia~ixH4#DA#qlMsEm&GDw4ysL>nP zQKM(QH+E+WeoubSZvHYBo9C(x+`GBCt4D|tT)4XZCm+w=oWJ?_VLdYY#e}#y5n+se z7fKx6-VtM{%YF;cU`7PQ)#JzJK%CCIrNq}MTTO=xRCOf|$<}E)KV6Wk))|SxQXtdY=Sy)JpDg}9EkVR;YZh4MtqM=wan!zIYnM_$(XCAlH4C}N$!+4{gQCm z8GcKk>5z^w=GxvOt^(-&U31LE^Egk;WUaC-Z%2>ic z1)2V;MrRxrD5s_x*yu0=h<>KJF1QvEzq(I@ z7>XJvMeiy>6m(JSmB9dXbjzqJHtLq4TOzlNiVfVBeiv7(UWTTu;Yraf>BL zI>pZyW{f<$>Z~$jXjL;W)>>OTf1V;j=yy6z7Xk#D&3@asmISiX30ni$e z1jKdRxbcVOnd4MR1e`;WkE0Pk!!vimSW(Z9MQ2hOpjTaEjuL3hyZ{vbif`rQ;9?Xoe+vZx@3VCE)fAdM-c%klt>0#C9+Tr)SDCO!!z7o z5&~z6@dRI9Ktlp{ zl^P-F^-uyHCrW^jb(8=l7lQHAPyp&8E4lzcbVm>X2&W|gc!3lE260mXU=ajU3Do`5 zcPZlei-r~@`iKZuVklK64aF+=ZQav5J0}Xv#M?GwY z+1dLt%)Yb5{4l}w?QMHX#g&*b!f`D4XObPC)t&5k5EXlN;*1q+S~2JsqdA;S@2nV2 zZfSYOqiysAL7gPJ>&?KS$MwT`E@@8b*=U^IiKqAV3UPJ_WopJL zspH1kUvLjhYjMlVn_CKmI-D=XY?}Pl7<(wR2z%9zV{o(?JHEHYCD4M~2X^?C6~U~) z+`6u0!!ki-+$Vmor_df?$I)>u_{~c@&8D18@6EDbE(&je++ng53dg}2!>rV0b~tm4 zDwH@nN&*c=>q;CaY$Z-^#Y&t237{);sT%i|l_KJHa-HV=c{ei#6RY|JcLQww zjqUH+WD~dM5^2TJa0xxMV~RW_#6+I$>%Ys&s}<(Ut)hRKv?m++s@X{?*6gmVzJdK7 zGoB^(48R4cP1Te6DA%|Mi$<6;RrDqrBV>L^iii+ z-H6T@q^*}zG)|T9++1^#**7#wv+%sQG)R+m_nVh?^5ilq{^jk=DmRd!Iq0G)_{Q$1 z(HO=WB*)PiA;r-NlHnjhLU3H95S+J0wq4R27g+ShsS>?;ccZ&?dW%aWy>UviTPHVl zh{bK3!)yD-T2B@?7mL5#>axC*E@!rH$Iyg|eKQ@KV!zFRxCEc(EI^+D`3Qa7G8lep z{7}79br^$mh;j*k8ixY@;u48JBmI#8K-;A1T~>XITszz`KmtCQl|Ii`%egs9&k(=a zu0FQ!tO913_ig&^5`DHcK>q;vMCjv!1brM+$Pc{|k)L}|h3KR6EcEf>68e;#0{R&0 zwU5->OOHb%sU9DaW$z%N20PP=k(JTZP}xo_|=G^}wEgUSiu!E5A9T z*O{xV;!6ExFgqQyk$G({4c!6B>+VXOHc#4g+#9l*yfkz=t}L+{ylg33e`EW*rZ53& zh%Z^~LPTNtW*WGZ)igsb#u2OQzUj;A{DRZBC=F(+^^(WlOr^!TJrw|z*+6VF%--E{ zwOPaZGYm1P?9MJn0!Svm0hMsz>(DDq$54JVywrlBUadxhw# zGzTFfmOGFs*CO(|DUc7g9CRVb)l++>9E!%$;!ONBVTLR#FhiP#!b}}E6z0>(?QCgP zc6`3X**l=3RffpTm7DuoGLbmK*t=!h%r5so*5NluM4-!=Y9)0!4kRubo}##D1SHbs zAV;XoT_@_XS3I($%yCJpsdZ&8E|JO{?T{`S$0n&VXScRMN6H+{aU$0*84AUk|HOPb zo~tZZ!G>5)1N6q9R!5~cl$ixTS}f6*;MdJ7OrYx61K9|BKKe&4RHAGdb{vpK&}+y; zd`;llyu#2Nl;}}z>g;|C<+P9Yrir}Y^7lJ;DiT`**Ew5{V7n3Cd z{yiO}4^KuFt#A%|bdg=Y4I} zvtbx=r#15Z;xNb4<%L;#Dl;XycNS(Z=_;m&mah zL`52HPFJQ*8@r%Y+mX?(yQgoy#q$enkIN)zZDjU#WloutB5qtFrV0TRqo>1UjfF z2Wf~Hjy@V?V1`O#vBiG+#4Jxl(_<;}Z24E}=2VfRSwk8AkOb+I$!w0rZaT{EGxM+| zn3-6v!tCL1nXdX}xGaKsB$RiyoCyxoEp`t~l$ji6q6{jUdR#{tR2+=52*32xG_1#M#6tS_ckzH0Rs3r>m)6 ztL?n=%?@Z%;hUFsTIqn5$=9>#&E=hr^NT4e_FF4jvcv4kgxL+IBbDt9^)8bq{f|1H z%3SAQpj~mGDb2PTZh_bhg7D}ssnaR% zAZ8Q<|exP#)CYJdIacL+lK^t~Cj~I6Bm1(=i>W&OhFLUSOe$q0AR^RGP6^ zTn?w0d2-E($a1400_W~ThTzys8E`TxLa@8VIK3e96MN2k7u2(W<|X)dx^BwXsEQ+box_UO+T@vb2P^G zLE3({`Ym!`$zyGZeK@EBL1hlFt7er$OKy+&-acCFf2?P}x#l{+zgpK-4x%i~^G^f- z9ia$-n^ln#z+h)60dzAh5dcCZ5fJ#+f*RwE&y5fHvSIWHguBLNX{iV>k;W7OC@!aP z`iTI>*k9~a=+nt}kXf4=$%NS&p2y>|ib^rE;~w=r<~R5swFs~jYA6A8VWkS7NF@@W zs{l@ML^=Qoi&6w~F8jtsAkhL((A-o6DhA+889*Nnkb&dLXR{2r?X71dz^p^U3crQ` z>@cBJ3IHDy2ms|l;!m@e;1A{~8i4GO3LsvrN(OIqSsV3tVkKD3jRTk z{}k~4bE8O+m>O=L+tdT+`trY7w;1!E{*r~kG>dy{`2pjCrdn4NqEd{p;>nEO&MU} zs7MB!YfohtX0GG+W`=UuL(y5=U?lgXGUKnT8E(1wv~#Y{y9ehWN<&?IlBY$#pNszv zyJhMqAFY=6=32MXXuYW_6%!r8s9}s22#j%=#F$MSBOZgj&LL)E7)P_I<~~qTErfZ_ zL>O;S264D!D8hW(P=t>cOYF|0V|+Ye^j+2BWg^?rS7VC!{SSj*-8~;px+}^ZoH?}Wi8Ee36z7L9`E+u1H8UnfkrZKW=TQb4%FMwTa~ZtRV8&qq zGk$l38Ti{UQ#fQ@NQB;4fKHGZP^w&ZNrpz`jps~^U2Db+z2S?8Qu#e5%dAxhnVePI z9rr|yk@bkD<8k(tbjE;7;EAKV(3OEJhSP~BhGU8+hNF=u#zzOMQt@od4ab!wM~rtj zURM2$?eE&>##7=$ei)8Z(h);Hq<$DQT8nKIhrtt^ko6mcAFl2=3#z(dvoDqlUNX-d z&Z?9~t29cZU4qiMY%`SJ-NLH`p*a>)8O?k^OKbC-$!OP@gweQTsMfAG4)IG*`uSg= z?GnojbcA2t%z%d$V=+UlOi5CaW$I7_t>^JJ-Btgi4zRc&Xj?Rf`Ph=y%-u?O4T>qe z#=|3C)13*44Tvpb@m>yZ*{pjb{YF zc?KfFst4KS;=xD)YJT)hWc!F1``EzJjmN3ts^glLsFjs2g!ZPLdt!ary3#87WR*= z9W58DI}~&@;%ld_LZpg*$+VjRR(xnA0nVi3urTfD6KUE3m_8`OoqfD@q&`6ig>B!n zH@wcRJ75IV^|=;Zfk-H0Z|Iw$*8L$ljPDnVuT}~p$7JU6k|N3m&pUz)btpl`eWQSk z5m|)sju}AXmBxt7t5XeNe5wL4S~L`3+J=F-5Gd!0A(=tqqnp{Ee*!1Xe=85VK|58G z`{V7<%+Z&MD-4n*G#(4#{j??+<0q<%J~)37kSG|VjT*+hc#}&F05Y$ zqn0t%+=5aCxY-H0McDi16{G+Dho1}=wwgEBcF-GW2!j8!Nrga1Dz+N^7};trO&ZTw z^I#mt0d|BMTK4{un+IY0`djLkSRZ(IHMJ^&9WK9{=}sjj?x`F{Upgvr zT(%h{j!qP28pczlHz6?;IBqd5H2IpQIH5HfxpBBiZVJ{w1%7-n{btU9yj@uFm3&&V z$1lbeDxsjyol*(T54E@eGC4I2yBBk{TCS9ys1o^1yM)14xZ` zM-I3~YN)Ul-^l&O`v;Kv#d7)$4^`$i-=gxLYO(DgYLl866II0i|Ip#_bhiIp2To&+*AjcyHK>l`ii#mJ8 zXgjm9O4(EMw2t9NuSaxu9vTh{PYuo4j2g$D8juB{$+zw4YzV8R%%SI9hj;hJ(Warw z9N*EBJ&mP8k2yNia)-XO>P(S%g@+Y{ce4Ex&GtH3~SS^RZxfoeEDvu zq_SwjGO9i%@XEKCnD+j=(3F;3=9~=IlqATH*AA6SuskQ1UTpCHq)Ob; zHEM^>%`zhoS?I=0C>3Fkd~16gvN6KAY%|ISksb7yzIwW%}^v^|8GL4BgZ+r0VnVC`u@3V9hHyNH3(%oKngz9OG9rX_R|W_KBN`y! z=5T=EbfCUI6}XfJ2zYC0fPkm8m&gHvvl?NX#}&*;0zFLpX-OQ75by;~-1?Ts`fV6jfM&BjoYR6=JdxX!TOyo7B>;>-hX12apgi8i{IIN z3ulfJR}2^pB{;vN#XcP)!R*A15*=nN_##FkZ6_ zcFf>R?u4?5BD+0nsWF^M3^ndi3^mPIWT@!|jFu7URE)35S-TUlRTh^N47Hv%vjIbW zwzyn;yJJ{H)8d$RHPJ6=akFrzG17932(&mZ+YG<=w&)WIejgl|rrH&YG8oRKLZ%jn zcqvme(8QV+hj#nEH7q=x@jQ4n=EJ6@^hQwSOj3`Z|*&9c;h?i@>8}d z;JC6?CU}#O8~`dLCmfRGWK2kM=k4g;IKH~%jSB|rB**(=-utv?ygQnpK`YXDJy&42 zFh|}vzQ74wKX3f`{1OwdmP4Lff?l@bO||t{)iYrs5(8(yU0zt2UO_?X$aFl?PLrYm zl#3d6wD!Ody{ajH{Pa=AnEQ^`;`Wdnh3R;+#&oE)BGak;TM_8zl)t?FytI~@ zX0|{oX>&6b79FFZ?D{0o=0HwjOtjSXNqC@*F-|5JrzK-D%aiaOroRS)n%W$n+Z|(k zWf@~?ih(ojzb^iXo0Yg3i2faTs993e3(TfGI-K0W(>wWi_U4@PJis(D zv_X+beNJPO>T_mlwm?VvoawLw6#604-R}*1e!iFw6N1OyGmBMuQxU8Yi)UU2wpoY9 z|ES|@ZpD8q15dyPDFUoQQd$N;-n7Xe&3f(TFI1keyE z0{q@o1h`zl6an<#00}s|#rD^Ds-^@cOPYFB3zXD6M_j5N0Em$maK@YxFKl_N=Bcp9 zQ1d%m6hdLjckJ@pKTqfd^3WpR&ys(-o>}RzFQ$K)5NBL=b4XS-neHjW_qeN^WH>$i zz-O-*n3oi%^VgB$u0W)uIO~^$6tC-U=^RG~mpD!Z+z`0_&h~drXb?F*WV-EKzLav@ zmC0YRzHi#if+!heXlOr#I8viIo%m{C2V$97u0i9UJuj&)(NA$89Uy zKEZj1Jb|vMFWXhFR7tX9TecG)$tS7&+BKFY)*MSSz9Wsjm8ZXJfdYXBx&e}+yuHu4 z{*h=%4u{0*)n8~dXr?rE4yFj8Svw|?o=%_BaS-~v*|jH39y540vM}sH?a0+XHp6>r zP~+vQP*aBnns@Tmt9vh^>E-7Is$|cM+PSg*JDWPGgf11pvua-Ib#O0I>vv0oqKQ#_ zAyeuYGsSPi(IgWCiHlV_SY>%__-;+3f9B|oyJd#x{y7!MjF~0b zIQ6g;hdfA%Ga|@2E|q!0aeQM0$4_q-%cWU@dbbUxN_4aCh1-6p;6DZki5d5JZ!?kZ zKeXU8QyZ!G#~Jt>qn_xqkR*j4-;Uv@3K<`l{(ymgn-GrSXN`0Yf5TTc@rNoLh5vGXjk_uW zP;%~WW@r$%{PffN#m(6TtLD-Stv$^G8BTP7v+YjdiPQ{6tfw!J(Xx#^c`c1R_FBPU z2_wIaOWgsqaCv62$wmkirzN! z4fU!@_paS{&h{M!K$7iAOIPMoB-t4rXJiMm3fcKx$#r_hv8Hz{(z7$Tvc`2>5{NvQ z>kt=d==CH#lh7v9+tC@?aKvR`P#dL_z0eJ?)N!UUdG3@|LUa%M9^-WXp*8m0C|>CM z)|^Gb91F!W5rFt9BB1DddJnM-U<@sNk1K03Ky*4~uFC-K5-v&NIh^Yw1mzSRfTD-^Ncm~-pz?7m{f-H-7`sv7Xeb1lGGbSVND)JOtay&B<<(;56YrofM*i3Z@U zqtYpSw7|!y3BczMbn+p=$0>B<_s8!jvmc zw@TC#9OL8vtc3L#+Jx&gB>z@icc#d95KLnFkz7Ymw>djQ13}HN$JXXnZ-auT#~L3M z_U4fd{BFL1i>q)8&^N4alHfBO%Fzd`imIn8B%M7b!eadKaF(vemnYEm_&rP4WsQZ)E`FinbV`cSz!Nz<41fW{wO>H)g2&$3*VNLYDi|nZE;=m3_ z%D9dMm}1U-Us-nwsa1oz(LKfhw-VFg6||TRuS&Sf+bfEoeppQR3wG+!NsgTENO-}= z`=AT7CAv5;3A#_WFi;|BXB44=>)JXRvu`dFILY3blIOWD!OT_6)8*ig2Z5*!h8sa^c*S~JeRy4m~&dwKCi}gs&jS=`EHFxu9i_yN& z-S#$_Nb^__w>hCsjS4$V)OvRoqde49_dOtfYZbJASkJzi`>GTZEQL}y=qN*R@3D8c zRNO8P78JMJH>}l)UR^VJ=Kt=3FWg7*zLioqm&c33FX-}zRo=7?MykchTs*D9NuOMM zHw;e$nn+yx4=8p8J6c_?(83vgVfn4LQ&?yyhBi>{uxtNyvN}Tv-Syp- z^^Rc0Dr)TJbOw=A<;u?8P^0%$8s1ni2jl;~^Z13s59Ua#fuG2X297C4gI|^YJ~$uy z`}mq=GH_~wj17OlFc~-{{e2wBEd~ZB2-PI^;ss{WzoG?&wdVC;`N`~DU;|GSRp7l9 zQBt)T@StlH>D`6TQKSdsIU0HV*bj?5q%udo^Kl%-n!ipH z2ak5Mm(e}hUzH0jrKX2I=m~zV=^;rm-yy1&rpI?{n%*^v?VxQRGKIQ;KERf$o^usB zLdub|<3Z#9UWBoqCQK1OtE8RCrmqE4rMD`8oI)@ECkI zu6h*?PSwEYw=39-OJrIPrzG|&^IhyZ>BQVC7E|9?jSZf&2i5ReYpg>t%zV>r#hM%X zy?r1gv1iNRNbNU|p*SS>nIz80kE;|nk48hfdFjj?b3zvVp>=8!z$AZ71o#V`lt>Z4 z+e;C6qRIfrz@$IUVSI)bVxjN+<$86C`zq`|y4aW=er;looj6o?58|FE02ew^-^CMi zu=+A|bR(>-s=y706kMVkHAg$j4WQQ?08l4ysK%_&(IuR$$s0UW{|q26u4Vu+AOZU> z?bvJS)ZuL5>MJyiPgdAd0+48vul3b49?VqB`{RM?-#0&i-xm<$vx+aTGcp4><@ltf z@NsThbaBJ6;J%$-f1WKDX9k7uvw8MDNclr@csJYK>9O5Pu(+#oDgOhdpr8zmpiR{9 z;u?pg`!5VoQ8QPlldNfJKG6__X_jIwjn7g`(?U&AK7l@C5I(`-Lfh_%_jdx8 z8pLFM4PyH3W0`!ixG@lZGyiHP>FY`wAgLJp;}s>HVN7K;1;z|!1;Q}Qs+M*a8fyA3 z2h@>CFji;qJ>bPyj79gu#p>#dUF$qsTe;Laz?Pc!#}k0vsFd1EO$G4dEza->h_~%+ z_~=hr#@H3nTfP<|1%Z0Ie)S$GJ8ypETnxGkS!;{jkF zMCB_OnW2aSD)^H!p2Cl0;S|23gi|=0gj4wLj}lH(io`Cu2t@Ks3X6mSlP5`=AI+|B z(6-%ZEcH89wj#_@!7naq;k3C5e$$=F;or~u6S%^uNC4alXDI<{L=8DnM1Um#i3~Km zxHOi6LPiaOTMMb35-Qh9(bifKH6&7s0bQM34xZpiOgZ2?4iBjhF&m3D%tlY(=d&eG zsd7+As4_vKOF`GzUy*{m@dgh0h0m!E%19*KY0^A277XcE? z6ajo=ga{mb{AljN^9FIbp#=&n+SxrzmjLKmcOn-6x?mE2P(DT_@2dnoSK0B1?2JK5_w#k15fUlz>nkp8>HKCQ-%4(ta{ziH#?1XD_;ZtE>0)Y zMZn%b|I3J-2&vH=gx=1pk}Yx}yGy_%zCLO#jgW4)xQ?B(gpP1p@(pjGx%&WjY|#Ex zsQ)gM?3G*neZIE*{leO&T3&{21685}t%T378*asf&R!r}WH_&MQ?7V=^@_pfw=Aqjh;<3#2Bnd>F(B*EiT(+uC<~ ze!j*j)t~D@Xs@@^1UkzgG@x@rN#azg>7U#noGlmZ{ra^amd%eeIq5ebbym1aNw%+txr1_?i z$b2_&M6usYbfmr;C>tAaSJ+^zc)_F8EI4V`Ip^(jhR^LJDDILT^6f+U2>ena6pAJm?up_Tjkp1zT>ePXf>3}aHIzCC7 z<3Pr898*}%ugdBN9EmxOj}Pml^9MRDAWI)`N-`V=a)MKhSn)Z&;Dl`$zp?+?H|*WA z#BO?;qyA=L#%E@$xrXXWg`LGLi8c3FZ?ik0|8Q6Kgxf^Oa}|q(p5m2Z4_+zQ^Scsz z`eZTq_<0L_JatXKm8GR1Z>=!AdeBtkjDTk$YabBZcn18s0lOBl_=;j)MEEqcIdlKlrd6mjtdZlmP1*TKtoW)_3XEh81MniB$<&!gxQGqI^YR$CE?oKJ~ za@U5w8+Jiq%Bncf7EVAsG_YaJt6g$s;E1-C0AV7dRNx66pGb|)~k5sx*0Qc*B@fSLX>1@APp^4

Pz1EHaK#c`}dqzkBAgY-FKx-%hfTIcij09rO+|fv1j)KRodFGs!59qd! z89ujuIMFi%fVEbAqO!-h&XfSiptCWV z@D=(V3ZSQ1$D0pR_pa_O&az`eDO24)!Gl@;8yAhZLDw8~$c>`wAqJYR$2(MYJtNxy ze`jHOO`099SKpSV2lwtiwUCy^_s0{Q`KGb|2AQ4$XWTZKINRM~9!EOqb}eK9 z{F-X7YY2O$rY0MBXaG};V}S8l0*nKb0NmueDB3MY0FwnTPMKQTz040n;3I6UHh0B} z8o)2kKaFeBZ_pP~n4s%DuzY3&5)wIMutTknExC7BcXflq=F9|67ZcO$Kmgz(D6gUT;n;_WE!tCRv+P%8*9zucJDEz zF2@Z6)rT4C)0_3;=H_aKvh+3Xq8RLg(``UKDz zI%FIeO={ly2T|pC6sgK_ASX4PxPWg$YJ6b?sZVcKt1s6&;Kz$A@K0OzUE?#Ja|oXy z4c~{)*cuUGzMwSZO{FxR$)Yq)nUr>~G9{%25{G;9!?%nJXX&U5WG&OSSV!lyg#dP!#xgv*R#~)vR7{?#er;Nu% zipJiCb7bgF=Ps{!#XF6S>s$`eH;gHW78(S;e|zxfi<6(9c|h&3?&WOse#!&2Mv5*z zKMcCc?5UlXfj+VH22PoJ!@bOudIQoDOvNy8U#%`L7faCHqx~#m>+{(a8v;c~o^_k) z*(;L}gQ&#CkR%SCH~w5E;{R?P0Z4k}2tZgcm=iey7_6R-03PCDV>$@Jzr+!EwfXqb zh3gvVAJ*7&_k3pMYi!(~dVo_0uxkR9dw|>AV?{#GZ2%(dxoA;>kMo%q2&k0Mj>%CO z+Htq`7=7H!is&JeHS{5uA%broGJI~scX3H2qUXcqeqeE9-If_c+dRETrQ?}V6Z2tZ zIfAC+&)H6u@tn&rD9_mmi92o*!A_MRVaI`{H3H;7Q zbCXX|r5ip=WA56oO_7D)-r~|J;j8U8=9cM9GFkSV+yvI0aTVSfz!IN}Tc+cIh<9

Od6Otl=cz#XWJ3iO~XRhn~bG@L2f!X?M}N}wH2v@UKC@gcJ< zz!^U#aK`olCeWPiw}37$O~M-}NCVQ*emS0dEYdGGH)!T==y1ZJ#BNl@l;b~U6gt~CUTCYRmIWX?H`jzw6i@?l(lyZLOwdvjy(*!IxeuB|GC z3vDB|Y2Kx&X1P>kt4SwmtJNh57`@$zbRiH571ht6f=qxVp|Z(aquuBQ$kNNz;lu5p`4 ztk|V5>>^?5aQY7>!GVUN!ttx4KqA-lJDx5m@Af&v=Qj4@FBst%e?f`>9l%@$sG3X}z&A$70GB41 zQ|m_;v-Kqt_A33&P`v3n>Zh?-Y18{%W$hVR8g0h)sFm|52S=|z69QcaxVBPK3ic>u zkrd#ImJnd%YcjxZcL3`$KsD1{B13XKN*@t$BWTJ1?mt2XPA{;L2IkTY8&F8F+ZLd) zJ41B~TJlWXZtmu#%>~GTc5prJ+>ycm-4)m1Dn$_}-NBxKH?a)3N@c?o;D;s<0--CU z5a6k$5ODQEt_OPwhReV~Yf3*zt&GMK}_^mO<@ z0Xv!_|77zRXc!r>v#eeX=Ggbb^7Z&7H|Sc?1=nCI&K}-zw6h063_4(#w}+7YW?}BI zc~Y2cspCu^MK}KbM4-IG7w;i(h8>&&W@=&yv(h7|d1U8ib|EMh)^`f18+8FKUO;?q zlW>sUu~73##zOtx6|HN0hGAd+^y(fxg1`k8Xk`tP1DKK_kcJZMi8Kb!CUlPUj?&xg#K8hf4kl+jV7-LOYEn@PEo24SIxp%Kx5z& zXI>>VMoGeq0h*-p8Kv#Gl~CRLyC%xTL>#~|VJ2{smOw}szEdK-lIm{LRCwbWqhDWN z&CF`Zy5^=2r4qV3k!bFAp=LCHLMtbsCO6gjlOm&WGSS^TTTh*H9IZZW*i1{e()hlm zx$joUb%QRD)se=}jOFuv79+85Gh4W6NFQ&^$&*5xPBs-*0cH%8Fxf}q+nfRNAKH`* ztJ$K6jA1mm69r_Dqs$k_T{K@DUzOn)GX$|KP60AJah$4o;=;{tA2N--C~HPIMsb#| zxHDl{&Xoa6(AtxvW!@mOx-w{9$jcO09h+S`8@9JG@ZWocmy`IrRiRNtzK3Cw0zjF` zBmibWlK@JO6alU!j77kWuDEA5U{h{qp!7d*|$up5FQ0NZR(E(L(0T%-V~JtY1#41Mv>;Ai$Qh97sg zG(ZpdiA#sA4nH0?TvCa}X$o`ni|v3~b__+I<#4m*`Smx% z(q)kW$SdRMYiIC7h&1@g4`~3NCiu6T7wHy&Ho7JOxFir$YOX%ty;dp3`mO+f7hm)W z>P|0a-|%fE>00ReR}0&jG*>k}#5Ym&4p=uLQ7{L)EAeIZtRBpfgM*rNlPdyXsuUGp z*M-6Wn0CSh*vV^FK7(&)0ON2KVA+s=7{D*roAbXhyE0qOv7MF%bfHSB5YdHpF)5G3 znLuQ(#rGHp`VXy2rwWhkdBZZ2R#SR4&YDF3cQ?a=V~R?rPEJ(1yMHvx#yLC7rb85J z8r?}^c@rK`+0y9rU!_r}y~# z$9IgOQ0N#i$#>jPr#ptvraOiu(dWBc^p6GInVUD#+cEhXArC_zuP0e`wk6I<5u>j$0>4vJVL091o=b#=?cz zf2T+oIUE35?u1Eez$HW()D;`3w~;nga3N@ z>I1|U{{8D;U!s_?^TVA?lEBHK=#th|hZKcNr1zf1Z+h7zH>xq=AE)VGEUxEE%&4Ck z;k&)i*^b^DGlQe}?H>OoZ{h3jX}6#GjQ+G5yI-gZ{v_ihqo4 z`;aNtLP8|`cofNg#*#Vvcib}?@jGrmg8fJ9)s@xM;KO?H<;w{28{EeO8q~*Yc`0Z$ zzr;O*9e2uL$FY8}OVshTd!Wv5OFfX0UR)wk$0>3m@1ab%G zGiy0p!N13ItJ>Zl&#gVqNW^g&`9A@Z zwxX>-ae7NCfELt>Zwh)|nfy)H+^z zgegDXu+6ik)$h?T(sJa1K*DqvrQ@)O*73P#w2mw7XdNe$Xua9ndIXV_;kq5IGb?6= z>#o|`TB-^tlyDu-Gd5bsg9O}%rRUzwZ_Z|ye_1Qs7F_$L^SZR#5@@Ir8}FDP?DaRs z#Q)j4Yq&JxH9kpIjsqF5acm5)$;|OiI)3yXyvDaBuW=yfH3lc(J4t~4er^}ceKHHF z2HS4LAQs*@B>~4vMu6k=g8@#+er&*}kICL_KF*h)4fB1EsTRv?!x$&XJ00#+Uo_K# z%N|CTKic9Pm{8^RMwxgjQ<2}9O>H8LT%EK!QrPcddi;<1?5h=X7pkVJGikjQxr3Kp zN!)GVA>oCFNZ`encT(tyHpdUE zx}1m@Q=~ZMBHaF_8s8YJ%kkB~^ntn@+bXS%fa~DA&>W~VKg!~M71*xoF;AE_TwW=a zIp+)`ZO-C8sm`H0dJ-ECl;l%b&=NBnWU~D59?T}|#xpx#bIxpH&h27>J6^y^T%isE zA?biNv8Py<_Ixq_go#sr?H{EkvF~T+>#mz@y+`Z;1Pf|IGa@hNtMw-<&MyfatB_Qo z^A;wTIZii8k;AY@5=W^KD=IWgf@4ackE4me@zy^`&KA*>Wq=c2zKV&G#E#B>#uqnrgKq-&F^(#%i`!(4o(eceLi) z#sNP-m-!rGqppG^GJ^_dxk^I>{Er`*d$Q3mS?nH>n#V6|ej?tDP#3#M-e9_gZ zZ0laAnzbXphJvW#%PJs)+6`+DCAZx|rD)a-UzY5K8=0{iqo`PI+&i(_Txr;o-uQBj z-jIYEz42h-5?M|Gh$X%Ca@nGnIF8@wN;AK?#*=9k70dHmz@91lgZGIXwtW2);OmLK5+cA(#Mt8sD@5plJuV5rwgMVHw?N}%(nHVg5XYk=*yG|{ zx3hvUSnyt9%X!Arh6NvxtzJT2z4T2`V@?HQq;w<7|XdZN#< zkySMb__*L6;6n@pvO4%eH%Q{+H4^YKemVF>!;agJFzjw8PCUE9`#1Bi3#)T*k1~17 zwkx{d8FeNDgmniSrFC}$sPzuUm9}-KzLo2qwzs$rTUhsPR?e%QJ5wE&ao@*n+?Z>= zJL_q+2^$>N%sa2HnfLqbrT%ud{De)CjV{pz`(B5dYgARTD;~T<7+PK+4N>WdG)Bor z8YdH^_qMp%y+FFKAYu1CG5>-wQJT~N%1Y^z}Pd0%QM|`?; zpw#7fI;qQzLU;(KX~^;XBXs#o6ha&Oryrqh__YCjp=Pe(3KJFmhN(5fknd7YN%%>v z$dES#3~w0qGNEnH@CSu@;qTVjM7cLdA3a9LEo=Bg;|FltrwpH4uyL34kay(TC71C! z<~=-4Zp_(z!xR7v*9ZaleYW{@VQBaQ_X`nR*RUpE_6RvQ@ZaNlK8e3uv>gjT-a&jD z#xxTE{D>j|t`UlK<{IZJ#v)LV0o=7F1JocL!F3tHUBV?&2SApk4tNYkr3qDhFdFM-F^0hAZWTa$YjCUcSy0Y^B9(gvGcoIR7b+jKi)UFrdE;)8gizb>KFh`1=UvAZ<*X8$e?A#1-q}8dG4aW|rCQ#~5 zM*SjKsnng!#de+AJ6GzQ&j=VN85*x$3Uj#HevAv0n*^J9Pk}hP?n*H|XM$8upO$9DBCb zH?hY#XIX|}f4sU~TDF|CGPefyr75?j%fe-D9V-G1a95-9=@W$DxDZv8IZ!fBVq6e9W)v@WwG zpf>wP^`+_nj8h^4&aUHfVc#)U(!RU#)$(&VVcU0pOWODSEye=UzVDje9PNu`-g(Y( zPV?q)PVwd-N1Ao##kKStBqSg|%$Wm|@nywb=avya{SK6=0|qMjhU<$|gc&R)5PQY$ z_B`+(+PU>)5qqxXsWf_$97;Ul_<-CEK$w8zR3e=kkQ~P;$#G`fb2ZN%00HU5#Cwq& zrsjW#fxp&8;6FEu%dcir>GBRSoiCRudZk;9M}3bOEB~Pt0miS9m**OpN=2{k5QEWE z6n$>$(?%xk4r;os$CK2AKp@;cW{P+rc*hS&A>d{Nxe##E98(BdM`&pE)VtNj%+Mdt zuFTD0Y(I~AH&p}J3V_XcB=~sI7<+)4K;N7Fu!GRG2tj}PnO@dpg`55Oh!^(H1PayCVPZ#cZZ>fPdOera|qv*Np& zhi5i10yv2Pv>qOg3l|T-N*B)!-UKe*&K51JaPb~*+dB%-9W)YlERk^ouDN*Ns^a3| z8=8y9i)$_(b6)}aVLsl$^~J|qd|mPK4s0*JN1_gzQPuHmU!cNF#TIAkAhQx?$bp`z zgFx7xJT;^ArVxppca3uhW(6KD5AS*8w0HJmBrHJlmPj>Y-a>}qjo z+ws*LLZEu`J)pMCxzd%YWi&)UKuuS@C)OAz8*564#2OodV8A8T&0{-k+xZFdVxf(v zG{FWm3aKF;D%kv$D%{WrqtRad_16_QiO_icVLku4S{razwKtwhf|17r-d02$IPK_)TPNtS^aBjG}R#(VSb*gHEFOQ1`d8&9Q>8NWOhWoXE; zR6c^@UM_A7H|&j>W-k>&7E-PBToN98&s*7})os=hrCPaw3@S3JpkThdryI^o6F>Y3 z)J5!vJC&$MVz_dGNQ@TN@WaVe>4>wSNQuQB@g`QX)r;M*L<|=xEoN5+)nA;SFRaxL z#f&>lqpHho8DI|7%R`%l!5Bn=ScX5S)DwPOnV=7~5`&KmtRM`hCIBC=Vu6nzkk~`q za_F6S`U1I0GH|~vmX~zRG&_#T^sg4|cJifRuUi%Mrd#=vuWj0Aa#jM*YU&92h5@ML zJ|`VB@G(Y82<{YzOu{fujNzwUw(y6f)bPh80o?XE!{;{Vf)_{+`T8m6M8?ys`u5Dt zBmRuVeVlXt<1p^O{C2yvJh{~pvZ1=|1p(+vC9)h0C$=r+zYlgu0btYNSODmLNdaJ0 zbtV813QYhY7YY6MA#$+*xS`dRKc20IKYliVtK(0-Z76^IfaE`4Emr^>Kofr4V+8zY z3(q1+O#vL5#@|lB8QjO{7{F(+sKu^f|RY`f?`g_(@;P))RaY_P?13BKj zfyH*58v*yx<@{>CywL%FH#1%OdYBIFrmB6Y)dYIy%hSNc=@`7uL;3VnEyAa7kcl%Q zHxSGU&!MB;fNME7qL=rn!jFvg=<#C#`GH~Em-+I{s-6IjR${ddvG?0j#SUGiL5yMl zUJyI`8ExNLh}xLmwjky?6Jz%}S-uU(kHwh&K*#Q37-JXh#oC%XIaz(2uWztoUA5fR z;2iI%5*cs$FGU%(tr#&$WP3wi*L}oF!QfR)m~|?{InZFHOw`hMpD|GkLd}UtU^TFFoH4NUw5w+CtDDB#x~HZp($wHyToM$%Kz%j#M18?;Fh z970T;ybp^E4mRrr(gL`zGJA94ww_QmOlfV6xjMe#RF|w0gMJ?3AihI zs!0H(ObHHh3E*O$*j2or1PFLrIZfflgQbUjZ0Kse#4fR{WXzq#;wov}e}n{_u9yJE zM$y<^Kof#eilFXJc!QF<6aIKe6F4`*Ddz9e*fg;+_+d56kK>jjfm3d{`}>OchGt-J zxN2Zc32=QiXtNr+FZLftc#s8KA(XFcrzWEdmVcYj7$U2 zn#9=S3d_9XmNn?fzAmeyL(d=RE~!9IK~$iJaWtXF6F6lv%(|a0SKn|`#k3#a04i;1 zMKRfLS6y(O~9q=s-g#U z3E3UScz%%#!Vo})Xw zA6MW!*)!|uoGv$#e0M8$IwA?S(#+=hdc1?iUL3B*VEnbQC?8`A&N9THsRgm^gg0Hg zsg%1-p$vXEUjajJK8nGdqb_|h`gWNk4qdMVVbHXRE0$(i+F~_52*W*Vz8Jlv&e3Wg zGkk8{Zai&z$fsFwO8R1SL~~z^LlS^af-p#ma|(ue;_qfy?TWi9uK3~FP1dR2xH$>i zZP%Ql&knMQZT7zR_G7%Q|Ih~DEY*s@=LHG&IFexxy`o@`K})cw|1ez;hMNUkb?C(< zG6=(aNa)!RBZr=S7$=7Q#r2I9f1h5=zFE|-8*j4+Ly5zkU&c2UE;z=kr-KHmw`-RdK_2UdK@PcJ&uk<3C7!xO^Epx7Q`dsY&^|a zh#_l%;=`2q^Uc!kCQ5tFPNS-?R+tiHe++qu&~Nfq@oI-jd`p9$rdCdsx%iv~#%z|Jx{bXp@SUm?bwmQr)YMwCTllCxRpI>^d!}Fz^_bo(3M`V;guf zFJ0WqXAoZ8Hf^;!SMq4Tw6qo9a=g==I6S}N!~rMG!bAS5P8<_AV@*6;57EXO`52xy z*k1GF9z#K@l=sIIoZYZQ-a1~YC(P8u1NME{*<$IHfSLPPs3Y zAT^M2qbkUpZXOHruUy!mX>(N0ejE!go@WTW7;?d4e3tOKmWoqk)SG-l?1>$Vxn+n( zO5M$-o0Fa@!cZV%5oQ=W7U6fRwYfh10F9-_k{H%$2*?!VI2Hp7B*?g6l35c>k$_8r z!w@Ky!o1GF7pF`DyH}aA`W=!n7TuH0+)SpuS*+0Gs?_vEyyPWuaWms5rHh!Hl;Y%VL&hkHs0Q&u{Q;#qBWQfRC08P@5e5 z*UMKQ@Yt9@ef{gppPvOZ-81!&0serIdU26J9DVCGs$6Rq=$;4wUVMZAIG5bo!>82Q z-<}`^jw<-9LbkwX9by8$)B)@UQwJa8qk+%wO7JOX^MV;mwTz|8TZ28x$O^*0;b*t5mH8V|W-<05Hf2{?32HVd2N& zb5AReD~DM5?JW$UaN~B&;uqj$nRuLXW37eT?p-nqkE3HPJmscS`NIjdX0M~MC?Bqt=hyhU z;>R7%)+UDw=~H`87Oo3R80?ky9_vE=hr4ML*M~XS?}i$z)sEh3Gr#Wt?kqU2Qk1yc zMnx)dw;OD%#fhk;#_0jn^te#c?USZ@95yUH3 zKdOuNbc-z|1+{GhPFrsz+n}Zg@*3J}EU#h20`kLn{cf?IeMBz+qeI~H)yCRx%Z$I6 zB~W?oG0$V~yfztHE+RHwP!b!bdlDPNnh+Z|`%#FEhm*v{d*#HY$)A?kCySNgvK=t9 zer|OaDL8F-auswh3_0jvk5uL$UI}yDsVC-mmIQO&I-2Olt$rM3j%Uj;$16;X`SbbZ zrx}seJax2)ydG2DWO5ynZ8+-mgC*{`Ho~3YiDn---4l1fn&8e`|ERd**>c<=3=`x2 ze)SFdU56bdsyD-{lluqX9%rar|7YvB0pt?o{i%*Y{5OJJrMgo_Mm-B9dU!qC^To{A zRPo?){!cJdORCwry(?Nuub)m$MRQL|+B{;)tNk$7jppyP!l!)Iy9oK#IPXSBP;DjI3Y>=oSu$3O29Y0T$zHDKbE^usT=!w8odA%k(J){MzTTLz8q)uLH)SL=T05No z3p5%ikZUx!=mFkI^2z8&x45y4Svm7mqj$HgCXG=HcEbS|?1qp?cGL5nq<9l8FK{bC z?|s`*brVa5qsnsw>5aD^>!`|pbi?TV4jp8n@U=|S!R6>VgRa3rPmEIy4mu|oPX_U0 z+*?5W7%E+jM2a=`5EK(&=&EUJ?DrrXf#3rHMo76>8f}T*Ek0XGrFxMrgGuHT^okV|HEwiqi4oVOpLakT+8yLwrsn%xFWsPR||)O@e8 zWM)Hwu~?s6piivD=Jn0RW(~x&j8aW!oBg8kqKU}Vxf^D-t z$A4)3G1tg7RPb<>6<8B{WMFB}AXDiU$x5T1DF!@tEycj3US*QsN$AnZ>t9|Uz4(Wm z%}q!=Pd%TToZC_V)Zy)2*+{M1T>vp$K*<_fiDd5Gqy#bo~+$z=&ibfU6Wmz>b`W2ym2R z5h!)6>!Anmv^70|9}JgBdw_3C8DNXHTn4zJ#gu_=N#`(wfUn1qfOy3HWj>2?b?H_4`qaa2Q>7gqy2ym?3LVc@>N9MNafoR@a_u4ih@)>hq# zG*bfwFEFP4?g=toayx{l_LQlb-7R{Mg2~%^jhU+1-YfMtaf>(2&Iou3h0U;XWAUbX z91Hl#{NrkKIk)IsSXga0v!WxWQn~Z%0?^(_H~FIKWHnCSjZzEZzz@Y>lYTOWgWB$C zuPI2CwY1Kh_$a;gDpKQ&yQGKwG90{ywAY+C&+WBL*CYWP1bT$MKAfL@w%U*ucJ#76 zcN6(;X^w(CS?$H(uG$Z;6ZvZSelHl!o~|11ogh!m5JOL0V8m?@rW*A8cEwf0UBV^O zRl}o5=sAn@U7&x1Eh90z1_3ZC`R}LesR30L(5(-&Q@yg8Kq?^cmRL?S(Y=N3iTc<`a89>tHfq9qJ`W*hc4tr@<)%M*Q=CpL+SuNzw1l8@)-et1Yf4GAl!XU86eOuSa zBhCq0|97{R8dqulTW-eF+?!L6dioWTQZwVOz`A0`@%ok>$1g|$AQf^6kgW(45#V&~ z2p{ffwfu?*;=P?dH?ou(KZ_Z zt3WrL)-;s)9&_VTnX?m8tjzcESt|1ZGpfp5S~{=Z?tZ3gb4W)`o70i1D0AE;T#^K3 z5CW;q88PJAoW%u!+KD#zvNz12pI$9)E_5+?k0ytJY?P;NNkMMV9m( z4B@&G#V{4O)pQBDxsX|6203i;J|jvuKibgI4pd-tQg_I)XpF=I)s z-M!9s)VDo|*<2}J#$Y%j*M#7L-x<4ctISiwJv2`Zia}=DfM}det^4NJn`?vHHt=J+K!46w8Afzg zSea7y-@09$QqC*o5l)kX&_bhA5_+n1qB9`9h90yqmO`gDkL>=^Pl=7;rr%A`cN$ix zb!Zjw`I%GacvwxX<43A$osS*x?|!tGTK^5(GFrIBE=s=D1D|Np`F z$K2H*FSh^+euz1ij=x+|@Y4 zosVd(R5C%`_awGUI!S+x3kIj7Hmz#KazihLOQhw7Z!0Y~^K^>o7NPef z%RS6CL2uh3FVwGhd6=Bwdu$Xf2~Kqo5xn8<+|qKh79e9d zMo|gKIHR1{Zuawx8SXqE-8YA-srlxDzIsUJ0;*06NQyg=i_6!!WGbgPmw^PJ6N=;9 zh?wj-Hrh5Q&N8Us_}fdyyXo*8|J}R~E#Ec9pWXE({sX2?Sn8MH&pmWw`~k8RlEW6% z@P}{)^{wO2A28s@t4a;eS*9F*XFE;&p#(!G;{Jfu2}Wx^7i9qZA^@pHry?!d5EkE zE*>*6;9jkB?;`(q>%-BK(@v^-k3%{WmDonB1i2`0^%xyJN0SZgGj! z?KmaLji=SP%}4b3CaN8`7(wkf;Pv&)>M;Fc^T{=nP=m7(z~dP$;8WZ&01@u(%Z*C7 z@v9Pd9Qk2!$FnK8PY(P0)zYqAUSP{cOP|-hcGI@AN;N^>`Rg#d!f%JhlzuxzVUqfE z=8`bek#D2hX)1MS6(!#WTdgVeJDR_E-i6$O-~JHyJk5`WPDgPcq4#eO z{(NzQ_DOf4_%G-tVD_jiEkEvov!UGav>M73>^M5Zn@f~&!6Ya@+2ZnEVY#3F@P-=# zH5TG`i{)yKuPe!ylUuX&p_EU>!#D){=TkXREzLG#vDrJN5LGg(F1c_6$m|vr*?R`T48ml#`EX1wYf8AU6JEaRl57*Arf~A9EXJ> z$LEJpud&5Iwfe(m|~XFI`mGCRMRU6Jmp8(xdB zJ%dwij#o;U{e9uh)`Q|fReZsH3tyc2e)!_~0;Rok*3S)lkJ@82m0V)S2aUgI17UBi zTWSPiX>FfX3DotPI&aJe(;aE;HbFEh&>|Qz&`JVqH-LKf6v8+`_6&CkF}Mt{@!L{s z(@oDTHO3-9Jc-tRHCwE?FS-u=@!a0Vt7xu0z$vMLK|*Q|JR%?eXKSr-X@on5ZaQm? zoBb%(8gD6a$Ft?Q3u^z!xZ~-2+G-CrzTyAWyfmEtABj8Os|W75>OOo&z~XugcM-Vj z&CBk>-M_fA@2XV<{7&2JH^0#J@I673kJHsZ+qL&mZ|5)icDXTIjZZOgGZP)5jZ%&uv8K3J>>@6)<*=K_&=?pZiZ;jl*=4loHaA|Lrw*e!S zTpg3?NeGZ_WZx!h)mh33n~NqD^G!zl6XymtUsLaADuHQE-@+2CU!({Z=4Oux^%@;_$R z=S;$BnwW$L>Tg^ zSZb)O(For|ncI7Ry*M{}Hx{FK?`ow=+#k==H#j2|I1WVW8=o+iqp~;rs?6T-*8f(8 z>v&*OX}2TukHcX~``zqfcB^yx#N5=e=9{r3ZHUr#4o%dY8=obmabS{}Q#4@*83mZ8Q07!!!gAx(<;~TFfq5G$t|A@rzY^p z@Ozd|24+f^Oz-SOkJn^(J3K?#{oCbywl?VfVZFi`)i1ZfZ)nU^^qW0kW5BL$R9frhTT}7^Y<@9P2)y_WOG5&m{_0bqir0)S#70_0OH z0?v3ov{L?3M0ZWgT%x0hdu6I!G z;x%-mr6_RadZZ5WRSq;DJw<`htMv0&u=iah0l#NS0_0Rm0_Q7pNqB&c%t)fom)H>! z&!$vT6(YD&B~`NrK{gD6Ka)#<(pIEl)+`c%hV37uI^c4Eo-&|V1g@=#g#b6RECPPH zW)WylI(6SZX87EuWN?@CkZOK%JjULQ@olc6^4affp>1&+YDyIae>^w^ zE}IL)MOGKX@-Qi&-7W4FB2qv*cJ(}+Gb;(ibB=)8!Z(8WI-8@z%a6_LipXGC4(KH^ zKvsR$ht=4oko2d9APgB52m^76FvGz~vV7ZH-0f39Txj~s(8Gin=S+wJrUEgv%~*)3 z)PkiShVk3gH>2(6bF?2pUig81yUrpsQ zxnx1ShjS*xPAg?B1{KFbe4mw0hwIrVBaeeZinY1JS|OWfChA=@mH_N+^lt9Dy9T@= zlhe4ij?!;~BxD8~cqxus9=~&TE%w|TPxMRCODB)e*A3Jc131RhE-bi4Zt;ll_9`z(>*? z7fhnc_f(pji{0E-%EmM-cSadVbiCGBqVH3LM$Xy1onLViWDV$dv$dtmi}SGdl2wm= z@W(vQxl5$~`z|^Itg&B?@oD+vnT?nBe-2*smT;UBWF+kITs4m4lE7)5s+98V|?Rq!wrN0aBoS!2dS zwd13`afpNBszCzA(whcuEWJ;^EwE9h)h-TgVPWJ)d28MF$SK4SMh#+^QK7{l6jS+9GKlg&QVEo`pTMa58*A#`ECkoxE1+Das@bl$p7+yfx08nMxdTq@awWV^OBo z9EVxzZvBCzwhSXg(}Z9Iewz2CdZX&dPRC` zlm|nTQ;sV8Y4O{YJe#;gYH|Q5HM#1nae6x7Jc%ZMHD6lJ+TrHr42`{2^lNDqJidy& zKOU^OV>U)cFdUyH!`;wLz;K9{h_~TYs2ZHt8U7hgnal=03bXO>qnRB8J}lCP3x-KF zYc5gK~_h^*4V51zp>G!p~(v~pT)#VWAC}~ zC?~lGeTR)r{fAZp(qcHmC^M+Qrl$x{dZY+2#n)2=@YGcipafN9fF4A5iFC?wN;>7t zq~tQd2_{npyz&{ALM?Q19-Bhv{DG@y>%nc3eSM@Bu4&^+2{FRi}|Io z;^l+61!j-f9AmC3dNVf17ew}@f4D}325hvmyT!tBL2|d^MrHZ-PHCS+x0J?O*~F~3 z;?{!&np@w8=&ZyFA~1~T?-t0TnR)a#bGLn-%5-~1yqob%^|%K==y6YO!tejSbL9lj z@rEsdJhcI?|8wvp8HPHx;uvC!&C$2Tr3Q$0)^+6A?qN1--%Wq2l?sj{L*U8|9piiMJs?WhHliED)3!BL{4@W?n&rr=p*d& zl4U5(4163@tU5+6vFd5XI=*)g^hw5wRi{SjE|G-Cw*B{Z07!g2PLek1pnC z$f;|&Ic$i#X#0Gttgl;jC}ULy{P7?HYJv`zJxm7n=@z9>m~?EXXE^RO{zj4xr`(8Z zZac`RSaV2#X3cqV&21OTewZ;oncrGr*}WQknduq4PomxX1!nl(B$Z8=69=3E;GHMt zoH+G>1u@T=7*j*%Id<@HEXMb#d^lNM%{e=!G5OtWX||X!BQtX)YEECTv}x}B-*>rJ zp}$)d6-Q-?`i2lF>KjK>_5B0eZoFu&y>ZY~-h92`%KKzBrnOsLaa+dQA5Rpw<3W^g zfgybu#l64v6oV*vZENlw>HtG^Geu|e8sAfR4T$~~ynZ)F3&y3<`|0g!168Z}dpKwOcISBkcRJPG z=hhR$gQbUjMiq}DoijBW<1~}WZ-~?gejlPD)h={eFU=pGs9&M zL-40tZo)fm3&r{&HP=F8W6_(o%cM7MsL&fn$I|;Ac^0k=zlPi>^$(>aOOf4p_nzQ- zZUayMXKS``AYwK?wVd&6#sUB5;4BMo9W4*U$9wRa-|l!knAiBW6+1cqIa_|7GjelRy5`QJ=X2W0`OR$e*A;DorcSWJ z$ttv|JA@DwRkHl?ie%xkhmkDYSjZJVtG0sI88U@arlr6+#ZuttSWAJg_7W+jP50sE z{G7Qs4ckJ2Ma>g1V|Z^x5@?}Igwg=(9tK)8a>gSFbaxvwOGAI&LYQ6XOo%~p1!DZN z2JxMJDDJmLw`45F@8-+%)!#<5-xCzrnB}iEn0N8$DubDFa3>O+vt?))5(zZ!s3&W1 z!k%E`x6*pZtQBapW3OP~;&%&aZvSi|4GB?@23i`@WPU9p$GU{ENdL0<^vS@vKxR8v zW1*XY>-{{y?*D9EJ^&NBeE8J5Q{KN4b1=K+9Pc3hBhNEMMmvGy~9-_j9*@ zsv5+XS_7F?G2NV7V6;b?voikg7Imn`2=z8866NRagF2Y?T^)TWs1OUt*%b)8bo6P( zGDn|nVkUnI{1wm9G^>?RPh27;0FNgnfF*#62o!5$kq2_tW!Rkh;d*mz zEjb?FyB+VUOF^db)u|T{TUv7*(5*R6*w&oiN~}2tI_~Oic;t9~%ZlTaX~gk^iV?@r zu|`~|`a+PWdE~40>f7~aBQ<-xvL?=JzM7|Tnw-U3s-7B$B+99mn^Qhh^X#@ZYAw(@j%ikkA~w=2qk+${XL7%G=e-<@OuI30O~3 z9CI?mYD7Z@?0UySJrvu!sK~RYvVr_gUhc}qppot7te8ZePGAN-hFCG(v}zK1hEXy0 zZd`T4jh~qSd)&ps9?vYX=bC`raC6OviG4ddLp!+rf%3;|+*P4}Y1LXinPz(K-HG72 zx5gU)2o(>d)>Fmfo3{1FNT$|%ivuYP_;#~vR!*>Z+tTtBXw!@XRmF@OSr^d18jIg& zb>iF0)vZy}{Bk{;UyZfnL)-LWzy!*8@17{*N*iUIOi*sNxUx>5ykk$JGZSQiTr%ZA zb6SicY=Pp#O!?6U>&gu>A6=M<@q!}nT5zXcT0?DZX!2dIOOQ6)waX0Et!_sxsmgi3 zo~qmxm`h#GUyOA*?`9ct=!2Rv=eIj6-9Bgd+*)nI)LkMCIetJYbFO>NmH88jtd)Q9 zA~!4ud$U}9yqs;=8$wg)M;Dv3QN5oUljPw{(B>Zcea)An(TxNlZ_}=Z<1ht^O?MMz zQV`fQD;9*d|B7zaCt9S2H?kzbP2QiJ{C0w5Q!4=M9GZ&)*RGkOz&*g7i5VsfX!Ufn zz}5}AC>+?Oay?8tbhzr?u@)&&3ax49TgTBG%N=@ZebRN%LKe>T2(6EWET(O59+j4| z;B`#n?p`Tt?1-lw8>2s=Aa(j+nC3rOptGNqX~$}|+0vSCuZ8Jd1NPJcKx8$jtLB~k zmnO-^pge-5KDZ*I7o_+D&iMxh%^6U71?Z)H=Q$t97R@4Xs``@BDUWyW7VM zpIciF9g-gMb#hL<#5$dUWUkYnLU{z3CyCR4Th7t;3%6B*wfD0P&Zw!Hl7HtFcFOK% z7gooBf1iDGy)3jjnsn(og-7DgmTVdRxQ}Aislbz1z2U4yEC8%nw;VkDcFn=-Apx$V zS0^RX!^=d#$g|S{GGzcS@089kv;GGpU@^P8)}`QZzP6V#73836HIm6ecP97mcG;m; z%0cr8gD?C$Tp|U5;eRFwKvNS0Y7yz*vD1HFIlxa(AO{6a0CW>a9~bAEfK3%mDQJl@ zG;4FXUR^IXh7gpt+_Xk@2yo!BPIYFN!Zp(FyCk8A=k*p-q%BkOO%q><56&7Y(w*%FjklaJw(9`uF?k4A?s^^yz_y}PiM!#AbIh=dW&tN=MC2#eP9LBQo=zZs z_p80yMM)#a5n(O&ExSHrkd+Nn7=FveklF$gNDOf!3+Q5oMH zM&+z|*j^z?NAJ#HvWo*OCgX7xB12}zqI{n<>Ms}T+4_8N!IYV!dkpzjH8m#_5(azg zyiNP=KeVPgX&%No2WhIA@yI+i+(z-#ATNonc6X0fycPU3+_Gk?@y1iyYQ2(XhriM# z+!HU&+l^t_nZwPsG57Y0jSB$R;8o0KFxGW0$Q|{^KvIIwoNDB!=i4Cy(F}V?h=M)t zonQ|$8oOyuLFz6#9f3+TPQj_TX?U)1iFDKOZRw`b#OF0s!b~T~x4anapWm!!-_YvY zYS~ZqersiDdlb{T%$lkN*y7UC0?f-wLEz3Pksz=`Y7Z@dVO1nSNz|tMY&nYQvH*#! z$$|j7easYiNMm})C+agKy?&r0sPhjC+KbD@pMuPK-O~LWi{0_THwok zd1kGJIK7xLlv7QB4Lw!0%6mpH2K03K6X-$22znM-NZNxM8T61Xh4o7Ljay_98Gpwp zo%Mj`VOfvgme}J}bL=UtCh2ht49-x}BPah2H&y25N(uRzXOCx5(f7xL@pleA4hzQ~ zpQU4u1Cu!RxQ~e3@L3Jqc%9+VYHQ+rgFn@u)Z?AP^KdjEJ z<>obDA%H(5unfWCr7d_)6<8d%fW@ir2P_?c&Uz06_Vs%9aiatKYH>d2nio}B*DIuG zT2yqE=~Ty8nr1;<+@L3MG1_)S&e4%em$2QH0obfoW&R}?hfy(iAlU^dQFZxZJQ zdswGjw(H<{rGYa}nK0vL6`1i(V_`;gG8Sd-Uu{*#6c8@Wh(p3vY(3@yTkmbrDf55d z&5#Jk3-5=)j9hMvU{AjbS0-^cj>H}}K3X0(PEFu-@dvulT*lovCA}^jP`s`f-ANp7 z_IojVhn?JxZXE^~RpcSs?s!1nX>I6Wp}ilB)#w)jp_JNt+w2!oV0B4( z(xjS|KAzozd|Ljr=4x9JG(=Y;vlIXCvD6p{W3U97?T%pufN^RX3*CpyeJwO{2Lt+r zeVAQ+nf-J~h^r05p3lE7E`K_i-Co~aT<;n7eDT-(a&czl4Ci_&a++ZpG8;7^+=h+&gpd{W7*R}^( z_YGKcu2NjdygFzBsel-4?+GzvW(dT)TTfFj@CjhId;`3P1#+==!o5#cJAr2%i*max z)0ane4oR=_n0IGqm*`WV16|6nmg*qkn5qaDbU370s&I@_q`~p2RgGS0<9h#RW!(#| zO!PO7OsBqav!I)G^(`)u0Ul77>YKumY47Qk_lpG`d>wTZz*+MtJ=kqP-&1|#vYy<= zpbp{o_M@$=Q=lE;6{w9<-%o$jHyD|PJ^ixA-n!PL+4DIz=eY(}n&V$b*-XT;AVM?K zcoCt)2dq?uNDKiQ?7JMX^F>b1{_Q#V<4G#ZU_~L;qd~xdg;fo&|%ie>_ud}t4i^I}Hdu!Als@gkU zDSv4WH3CC}34mGU_b?3J-Qo zl6l3xY4i1sm3gh{>Y+vHE)0f?aH#Js947QC2W1%juF)O&)cG(iE>Qrsb6&VSeeLS_pzH$D8UbKu0Fr?A^nOd}r-|m+m z!-a8x<=CRvEy4SmUn zKumML&E#m;J0ex(-9FgCR(rg6NIUE(pNYuc`%>uIR zQ-;rND2>NY5BXA6r*Yz*5?nEwgUZYQ1rXhZIbKk4-#>m`Ax}n)|mE8 zq$v$#(z3-CwoKRB4D1d-AmsvwjZuOeY)1=;E&n4 zl}sy@$OM=(0eIv^BXHegt_T2E_(cNHPy;3sfOpO$07Ivkdi-*dN~PV5wY?i#T>x<9 z1Oh+_>tsacQX%|O0Q7|k6blo72$9o1!|*>{Tz;Kj->i+rX$4QOBm>~8>gr(#3m~hLTNb%_IGsU{V+!;f7&!~dTlXtY$CVQxkGojN)1HX_0k}42K1HWXrY5|{ z7e?UzZhp2{G)yCsBrHyw3H-U{uU4_-m2QNlUz>Loi!{$AuU>$R02A}EO8~GTVa;j5v*ddb&oAFL#F}?@H z@6JB07i`|7q5R8wZml>eO>{SQC+9+^zdg3om3Eqm(AZ7O8~D;~?P;r(-d=2$7F%n0 za#m3@hIj&NO()Cn(_Ew&N98Q2&&_qiMpJ>$iNVHGjxgA7z+@vKQ>a$6({T+KeW>*A zta#wJM|?Y9Ldxz?5&TT!;_V_wP0hz)3=~gISx89PiP2{TtOY+_s0Ke|DPY?^XZYOu z z=tv40%83*M_EyN00KTP&0YykU?Chc=qyV2SDWDM5#DHJ#D6WeE8QfhW#Q-8G#ekW1 zMGG+8HMIb4GC~esuGUzzWCUwmNe(!*#BANz<9Sp;@W+EPP>nqZ{V>LUcgyXz5idY& z?8-I(YZ-f-as&Qdj6H=gKtIf(JDR!uPBmNqZt-cu!c`sP`PpiDj*{tXq;1~+cKZ6m z>)(F8%fb8{+VW>vnNokheDxC74#Z3+uO-tt@s=?idPZS7jwVcJ;-fFq@yj)$)A$4$ z>r98}hD#*VS@0>T&JGtj)!9QMz@6B8++zgOu?aTrsf6b*u5acaM=xWd|EBUB0_%<^ zJa?f^G|h^yOP;#{o+L!ic>cs*`#_Cr5&cdgJJJa^-n~y2x&flGZvA~`S=klj`pO8> z4`yrje$%Obv0C3;->kmmL}!B+72wB^^913}t;_p1GYb;YoV{u#%SnxlLu;mO2Q z>%OhxIl*bEcPImcewia@fMJ1 zL0M@At?A0qQ-IlEJ3-iM?j34r|DjdmNq4t~J9V@g(i?BO`2O#1Jqm_iG34~76GhGh z!gPw97*s5|F!qVJj;EH2oLx|JMc&BhY!Y|td^D#^oYzc>bg4j zCla3>`de_F2=^pA02nsmLfQz8-aOqlw_KucYN_-m-hyf>o%Z;3rqc1#8rvIg;6>Dn z9SB@x*v+X9Q|ceqv#Tq!waDS-<45@PD&_4|98+8>%B)i`_}=7xJKG6GZkw`v7#C!8 z$AD>c$1f*z=iFaRcjz0-Z^M^sejC4Cp*x-|Tq5a?Z%ei_B+1#%H8nx8CuGMxoQWA$ zH1TeA{eL&+_G-t4*;TVZ5)Y!H?~ezHd;N|x{!gu7qJ6U{503*G?l`94&aX;$9Y=bi zjw>fX9e3%CIuIm^n4 zJ4>~9`&DYHow_<#?KrBc_CAUoS|DIQOtHV5VKXhX>c$-uEB@mA+$?{sn{hnW4+-?% z7WbJHINLib6bv{{ncg}%nuCm^8pwBqE?RGRdSfwW`MSMF((FD9mKd?97z^2-8zEUSwThQlK*lSM1(^lDV^Kcb++13zHKbM$ ztqt_gy7XxpkFh_A13h5t?eaE*IRBwF(k}58w#8I-M^el*MpiM?xOZZv+50)R(~OnS zJm)REfYc~Y)0$n9X&-f)$n3VFBDN?^4IdWW{qe`s}kvS5MRFn~r8?)aH31j8{!w`0%} z-A+>y>-KinAziss$D|7XSy0sOq58~>LvX{lqI488!8GNM>U1ci)#v zgT_Twc9}e#3L_rCsexhz&OfwRDIMWXZ|2`->l?_W64rAUj-{Evu`{*}=T=mB8>K;o zHx+x#Qu+_A-Qddj$X#&vbvpW)w-%i5^ncgpXu!V2bYych&LkVA16HkOI`9I)EVK_A zKDTCqzhDFg6enRhSTdYj4px)~Es_KVoY8QCH!N@a`)s+sHC%^7*4C+nq@8&9+5_#PJA=STp?()H9g*xuPSbF}nkEuiVBnMjkMdHN2o zH5O@3_62emTm(gzAI+B6sC3n#K3=i-RYkhb+I4$POm`-L?Unc*H+S|Q?ld~yFV^UH zd9x@KP>W>29VHfENo=Cg8{cna(FcG7Tx2lEpp$_u&&uGiMI{Al>mUEc8VYu0gR0l09+d3Pm?SHK^(}; zxeh*#CfMUhj69Jua(p;70q*>PPBbL$I3-Ov4k)nm3H>V81ay311nAFam-B0c`fEcA zpnH+E5W<{*?GkutW{(*>1NMAZHOYHg0tq~L(G&RGjpKuSSL6#$9G*@hk85+}LFtK+ zf4N5IZfn<`qxEd@$?Nf%&EzS7@-6fLw%pOSrV2F}S0Jo_$MF8?Vg8ZkmE4Kzl z^Yo54=E4YD-S2vgGv?#r4eB;NXTZL7r(%PzAdj0U$YW>|m5=9$iO<_?}E{-FfR)>SqbqQ)xH4o?U z+hd`YvpmC~Mvvgj#q8=D!lLA6|5&Ze3B5Ynrcd9UNYMuU1lkano?rtDo7Fg(uzGik zVZ8u*`;pyuu-Ptw4W^mAcJkZ1bh-zx*^gr^ub?%`+=k=p(uQ;4PHw{+t`84@CjtCOBYw1=Uw<*=;1_Jn z^~K8K*>3zk_@t|LXtu588!{`a<%*$8JxfIY>LW1xSapfPy(m+vgc+GxTx?q zTmxa?zsFuTUKm;(z$H`>{tcaWDF9qz(N_Y1Vj=>rNF-`L*zSO&XKwM+H5qV$Rz(P? zWx7kG5VRggoVw-Pf%dyL?e`V6&vsZb@9(x{h%z|L?q%RZiz?uAnN)&)^9X|;0q9l06~n7b zK!Nxe94P_JXeqoeCh+){!vlD>rd-TE8P*^3@C$Pj2{U_7+ofuMuM&cz@A=M%&;Q-3 zehgP60XQwWkDtik$Dk?j<7k3D#3aTZ7g*5aRE_)mc1LU-d2xwU`8Xw!hmy;I$GA<* z{a-HUH)q&Z05?{w{`1)yn|>K8--ey-cvRe-t8G~QUCy2+(Bnviz5BLApTCsBkD*iG zr{qY-KkYS5c*alK;a6+;<8{I%5`Wlti9d^#aseoM{g93k#D~E*7v8j+#~Aq6a(DDm zRVlzYCo$QuEH~s@?hbCalF-37w1f^0 z6YXN|j;j&_{f8xV{u6UBn0L2IbPKx+)L`y5ib*GGdcq6_OPC!?Cg@$Yy~WA~k<8&5 z3bja=g&$_p4uWZd?1VN;ZMeL)i5GS^2J>wE0M*;US^ICthexN8pJ zEDI0jN~D2?jxR14*ZMjv(-myp$l4!e_dFFug|ubtt`3%`9!M&&9eC> zL{qroFyA!Fu~L6*&$Sh&&~YC(?3x>n$5pgCj*ivpZHmZO(F}9LPgdt^b_+JL?7z;} z<`y6|H;p-GHQa*4Cko!-3sL<(I6qNVKtDFat)eIHBU}w!NYIC>jC}?wblW7&DSK%y~ z^M7~sP>?M}ox5p9X~vnEjO{otu@pLfXo93#L80T6ROp;^$~8Lsbp@a&G3BqY72+?_qI6iE)4dz-6@v+ zr7V=GT1|tkD{?%}SVfKp84L62VtICn&i^`(v4HifRT5D}8AGY6a|2}@{yr%0WIM0Y zM^||a6fJ~HmfF2amNUBtz<9o~0ROSLxzWKbgx@$qSZDq~H!CK?Eu50p83%IX41*A`oy0Z+@{2hFD$PGD zuR5<=XJB<7Qey-jMs=sA8J|@~;E~(cWuB6^hDG17vJ!IYE;JQ|HV|s+fPj8V&!V7WbFohClml z^Yn+_S!G-QpXl zfL!yty6sR??;0vm^RA)x1hScTO;t5TIt*KCrCNILk(nkb%}R6P-9#Af$+QU9(j;(9 zTt6W=Ew6uneahXh{^KY7dnP}|NB-e@107+eYdyotA%9_e*nOEd_~}`DWA}IaA9v&* zN3TEp`Sa1m;>?vZ@uK!n_~zYr+Z(#=K7Kk~{j+`2^jDN-2dCHy?B=tvLr`Jg$GZPu z&cSsjl58zwY$2a?)_bZRTsbWKV>O=8KHH7mdwAz@X>o+xQk0o;EtLmKDKZLnXzm0F zTC1F3)!|q*^eZ-XvU(Yya?)DbTnZ5yGnd!(GYv3%e_}gtE6>FY(R>`&I!gC~+{{7+ zjII(YAP1I&3aqh?V+D-A6)WIWEmpv@g+QQv%J8`j44fv_r)HesiGpYx$%BPrdm`N4 zX~kh>@kg_(x!dwfb14g4FJyALIhpEL2D!-L-(i~9%fekM7sDRGPxFz$kFR9#gY^ph zkfa2DE~<&)Z-MB_nWQ(UeAhEqch<;r1!$NV4QNV54QNs)=`iwj_ulqcc$i27`&-A-z?u0T(xAKVt}07PgKMB2i-WU~x&sHs)uh2G zCGVZ<(%>fhJBg9mey50H2hLTR}c;{#V2RzmzEjd`nRZTo9W0@lSCmmI8dVbOKJ*q=4V<5Z0xjq!S?Ga?a;5 zNRSIb)0U0+8mmMn;2H4CLl?FlVrw{KX5irRQpcc5UiRI^ic&=>YboSRe)EKLO?s13jv8? zYJzbx@aya=RFOyg;rCkZz#knikRF#1J=de~XTQ zbORr6n+<7q_DYF6df|?JT3ClyRa`-O0-7%fF{=84GNUuh7d%?6vH7QgHagrGgfC_8 zx>~z*_JDfb@C0|5?~#P3M;Y;*Dq7N=L)Me-)E$Y+rv`{sJ|KHol~1M`WQRydvQvO^ zva?T!Np`;BaD@*?@9%}RROan!W593gdr#yPd!T1Pk8eoeAte#+_#|P+fedvVQw%-7 zDs?=L#F#tfIojF7D+NH?XAGZPZI0hd4|&%grxMIn$P0+`^@byUG`n6fA*Z?hZ=v{a z4ajS`I*g@i=JAvX@J^%Su+Z%IEEPKr=!g((l&r3O#V~#T3-q}e+}n$yYUna_QsKq8Xz=0<1iTochk;kyWCnIY zh1Xel17Dsq0d{Yb731OPSc@%tQw&4+;N!=YRgZT(|LZRRq9y5UmCf>Y727U)3tpMm z)Z1)2=|8lY8)n2Kz0Lkaty1*bS}mH#AS#-ht0oiGO$Q;?-S{=jUgK2FJIDQlVs0NZ zd~S6&Bs)FibvNE!>TZl+Zmya64Kz;TGiWgT%qdckkJ>cj9G_0+hx^vl@e=2ci-Dvo4Chy+Z}#0+IXVx+ zC+lSNoe><(X219FMk>KE2z~gCH5YUBx~lczwl1gAi&2$GcTyZf=keupdTnxZpSFGGYYX_gWWY;tT*a zoIzNj$1zS4XGan(zk(A(+;CuuVNLY7UAjlt!*s<#MpKi+YAZnFwKSk<#cC66_gTsM z+xmR9wAQh{n|=CpX|80o-L-}d%T+EnT|@ZXVcW~6+R}cJTnaDMIRmFmpF@%qeNN{t zQRv)CC-&IjBv~FC5S_qd!y{N88y-b^Y5h)g*?N$7DPLmtNz+ajJ$pzul$H+Gh-( zhi6%EyWEhwCp0EfaBc+Z$Eyo_bLTOe{D2`U=e;h-5vO-1AqR6K$nmKTxlVSDiG&=g zKD)VSqcAPERw#q4Id6CY6??#-*>oHhHXWZI#->BSgh9t=H7vvP zTecjh9J{o%ntP8d>jH_?OgS&EnR5KtSeNbnZ2f;XhF^YspUxu|ww$Eaz-1uv`d4(> zxq}q^e;9C`6E5I&Gq5VatjM+yW+ZGv?A*@`VqmF3+{atvtj<`BU#-3|`>m1paD_!S zs*SF3cj)FUPUs_i@zN4szTh+!jtsa3z}TDxQl}!!aK}P;#4?5ZkoX=5pDh0V+lab8 ztj^CzLW~FcA=Pt(BGv@qeRD#a8CMfvfNxUR32a8;djJfbFqXlm%Ueul>lCI(W&m8z zkCG}X#teJ8>S{Zlvf$0p^&T~qp8vZ|j^f(LyKZw7qx$|~SQGCWl<4VQQ+n>d_lgrv zJu`|^GLRzl>L?*_WMUwct#g&-=FcTH{outNF zZVawXT+hz`!t6F~tTYWf!jy@j-fJ#ehjuCl5N%DjgP+0$2mDgEDhJE(yL~ zqjN2+|68B>qLLPUT|tbV2{SxTLUAq+i4{B^>>e32JX)Y{`;g&tOK1Lo!Eii^Bsd;7 zSMZRv0C7TZ+`-A(usmt8tl=IJwq@G=@u1=zsc~448!sctjRTXAdv}Y9U6R{gC|)Et zz%a>8zD`@4MDgk-JqzrHd2>Aty4C~5Rs?U*$$57D)6wGQw*6wU9RZpH&MvR-E_96$ zMBjq;6uODxxlF?WH`kOcyympB1y$2aw3K!e&u54U41^K+Eg@9{V}jD+6(E-Y z?5)K9}#aWEeI>Qa}!crx`2B)u^VPXFtLJ;Mb?BXcW3;kAqIsw!3-0+ zy<36_@xXJx5C3;71mI&FBS3(p1mHj}04n~FiUfYxkr;i)B0WC9uhilLjC{aa#~x1* zE|J(n@Dl7b=ovf*q9^V{J2XS1gXgzac7DFTnn5TG;O(41w^A_;3%EVH!TxPVLlStd zaE_4YW@Qq2pw|<5T$v!x_#lQJVrTIlr)tm(Ty^M$bV%s&0}^@$9XaipXPOxM)6JJJ zR!MuIBDQAT@vf?M$7?096rWov<8B(reW=W~LSsSx zuvssyP^`2ylUdqqX=zXG+&P0dNE3~Bjx01jNdt`onSsVJ#Wm+wWnB!8#P%5sj}L$H+9eOS{kbz z3!;g2e3q;R%_m{?PQG*E-i|pX=Q@BGy4yv1mVtI!YRY+6I;vw?{pV`4M3X-a;}`3t znP0J;Z?nHYtHl#|J(*d&>HltnE?^uHn|4hy8(-+jY>aloYS^ z{rfyKYu1{c>0T#EoT6M1YaM6J)93E#F<2LmFge$KM}{np_T^lDcbim;5)Xely;{7V zqAlx<&b~{;MauYBDxERhL^V5Dsf}WP3i3$8>sqj_qNZ_w>u%~cbFezb7={Hh8;6@H zyU%HhGKG0lWMmoseLconQLHl3L4G#=IGv0aIIEi2&3d>Dai>tzG}W+%XDG17WfE&l z$Vyll<;kh29t?I(Gu!Q=Xs~o`g))p{#d!t|^yO)`4DQR+C!a^7tC7LcHC0fi^L!T{ zqDpFR9Gu`AOzYh|xxpY)WN34;rJ+r7X=!QW%GA=vSX!FecyW`djiFW^vL9vj+k4E8 zM)mSHDt*zfkzP}7YuA=PVFXUNbjxjFw>)!CYwpNnB{*s_TX!HERrKjeS?DvF8AzeX zZ_3;jmnSCZAl{Ol)Yz`w^SknqzGj+q}smB0^*$>7H>85`BJp&iu6*E)22<^Zzl%J}>5q@#j;E`Rm%WVI(U0_(}tO zkcEUEv=AZh3A!wKwnJXVREJ|3A(^=lilvd5*0`haT#4BQX6zDpvXHDtxzlIv+Yw&Z#yjS~0= zED?wa0A_(j02tf~2!MK~A`(dfw9q99Fg=&kz(A(I5-RY5v1?(&%js-3p6gTqn*g9` z(epaSAJ|qs%M1-7*w2Z8S*Mu)izOtH1n58nlpcxb=i?(8fGaE#z?DrT&_eVJ<3LgX z?v?~VlQ$;-vX?0ZynO76|MBqSM;-mw!-auTDLT7ex7 zk%&q?5DsG7L|>9xU#5C7o{z4ESAbQM`7eiT`=+vcQ^1w2K2*5F@dWJ*>=bL#`HCVo z3D$@93J2z$&5#Xf#_3oMXq;1UhK5mLZUHiD0{f!;dUSO@8JWiX>B#QE*`oD6dxfh= zv)bGc?Yo>=Q6BJvg56l;`XDgetR+Q$W`~~ zaY{;dW^{7NE*F+3njIr>wAan0JDOg(xB-f5l1$PxCW zD2YAGi(6u!M|09Gvm7Vs)v6drlU8qmJqDYSL+IlOs+QRE%W~}5%Ch421bfg=5A2@} zi#=5}@ayVeZ1gcm6?|wj1A9`sWP6Ex#`HMd5_epg@H`$C8*>B>>qB0^SWGd-sr1mj z3PDGTynEdaolxWn=={9yt(RZDd&gpAbD6^2a^p|>ZoPwVFt9&l6HN*JU-x4b;`rH2d-a?leamj#u2TrsDxwOFtgS*?v4>m3}-( zO?dH~YBd#4lC%7}f*eanTKI7wLh;*IHsgc(`gJY{*wcH*-brT{7Q(hzZ^qYd-kh6u z4q?}lLBwj>V+@SCV7$M@5TAf~m*drO?jNebOBAF*VH(mTi^edG;P8^XTE%I89?pkX zM#;sA8Im>i_w4{<%E0tH3_-rjNUap)>=+jbayqP1g?Dh-@!~{`Q{4zLZcA~BiH06D zocvNk>q;DiSY0B8I8I3+&M(V_I7@#_AGxAF@xjxdSv)uC+{I)-E-sN0 z9H*oN$AMgc^8qC%uOtF=iLp$97Cusk;0r~kv#DXWFKcH1`BK$w9kQIU)po-s1uNpr9`&96k zFKy*X?5^0;$~f9JwOXQi>vku=xH8kxO@S13FV zoDv>Sv*dXC5cNgsLPv(Vm&$;L12&f+JihZzsw8{w83 zCU3IZz;-GLh@+HjfVZ#Tgjf8&JB-@Hrmcgoxtr|O5{xW(8w?t^8sHceg?SH^Y?)+Xah&ZY*7w099LijpE4W8(!1V`F|2 z#)cq_b+S{peT!6wShqb{g>tG zgRRsNt7-LV^L3c&Z=J7y9=)AdJwIyMy4Dvf)7dGyP%H_$ftj1q+_b^HE(}^4Qyl&* z11zBaR(wsF5d~Nr)98GSw=~)06srw3xwu5?Yn+lMIp{CvYhjTkZ0%>JYqjz#G+_J( zBUi%X&qim9=?qY50aP#W()Y-a>KO| z>@@nMfX9Tj#2o~b7;YF~OWao@;PI472Q&zH@j(U!+4Qp!V|~o(Mr%cq zjXO+f6?TlzQ0oA-j@SK%{aZymiKg_&x+`^Pz7B2MACDHZ>3HF9s%_bRI)3jGZhZQi z=9cY`X4m!xn%E`6BC{qB!ICjKv3*$&0q++fJ&?HS_47I-P2%F1`P3g-Pf8or+x-CUo}MyQiiO~mFLJA|DoNP;<0Sq&<-kVET<=p5vIKKJgJ z`E{;AupoBzl~yGMG@F7kv{+w^m)Ya|(a3Jde>!6i6BT4zOO#cusZkg>x-(ZgWancJBxQe}A!@a+91fu{hAkq6O85ld3z;Cj0T7Oa9%PwA&pzQRhW_PrbTP4to|h|YH}~Br zPgD}$>GQno8KAY)=uixHjs}lOax{BeKwH>rySoFk`55lE6gn3dfh0=8PhR@{4SMO!t6K_F*~>K zkj&0oGH%BlE8I?0q@1VNj0HWOWpO(mxdLt{ua*Kqay#BFxm~ZfVBdR_-9hk%kR*_HrOB7Av#z5h4RB#YLIi^yDwIgw zQwLaNfKw|V0}zu%28{SfIw0b?*hgQ)8bat<2!DDpz5Hly)%g5+j*XZAofhkFcI3LR zc)?US9dNIZ_XMV;=qFBQ`3=>wOwaFc?ck0?_G5xoZ+%c(EA1szDTalt#}mcxrDZ>t z@mJEId}%w`KN-%fT z?33;?mWOPLS-wW_Q&qer$*29!=jG z>~6dCgA(f0rytaGD>R#nMDIIOEOhIfJ@gU`dlTMX#*X8D9i+Y*YGYf|D2T7;m3=c z;0G;L*!odczrDpTE|E?EULfJ$$Cu@*fStcg`2Fm3t?@V-!niY=k=D9_jdj;{YeJxr z(1A1fFhj!d!)4O$Ba5!0Q`zAbJ0%Fa&k?6;_p!3ta{0N~&M}=<#RWy9n0>&oZwz0u ze>bxa<(Zb%XLc#6V{?3lD&WKM+~C6*$>2B;@i%@+?#6+Pxp7R9?Yvcb_&5@acKozF zJDh6bZQfnVZJoEpC6c#sO7b=i7kr8Re0I0o#sKkJibH#!MVb~CS2Hecay2D$z6m>YRbQ^=;-U)sZe0D`-jD5c zdTywPbD|tWD%LtQ0A?oeQ%H!HrEpoUV-ei5}YfvC$H-FlPMR#27Y&f-&ys zi?Q1krR48!7(bm}{WF|c3*%l~v!SpWo2hfw0FjDtZIEssb1y}B>apitgE`Nc7{@V1 zjFTl3InK>-V?oX^nI*|FxD`lpGGi(1r6hMI!||(Q6eD>7PM0?q<%gi2ik}mIJ=hD~ zWcjn}D|5dHmzcTrqB{*Rqps5cqb!Li53nEH1$NPU|d=;C=T8E|LjH~d`| zVq-QH8US4;BtX`QNr1z)NPy9VCK}-FCBS~1)o%|tiLAUt(g40*YI6=RrvZkuO&Y)m zoaE_71COuG)iuXcbfYqN25OS_<@P^m;KoqsI4rDm%$Ah)xL_q#I(k`vw4{}et%9uZ zEM98~`(kN8DR+f%2OQ$JkIv#~Utv#2wF19wo+w@0%Z zZjY{N^d?reMLIq-k?4+xp^gMEXCC*eczHkHY?G;f)#1KV!^t!VOtTF_SHT$H)R&VF zaBp8u##&c&o|)?)*R7^#ky0}hH+WP78Ph31UK=Y9Zs*I{@0zud^s-EWjdLc zQy`4KFoQYzqKx<%>L@-L5!B=1wN=|}mdTYt(o|40UY)=j3^DS3_}d$0V1>vg!v}gx zCs<*)KKxl4WSCaPAS1uY*b0Ub+he%RGRJVL$r|I`rK#n|SpD{L%Uf?i{weUZAV4{wq4F92$BLI+9L{}j_peV zAeMv#XlKVXfJyvnGyrfV4S*-CDCzqE<;g0!|jLlEaIh#9}RtP839iP*KvyZSW4i8o8B5Ss4Yh-PV zQ^g-+OyGB{jmy7`)rR_4pdeU#|G>1`-1cdP1I7|-%%;p}R7=89Z3Mt^>7?#XC})!7tHr2*D*COjTcA8^O&@WYGvWX~Sbh@=2KEh7M2 zq!0k~a6$mCvvgGUfGd_D0DN$Tu1)}?FhjyaKS=yBm>hoEL?-+=*8_g6OdQX#i-Tpq zos1S}9jwcM=CTKjNu>f#eL@9J!b6S;Ne_6}6KX^7)UjL?Y8#N))F=#VOf4I(sL3m3 zuCUoaiQ>1n;Xnp`1wCVa6##G~9`PhlQziyM!#`+$WmxH8F zFb0PUjKK>MV_dKje{64y-7*Bm2Ro)eM$6g4xQEdO>gvnOvY5FWFF%}o8qTad_|xHA zD{5X>#>81v0jAbXgmuu;gS%jh@WZQY@_b~q0PN3tk5bXFZ1v?I`NU(FK7$Mm0Y-o*G$0<{D zg8)}+p<%=o@ZE&^&u(u9%{)U@gMqD?JF!5mVQ6rbWN0bL@nw>s+sknxAf0avFJ|Z( zF%D{4FGJ(qlA$pVxe&*ID_j#9jv@A7XkqF(K<> z=xkRl)^V#o7tILEPUAU~qlsQ#l!~K$?KJ6hcVp>i!`X}tqI8TOkNR)U5p1$FgpLo@ zFb4DjV{qhpFh1PkQbi%O>1pd2rMR-W zd7K@pe?B}jO$6KM2RKCxiWy?lW?#E--sQ_&lD z+`Bs5iAfo9_hYPnHx||4ZYf`IAjciTbj7&;j9vLGqv6@Lxr1ACxg6+Lg+3gwRLhrj z_3(aSGT^srG4MJg!|H2;qZ#U! zlALX(B2ix0*ipAx=+>4zjVlvb-acvv4qLA*r_CvXc65KT9bcHBst`{6J)Co-)~!gY zW`Wug#q5k{cVTHtusHRC>5z^9KYSJI)AqKQU>97yXZI3w**JrFqi{hXd%pPLc+ zQmqKG)=tA9TXJ@Q8+U|1H|BY|Nn;)6IM)O7pJz9hR;KlIHvD8RN3Bc44a)X_ssBHq zKsEIF0T*CO>{*!*as43g9LQxS8QX(>l`sI4l?mUoiZeFUeQmkU__(`?@d;0ZrH&7( zE|C?IESHj;&&XoV`GYl7KR=vfd)CXL-*Hik1l82YHsxl_9sMv}2`M3!{XF0`oxfBDi@^)tT>-{0VF zrF!P*VmNafL+LWUu}rG${F-unc_Q)4xP(E#Ox}|?J9g5x?E{5$=?jSRSKJ}YxZQzW zUa@z#ZWnbv%E?)+h=!Uv2OIR-7wp3&Ou1L@M@Fg@U|Y>3n*Hyx&2x%0K3-$z6lch^ zmM{bVZZNZ{sDPQGM`dS9R?7>oUD5#gpdh>iJ4<2Sjh#;}uGy|%@91cE02;>#6um*C%J(0xABq2*H4n_CYj)5{^QQ029mQXiD@ zl$3i1J1}q({td?1{toyk{P0$}t30oI`4!EwqfiS~YfHl{_s7Vdv0X+@U=+&$oF=mj z1X|f#z#gNletR1XPe~7jx^|3CT4j2L6epxo%PaB69*>8{vW6$)s~*^c@>RYr_hqeA zbO4*hy5VmRJPyPbIatuoXuui=!k-OpI2EZ(mw?JzqE6ND@b{%aW=G{)48 zh&0n+jMt!!QxbI?$W3udwz3aUf;m60yARIh%|%-CZ6nEWYGX4s!c?|YbuESK&#&#a*QoK6eK?+x+DVjg35`2u}G5!aIOapJRO^x z^wuqNu%oc9(Yz}B*#XKRfIZ?N2_Wg>%B6-9f0|wy{&8vM%w_{{Dj$OZu%a#ICK00TC! z79t>51DXk-ivX7vNk{-B6_Wr5Zpr`*uZac(z&Z`ohyYkSQUtJvRZau!abwZ|od#3TNt*H+X$=5!Nd#1kIS~wM6Rx*2zH z*!1yDQC}8SSKtP|cA1s%!#jGE&5@vI_f^T__~e$F9@LXCIPQ(vn+$6B_ZUSJd*hNu z_Kq)++?|Pdih`WG=@3;2uf*4LTk)_voW|Xck$ba(v@N<0)uiCKFw;=@2f5ov6E7kI zr5`Al!)Js5OcYU137Jp=>jPp^z{4z3SdFCT3k*RIh}#Hy?q+C`0-n)B&ZGGCgT0Po z{GqR$@0xo9K0?mpx0Lga4OUTGGuUDQh!FI2r1W?$hG9kPsma`gYxx*FP>JW_4lN%8 z@*>+md0dL`3gmEzIokZ_y;jM$N9g!!Vq8-M_|HNGt4(kI>i~ha1^j2=>79?YNuE7tF|Q%`%!630I55b&&a;adjjnX+0nh_O~Dw z1jyhoRe;mRGAbi+%EZ@w$|@I0EhD|JXK#&kI!rZ7KR2UW_LE9^xGagTtaXR$GKCYE z&@LHuI9{M0f$;8*-FpO2w@|)|nKxl(NF}eDfB@;s#kPJg(Bsx~?We8pb%4TVjo?+FEvXP8>6Qfg`(aW$v%1(~04h*-5y@kPD_mwbbyP z2@NH<%{$c@~R1w}-;%Fw) z%M*#F#-s@j1{CYz;JqzIqy@|aCgW8ero6OpW_r)W8TTnTgBAPY%&c@@oUs4iTUgzgAXQwORyWv5Ff-6-2{m}i zHo7V5Vj}w%y$*v;*U| zWN1hK5kup5OO$b?jWSLqDC3BL7{66c=5>a}g;OTHIH$miqkZAUt$juH$>?u0?0Gys zyWrM2s!FaaYbz7hDX=UZP0Z=wQn}5?!w;Y2;+mkyqhjg|a~cIc)n$d?PsO2fk!TtE zWENa_RGdXU_K9aUIaI`;be8-utKUA*DK3$*PiG;-o&In@NplI?EAgfvG#tfs^QQhB zPfYxuT+PO)b~6Z|)~z-JCyF^1KxY+S_`Aq*;+rA@$cq{=OPNEBrzrSi<`Re6rBI{( zA8u}S_(6D0@Pk4sxOMo&B@%wTTYA%YV-7#qp`d^0d9{jj!ozvJ1i9jj4}@Z)YPr4>I8fwFXny+HCEi z4Xg~p4S#xr4HL5A>Z4!Zy!!q0KBjZUT>Tmwna&N>{p9@Ibk&*yYd}r4*~0NeZR7Bl z;c2|+-{OX&Xb|lku10eZjS^8qK=sUE=?%CYVx^#{z5*M!R_t6Ou;G1n=;rxyd_ z3uLu<+nc&>o~NJ39eR2GhJEuEH##{zHaA>qq9(Jy0H&vn{)fg`kN)4!cUIfAiy>-D z)1jKboE_j7QT5x}`gDe^Q72ZDftURYQ*XoNP&Gm@5fY8SU{zezgut3anYwTlrE7az zhd@Utg}aAlqJ`@}En|;$(I(0me|sMeFKH}YlPghcq=(X@_2j};FllN59sdliQ0u+n z({*%nCJyFZUV07^iA+6`F{U%};DA%hFbyU!3DdL>%tD^5H%x=?k{%qsD-Y91E?$WX z_Y$QLR*Tl!-d~L(9KN^;e~!ND!mUk&?`_?2VelwM0}&4DP;)Pwa*MeOYApFoFf;BG zzkPrPU)9$#V92MHX`nGT9JzPnE9`=)@ppec&D4nsa^U?$f`gld1P6^s365D?i3C5~ zV%G{G!MR3Ph1hMoW#re2WRMOqMRJ}$T>@fe9{NK3=Wv2xj1KVW^xRrTRPU;rZGZ-p z)DHOy;UL5hNEP9Sci3HDhiYxXfNguF9H;8a1UZDDl4D^!H<9FA0TD;sX+K&acf5K9 zvK*9KMdf9dMd{&3y3cy^u3@i7l({rR; zuC2}VenyDiYTVl#jx`sVP$?yJ-~>G8K?r(avMxc-dNnENG3DI_eVBDEEK6f77xZ|q zrpxi5Mnj#gZz}vt?b0gQ^6>pNI)hk&Srk)PFgICjT)?j~I|kbZW3au%n9GG%qRmm{ z<`^q7*>2kl!pU5g$;NZ0KStzom~m8tx!~*im4brJyTSbQdN?~Do*OOpmx;r&N~;Ub zj{2i|C`^dNnZ&#bw|jq!TcrzLKD1YivAETQ8DlnKCUQBC6yEGG+86HQ;W^iFYhWXZZW`&0C5~n7Lk2azNE6<9QAB&JF{Ry# zBEXFT>fM1IGp(v^rrXI8|d-ES2b!-*Z5;iN}B7jx(IJ4 z>>}Zf-|OP&EM`ijtr}u%Kx&C@exM1lF1ihfd2tiOpqai1KbxLGa=+J^xo%6udY8Lw z!J2Y%S?yX0?cik%*o`9m@E*5Y$5**rbk{MfQQ#ETYvyl*DHM+j)0?>4#3=T-z?GKA zg-14dT;#Hb0yn;(yrjwF!j};d~32};j?Uo4?Qxg27L!awVDbZIu^Wr%Q{9us;{@a+Z7=CcG1wUqe1@MF1 zEcm@+5SKLbNRB^az9#-SC+MLa{&1Twrk}<`_*)tYoSaP;3&VD+t7^x5s{$UMl_>H6 zI-_0T@1mj^09DNIm1e~Iz%S!{98Q^jygl;GYPfeh91CUdHDJClM!d*qO=SO#Vyp7ci;HnkiPrfZlMBK&RZIs-6#M(d&XC9v0DyCrx*aIquI<717_F0 z0d<#71YCV>i2r@Am6lQ<>C8Xa!4;AS=zeEJU`PQBDv<(Ai^W6$zO!fmLyGsf0M_K*$5?@>*m4HjoOZ-(C z!0pIP{5^YgbpTFBpFhmsUX#x?5%9>|%DP3u1*28TpzJYGk%1n%!<4mj^vR*GUcP?u z zpW6L+mZkAAv?fBp*eaBDLLh@w*GNjhse}^n8961$a>#@d{Pc7c`A1VU_%jH>ZX8sr z@wvf;K>*>5LI4C)5`eSGDyQEy|4j%0$Ycb7c~A(z8O4$UxUw;p1^BrH1qgI?3V<1B z5de5b5`exR6NhYy0*nCf?MeVh$z2T5T1JxuM;Egms(^V+6@Nw52gF&I6aFqDJ~$Gy zKNv!S51cdXF$Eg-8S_)LL`&H5swI$b!TW+ZB zF^VhIJgsd6Sdt&p?Fb@gdnQgK?wG0wb|BLib+UtG_=MjHiV^7nq)ifEK&tb*xJ06k zQxbKIEeCz|nEx@Hz>gan4TzqjJv%8vh23=W%RSH2Ab(kGso>56M+rQW-VyT5_ekhL zDjCD$9);m)GD_$vPGgR*!5*iYu*dT&L9Syjgo4B#+#s>%m*?QilBz`6V~}`nS6Tn- zbpC;>9aULBotYhpYSldkuS$Dc*XODpSVz!jOpmY4kjF6vc?>2&o{x@1J#Mp@9;cds z=iLpSy|_dIkHJacF}56cXD61;#a4nl+LcV_I_%ha$>4We%;OVP#KZBN-El@TI}SvE zY_eIm14msFkKHs>&}+hYySE{A zCGPAE90AYRyF{K2dQ0dTYmv}1Bo<>2hFdSQ({L>1fn;_JMPkpDb2<367QMH5yGnVi zfHG4#PiA9lL3DEruX9Zn?xBUT zX|7v>9Q>0&8$>rPlJK`yj^XFBS2+froXBvHQY^x8!FokF-e-w$yjzNJl6h8$1-hvC zH!;_6gQK!;nb?$lssV?ACVcNiISvbv z4m8#y(xJLRe?oP0!w0?27OL5j=nN%w6zTZJz9OAIdxd>BwS6?4&4%+cVY?aAohOwI5~z1aSDFUD4nQ`@%)Y@Nc+)&K^-?=oL38J_w-!aZ+b4uPKG4g{2dFs@8+cMp$nCoG1U z*#dU~s*XEI&%zzEB3YhBOwRK(kSZmV0FP(%;Cb{!#Com~3;H!vvo7;n=V6pPO{4@+ zmV@U$2KSkl_`4S4RpvkEdrWCd{BynsMZ^-H=&TifK_N0m2$GfX(;UkjOka>}RD0v; zeb66HE=S{;VS&DyO@>!osI8iywzj{G;j7xdtO&@#A5bz#^waGNGN})4$j}Fj3ioq% zWQdpr;4%Doy9GbKrb*$GFIOn_soNF&sasZ?5PAj=gq0l{>%Sg-G>~s{KFrl#_^d?u zJEsG{3g-jgDV-18w<_nuN!loC$bK)Xi^(9j<$BPZb=;;!hVekf^}y5mx*imJ74F?! z%x5*bG9w3Ib<}7M4AW{-c^qO@!`RJAv|Jb8v>uGn&JYYM98A)qjxnz^8e~ycnHb~a z6^wCI!_f-inaH)m`4*Z@78p?y9)_zA$_L~X?PY2(jamT=^bX18nIpi>gU5tYhlI`?Mh8v z2Gh3TWn9)0WL#;3jFSn-gDq~YD?n!IQ0HYEU_8U*WV}?t7e`fmTc9gU^lrlX^=NiB z{xpQQrRigItGxjpTE~9Ok zVR13tITK}2fPylP_C=Y#YF~N%csxft^6{0i&i3ihTwhnW#Qwm-xZ0t)64t@cKHx5u z;YU$!k6DR)?g5)fVqT+Q3k7dxcjGR_?#9u??shBqBd?n-o@IFBi<%5?cYA~7jX_k# zdE$6E;uDMP3ki+RG`(|gN9=F3v*~u0pM$*0)en6))aAZi6jD7NFZnc zgUj77h9@GDkc~DlxlZ_WOYlB9M@;p6j^`>_K6^9nI{ZP<2L8FM5BMM4#RrEUMPi?r z#bTf^k|Hrc9gv9u%!5Jz%pNClz^&$vdejv+HxU3{TLG>UfVe~wfb$7s0-!9- z31C3mW5pD3f&h#q15TORQT2VB>&&4_I&`{Fvu(dkZeu?(qB&u0(^E40{9t0cyrUSCA_<`qj5?AIsVS*$J_|x zAN(nef6&HCZ2$c&ASF!ygFAL50C6!wH55Q@!nJT52u3me0cu~^)_Js|3>m(cw@{+V9;tGc@=F;`qc6zwdqHwT9)v&d zRdPF5IGdQ;N1B*3$fAPIn42Ni2W*ESaZjqF$Y2tA@McToAz>2azxaY#VZBS}Gt-TdOVVRz{n< z0ha6NDue@PyWWW5M-h^PP>AF^%{$5FFnTgJ$6P1^o_I(B&w8_#5*}RGB;j#MlY}Qo zrIRk%+!37!$7x(%TLkGvVAqiRpV%^PJiMG^Xi7Zxc=Z0gDdX*stXcYu88>7+Lw6E; zqEiK5DDWlpn9U4&Fth?au1uiET*b)a*;ecYr<#x#u=+8kFjLSN$m5jM?fC9ozSBvp z7(B7s`F-7&+dRKMw{-jS(YtrOg?rHBscZ+p3HDB;WAH+>W5U;Cupe&Gj1Z!odPtS* zb%jv1YcL>X6oVbIp|518Em8sBO|n0mg15Q-ug2O~43DaJRe{$D_Q4JSQw=zluqOZq zhTh9V?2OPw?9F&d!rq{ljK49x8h?`kq-x5ax|&z!P2iPazhuJsS(P$b<20l0LT zNV#kHgJ?ATGx%vF3-~#)I{LV>Nx!;D>LAZq?=U&TerG_ETzhQ%&>twV{{UC%t(Yg#kOF1ydo3%IMxSu z+_nVnyt}m5{1~g>p1s8-eK;QSZN(h_cr>{-S$>H;3-`!Up~oB~&^uEO6d_DHe5o|) zzy~Wyv|uYir&^=h_S-cJYcUSXU9YWtCfnwHPoV2GqYiUL9^NUC3PA~|xe3c~QNG$KR8Qkd;-4+zz6r&Vb| z%mNpuMzxHo4l#~u5aT7y5aVqXdv}BQ&!L?zeTW_2ttF|=Ff(hcaRR=EJQWXbS=JUAu>o>zZI>#u;Jmh%>%Y!?_JBQ*iXf`H$fh7n@<R(>XT|T9aO27@aN}};HGZoM&Fc)Dams`l=M;&JqZ-V- zxJhE;n%*dnCfNMdSYdt){BJCDM`x&%y!B`}$IY)_o&0jj4h#1@o@Hrk>VwslIMK_N z1{ouDkwwN;L*Cp8b{OMs;qNj%4#rSyGH^m-i!+=b8{TVdAb_H%22A0}sVVSv; zE^&mFoSR$YuFBuaqT%t5p{l*9Tq*%eoP2}8L*fD zeDe;S`$5o6+%MqubXrk__rEYP>7l^=I3+27WfwUGFl}X000!BE`yXQucheVtK6Iy4 z5@=z~VYE6W1T(K91gF+r?l{)(Jg^=G4|@(gD|GyT=_^&sb2}$X&DSPU0g0mLakQ_V zm%B`KGushvUl={>mMCRCu-F~YIq(yVF~`D$!*7YOW2BWhW?4RE&))Q#jT|h9dCqZ` z=9yu*3dGReeIaJSU&Z4cA%-pXe)!plsv?vAezcg*FRa}InhILG23o-80&9?p#M-IN z6=1!$MF&xE^WB=(qmWA$*0q5aN4st7B6x9=+0Hy(U#uSuuisx>4vq4s=ht($m4Gg~ z>nwM>0t!={aC9)(1~mD?kHUJx)sf{+vtL?okf|+=H?UZuyWPr%XmJAdgk`;998J2r zh4lu~wRAVATHy9Z>lnQv*73ByVx8VhUsiuQ8D7tE zyAo8bS%F@cCg{F2WKr@ z_<1(GvT|j0=(Ew1iav9F2Kpq7KnY&L6P~I73Ns|3$82Wuy*2a_d1BY!H*bNwxI`k4 zQwj2zqa1ngQ^nnhYmP_s0G|5*TKx#nKOR77Ay+df12|O3F*pNpe5V8)mqvj5uZq^3 z;y_Eld8>@Z@X?Vz$IovEa118L+l?@+Q&SV3npHXdS`~YmQ3?1? zg+rn{zcl#qZsmG~`Uo zfXnG@^oe_as=VFgo|z__0W%H@nDKi(Fz; zL>ShL>oQ&TgRbol{c8AuIW>P%eShf1#bmS?pFt$>1!)&*rA73~$MNW&vC%m47Jnsz)6#V=47TqpoSE45mzQEgdS$BLvWv^LqJDO z<|3{2(iZfitbR9|i_A(%t3j`*T*LOkFGlkVYv;&EvoYfZDh=2XeN2U}J&NO1fG^;U zJHnsdR3tQ&sR(2e+!feyivl~2Ca~jSG3vO$LLH}?Q0LtZsEbRasfeLS&~a^UIe;`O z&9M^8;dC^4e=$PqB0O1f5o;=`n>32$B?g0{g5D_-es3HCz=W(zq0kbDJCxCsQ6m8C zMrG*}e4eRKaGPQ>;Haig@Sy~{wx|jtu5j;WF+3aITv;X3Xi0Q=WAJpdX=oNm9l_&aChI_Pil9+Zrd;h2u-MU>yTfnVO=nFE!He#=nM9% z|GhSMnW@Rsw7?o4qziIjsJQ_U;{s~VNK33SNjBCvnP83ff@IT}q1~cdm7{kL>X2q^ zqa)I=IQs(4qP@OAzZ{;uL!JB3+SQ{5Y}3oXV_{r2p{w{gsOViTiwHl8;5{fJQrOJ# zB^n!WWtJLnRa7=+C{fw;`C^@oF)yLBd3S@(rphfZkvf~+h79ac@I#g(uu%bfzczZ0 zXE=)MR(6AQ>_4MROKI0s!8a*w8lNhk10RjeF@b{3aoKvf{9ubU9D>X5nrY@-*p)-TuN%g7rrER6aM8DSRN&Rr=MKj!G48O%VSzDzuLom1 zSTHbtt4f;J8SINwCi~)?Vs+zaU-reVl^E{^?aL_wFf*NhYjkrH`w)%Q%N4=IMi>h|RCwKnaR$CzPG#}IS{PWKZkm{611GTk;HdxdceE8EF*ZAbfw#F~aw>i5%?1jC-9TgY2u zL*}}2*4nkDW|8BdJX0y;fc61XQsIY(o*j8&?3vJ#=&Ox;!8?j$<%}emMjbHD5xZ87 zbM%^KoLWIR6oSBVt$aydyrhBNY3^GMcHAg~{gqHbP$nInt5Iz$cPuvjFtT{xj)V`G z#!}gT!1#Wmy#w?IT(c2=6e56KDq=z?+NDNffVw#IuyL0{2%PA|#io%F(*X(1^0DbJ zHchS#;n4)WA7^tsR|$>Rm~nOWGu29l z)I1+Bsb$c?J{vMhI&eELREZJph?9r|Oh`rr07Ih!Y7!|9xLs>Z1^8(Z0p!LCXn^pQ zWFl#Rxjh+a;qncQ1{|px(tzLCy-@Oa`rae~tV+JJBteUqD!y112#j$hbl@B`vbFHk zzz8kPc@T#kQzf)=VKQo>X~D(j{O+DzTfqum%iOEojR`MN%ssGyX3>Lm8!dXdqh>ds z?c{Q3EkA!aoSB>DHd*xL&bTSRgKbol6Oe&UOOVNjuU@`>@#OT$n}V<^Tye7cfu%za z@1u+YGc7=7s8ToRL6R22Jq&t0O+y%CY(!X!`Hl#GoDJV!8_QLnSoO}e(!A{Xm!dmR z`aK2#zRzw*;fL4iPU%+Ce83gsi8u#XnK;LcDdL<#sYIO9cWWum!Dmh4oWxYhYF(uZ ztuMtnXGV&19?8WyH)}98IxUYL;{5RxD<9Ca9b#Kk>h>lrO>7QCRbh^A(6}6jg($~# zt%u7YwNox9^HMz%`4z=Jf#u>lEX6w=iqV%ce{OA z3(_(E9{hd8rW2@EGragyCRNm^O$&Ps-=Gpf>v{(oa4oqU@9Lt<_qS+g30=NRKT&1# z`f>-*pvmVDa*Dl1mE2dIb7g*CPJcAT0;chMYlFq(;k)6;+UmZxZ1Il8?g!yS&hEQ8 zyK|%8h~4)g)G)IVyHlq2=XZb+^E+tEa?o1vJMZqV(ZQ=yp);1G@H<6@#qW@fJ^20E zXf`^xS^+*CoA57&GpuoZf9v&lapr0s(Msx?_`Y?Giu989^2 z$sM#HPysR6c|8yxVhP-cp>=fmn*uM6kp-E^RW~0AG_B zrwlG7y)@hKrBaM@gWg<>Gl5$HokUtO$R1)G+bCU(KF{%71^?5(4EUQkoRMpl0>BrV z5C@Qg!|^SW!`O+AVR3xK*I;oN*kd!6J~Y>twAmYz zt-Vy72c_edv-$zev%vR02*c9bh}8#Nbs~A4rhdljm{2VU1L8|vG%jq3x!p06?xW#F zP0a47F=S2rI8)BX2c(BW+aU0OWOf$V<;>0<+f8QYCw331zedw;Zhoop`a_)SZ?r-D zsdNyIS5)Bm98)0p9TO_~9T%*mc49Edcg?$FmZN_=VHhZc3j1zG8#hk78cvMy17)h*Jd@BBqdIb5+`uQvgP3QNSS(3FJ_u$uX-D z`;#Fg_mk8z3V=jVD8OYjB?-7#Pt;2WkGHrVrd1OO;L9p_b?z5LA}K)0l7rrJ_P;|D z(K6mZVLiAXEo0cdSL6Oili>&Gg`NuDtc58hM55vWfKc%u_m(OinA>4>oJ@*gpyDxC zLd6dT_5!5Z?J)tWn{-XfyAR6Z>QvQzEjbqIRrr^-#rEqL5C7-M@%{f>_7+jPemp{0 z9b9W*U9*m`E?G{#Qq4H%GlSEa2$qbFXGDCCUy{vnAmegf&f{nz=lSS}$MI^5$8m2H zkK>Xi9v7EL9>=Lf%4-}>Ibt!mpP8;Q{txu+N1oh}^3SKU_a=|qLLSnoBkcGvi8|(^ z4t1RFjXED4p^l$l4Rv59QO96%xsIz$)Nw-(v3_(hLwA6Yy?xg4O^+5Y5R(S*)qaYV zR1bLXF_sg46uKRXEJB}|7|HZp=bzz^V~TF)y@_ha`(gsX6&3-sQ0*?N(_q;P^)LAz zFP9X+!b^n&7%w)7pdai1HZnI^s;OwVJv)q0<$5SVOW5P54RspudblyZ}90#a;>gCU{iiAV4NED7-xdL z8g9pBEsZ&1)1k6?u*Ix}Fy^qn+z8lE8e8$2V!ky)4!5l~A}4_FX2xNol}{t1J^z#0 z_zZAp+4EW>&f@FRy31vH-sTk#i5Q#3D3Y^rAmeOIY%9*@vm=F$+bqt;sV~ggpuU{5 zoyIZNrnKN}OiPDefhXu6u$aJ#bN&C0emVUg=?PSA0b%8juiO%s>R7GQa6FyQ4Pk<{ zamed+CN{cWsuHQ$beHP^r4~u5YgtJlsSO7*YQw}U)P`p#dIXdjQyXrxsEx9!p`SEF z(JGw=d{<6wF8*YB4K2t`@fIJxee3!_YHH;5@Y;+EHOBohxhnkOcqM9_9wB#qkydg0 zTdbiKdStgok?>kelaQjiMgH-XeM161=NLpyH(l~_dSf-Ncs{%wn|@&<$mOcG)LJM{ zBrv-+=`qgjgFhe~+g`xLCcz)HHo!rN^Pb%uWA_XZqY^&K<};EFw`*yn0>h! zlUezI-5_RsPlp!^VaC>ix7w;o^w>6}^?>$W0&mdJ`wZ}gAKu;PHWIOMNOqU{ngykq z!Ulh73LB44H8$>Usj?|98dNs$Yf{;?@5+#X^!Lg5(&5(&G1&E?!aB+11(EnrZTe3}6}XAFxM{A>1q1-Dun2(Wjidk`$thsK!zKw}m_10~;mvSj1oh_b8SEdjYy0E`id}SSU_OBduU?J;uy0-aycm*qrnU-oAk4m&O3$J4{ z4MSVPOmixvel-N~G0Zd!MJj2Qs^m(#cGC^-z;w;UKfD~y%q7fYR^h2S+Rn(aE2GNR zWv`Md&=TNw#z*+!u}`Yxi-krJ_L$8KdPo$-LZd2`7J7m_t^FmS$6G60y&YNo_U1TW zU_g&k5_-K^0STf~(TQ)yFYKO~dwMDi2no1HYoyFS{M<*AYc2suL%LXLlFY5O_Y=Zu6tul3b@lE z1)N#|DL`FS)Y3o-16E(;DuAlhlmnhmx~hQV>G_P6wMGi};binTVAE2#Cw4j3z}#`W zI+F{6Lt-wez}1&WWUx#5=s7Kj0G3XNzwZU?aV-3KuaW?aSXaO95&Z#0YfJ{DDw7WI z@+LySB^AoLOdv~_VnY&w_b%ezqYN77gfO7{U{V4@emzveqw5PKWAR)?BiKu8YK#F) zw<-!)Hfl(MT^eSR2DttuA^|QHl@!q44s>x|DF@QBYYOcC8iA;k(CX-8QmRWN`Zy&u z0{x2|d|7q75^LbO0Xw=(-okpEcklXio$V(C2)85vw}vC4fWaM4-;qhcn=&DQV~P-< zs!4?anLZW*4C`4`0Df$e0ld3`3Tk8kULdL9Hb$3ILA?k7PxO!gCm*MizfhBbz>OvZ zY7*^D{(jkSI*sVy+Y(XW{Cv!h@b_`qdLlZ9TMT~-hyM;*kXlhY%&QgA>Cy3og1T== zhpXBu4bUxYhsHJ7q*({KrV_IWWOI!)K2rl33@SjzZwWFNEv_VPx4XrXUV-xMyC%wR z`HN9a!E=tYwAifspe(Gj1jxQ6OTTK0N7oa8j1Iy&&UU-W=I|^Om8#ftz3%YE1PGjQ znZy|vtOVx4LEh^C9gPfTc9~HF=F)`HV0PcLN|-UizN~!mdHimnBmBtT3Zw@6?k_HJDn z9O=T~XjV>j#u^`{flbP20^4t0&`)%~9$t;nE!v3AJ)h34+KDC&T^6;bW*ELmV`@@$ zj5IEjK$D_Zkv!eq;tm7}(1zTu0U4)EjCJL_1j2Y(Uv54by}vfnrgha(wVS<|Jq2BW zJwZ2kSO;9#6n^;NJ0oqex0O5GEV#(LEl`5uZMn7+(&J{g*x1 zBQ&bQKL*Y0sG6dGo;#Xbl~gMyv_B1NJTyO$pV|x;tPL!{f_yLK+hOniDL*8$LlI=m zj*Ap#$D}1<-YtKN#5_i3aXU^m0S_vwSlJJ<`t9vEzMzb-H<)y)MORxu@dgr&V#(K{e zCECdn#)&NJy~yL2cC@h8ai92o&l&CDhrY%;*F98nzIV${a(Hq8%TW zi+0ZGiot(;ea4ap6?U`e-R5~lu~pQ|oq2QAcPX``>CUS8i0OH=WO@=&20X4(R5}HQ zROysIG4Ref?!9tJu(YnzNm-@Zmt2o`OW--U9C?T8~=S8W0}XOP2CGVFy?2I(cD1WW^{%LRYnJzhHht8P_j8=I1!su8z*e8a5<)3;c|kJ zaJgi1JbMZ0j>lIVtTVZ|L^3(vEtylkOv{A$m8Z%zT-X%dLYxJ*r^>#;oTW} zyjwy~X$NLU4r;vKDIz*sWpVy0X$O+rOR?MF&)j`Gyw1J;9+YKKbaW-!BJQJ)noDP4MRqQyXFgA{+jNO{Aac>h} zNpTX@A)Xov0;uf>~X9W_MpsIrq`%*jQ9&us%uEK4xJ=VM}H3? zw|UE(x;~P}L)JNmD}Q_V@`>y?S@fg77nH-v`0^50hoAQ%#ZXiJZd?CcxTexlxf(K8 z6N7xU8|$hfMZpSxX(Z5L$TS5|Q#6HZK`A}Ibd`%^OFh1W_pCryP#KjjLWAq4uX*Rb z)s^T-ECr~S9@^r?=#$-e4z1~X>WVUo(^FYs#@A*2r^f#Qgodgw_zjjcaZ6~sb&8_L_QAlnuC%iB;Ztmx#Y%KVu>5X=_`}~tU8P{|Vu3+q1ww?j zNsg|38ZfQ^H;3b@mr}yCO zlab~1Jsi$hfE9`Pdu2gYi^y$2Ifb89776^$RN%J>OOSHAC3oYIc7Z$vsw|MlvIpED zjD_8Sy`67;=r&Bd=HUCH|IEV&(e~wW)&W)m{Oj%vV_YnXEbPN}A4yf#9g3$-Qs%+zM^W9AeC zE{8uYrgi+I+i!Se6a2ipl;FCM$5X0Hr2Pg7Cxtw#8gl%}RTYmXQXX^BL&~3Cpf?Rr z))@cAa5mdG`y~YinHm^Plj}IQT2D|Q9!7uy<`!Fm{Wkh+CglDEv}X3z_RP!x9>_mY z-9eDEwMO{65bA=fac9ER3QuDe6rLs!(r|-D|DqC|=#*S9SsJG#OJg2#j&_#iO3Uc} z94!`ZPcWUQpW8LFP4T-!2ryNu<5LsxowR=j<(!2S)B(dEs|$jqEh&&{UbjcZskrQE6q_}yz+(p8FQI_S74o3Y5d3@X?8S?Me_-aUh&VE zgf(bj4c<8CS@1gKX-i&*RIqs+6O-{eo0JP)-yPWPfp_lK)lt#@x8iNo%)ey5IEke% zyK_J5zU=-Q+dU!rZY^;`3)!o_Mmd``b8658ewfv7?@mKF zr-$wpezlV(u{p+G2)Q}NDXE}N%rQLP(OkEn>Ct33GbA^=VH$a)snBH0n;8}Qp}J82g zz#w~20E*<_E<F z#;*y*y50K{VGlGSKF{}f!TF5KK^+R0Qxi&~kB&e~9>fBQH{~tDKq!v&`eG{KkS1rTmxLxwOo54Sx zz~D2LhB zEP0%2;%(mDV91M0ByZ!C&TAuj8jb8l^z=Ol+Gn`q~DI_}7L9J8n>aK%jTuvh^U+r;HU zJ$KgNkn57mNg>%{Wr=da=0TZs3)?-thW55IYfs!x9lg|dxn*{0#9^i!ZB!)SxSMh< zHq+P0W=jrd(;1dp(FUiKt3r4~IEdfA2nrwBSD4FAQ{_ur z6+!*~qp`I_p{D%l^kO`-`oNa`gHk>&PgL**@$L3c3O~HH#t>6vu5sCwwAV;2nJ@?7 ziZI8cQg1AYxKJARuUqHc4Hg^VtuB!k8w0;mnD466yw)AV5S(P`Ce4wAxmj3M635e< zGX!IFF1L5uD`D=0KOmC$vj`!=pJCese@t6T{9#?B_=9+2i`_*u^ul$VYSQHbTtCVb zY;lRi-}#1d0>&+D$&~pT;Qx6#pIe3Nr_iTktB}DRedW6=xhuW3Zu0$Tcs`ts7P`5rCDJ&s5~S~Jxgs_M*zN^?d*3W$hP=0M#;K0Hj4$X5vr{yE>(xh} zMptv4m0vJ#q5|Azi)0q0suG)p9|_1oP4DIVqEhuoI-7+qiO$v>aLm7AvvEj?(q^_K zR@x*a!)U`mnv^#0ZZO)MObTscrP8L{5LFYT#oPn0O_T8pyVo&48p2{XsNm!bfmeeL zRvkX)_s0G=zGgjqj{Pv5i4^ns7hA6YIp3GnU823G+CI4$P0lUDybk4>EEy(I<#F(e z!Qu>p1$NhB=(tnzH)tR;$#G0!Z_>U@kb$0BGB@sB0dwOi75_Idx60d?oJwIQ!8Lwf zch-JAnvLfd#!d|{QPXa)c5}=nV4V77;dteeJD$ey1#jcB_3(DqR|xtsDD!)Y z@TbPtIK4W)#>2i9zQ$0b(#C;YXXD_C`TE4|l!?bGsWGHX48Tuns~kIH4jHz=9Nk>M z5avhMHxsL%?d5o4iE^9EE%gmt4fUOAZx}O?d5&L_%ONUTYHz$V(cXOYx0(T0wfE|| z{B$-OQk`2Vfl!@b>uo!dCGyR*Bue|vBD@Yb!tE^hd*TRZCi&^-{@zuhA;SO&sUU)UV!tLAsf8xh> zakCg%JeW7*tE-biAX1(0gbNC%Q3~{^MB`~Joyw46Pv$snkU1za%NvoKhAHi7Q??E zEN(tN@OQY2j*2_{o3T271~#da|o`>uAA?ufN~wlQ-W zG{tC*JJv`Mc%uV8&_+q!yPC$6Fe+d~eJ(JWh`{{rrP>QQN{N-!L;oT_!k^cCzI3Z{Q6Q8J#QXxs@_2EjA9|O z9o@t~aciS0_4JcVxA+y8qe}<6zp5ZPIb%WX`q%o;y&6)d7T;d{C}uCrK)`7&bWu0G zf?8f*&o3O;tYsLIVEntDUqVCWYzFfK!eQ~#p~++J#$xa>q2bQA)AO63Zu8CpQykkE zGfNmV|L6Y%L*~mXFK933^wSK z|D}i*m^&CHiyV&Fj9`T}We?$Fk(lN+xJE7ZrisB6n zwel9rZ}fLZ7m0Ve<0tGhEhwzK+s#n<%NgMLhe@|6eq`~k0%=JsZgPW}QU_jZTjv;E zKR%5}6%wpKMzH>>3j}XfJRqo`{BYBd7nc{jC7#^GM(cZWCwXw>?3mnMdIKlfMX8Et zPG&aMldkl9GtzZDkkeD53$oiHVqQJPyCIjryGhJP*p}j<$mk?IoT1;?BLmlIn$CqR`t!XcmM^O5 zyuOb$T{8z%j3Jn=l!9geM_e9-r@dXl@M0o20XzpM4#-+!tjId_< zQ(^CDFvxoUs3_o6I5=4x`^9R)-EnM72&~AuPUlN}ZX3)hv--E`!dceE|6}7?gj)P7 zzJ{4$SSLviC9_Vf+e|aQm~26*+fHWa#EqEyPj_jEi97r}RrWz=OT&kqb3@~Jw1D9v z-Uao^W6JP~$M{ix*E_jvSPLrQ0f30dEKtZzD+GF(ec$F4Igq)%moXa?tB(8e7j1GW3+ zEYshyoLE~0TP)L zS`2&rD+HPNQ1TjrSzz2yDNs?Ux6iiSIiqe2G{~@9k&XMenPXN>T*8@~C||EAU**Z> z38x36zptWbSyTiRRo|o7B#y@zahwOwhS8HkTr1N{U$DBZUU`KP`p?P6>1-bg4G|N} zF00Yh{=sK{@xhwklEuir?fBkT4=}ENNkOCJ_YRX+P8CbqzPnSDx-M&}ky|A&VCv=Lnc~#(FU^csOSUITxRhB)pv;zH5;zt@?zNn zs(%(2v+LjqELoWc-@Sc=glKyD>UEl@>v! zcW6*BW*eIV*+l_6xf<S+_RkNyXBI zL6kjwiO-Kp4K>>8G0|i09wkm-t^{me% z_#wt2$&8eer7=brGQZ6%OR9ZaLV9iAme70K+ftQq6x~=%tBj?)>M3u~x2UlT$oLzy zLIA{zXFu(_sVWV@fis@ud&9lX*=T7{nsGTFkcuqUG-sWYq8D0j74dGHjDE`@nvRO@ z{&;E)vs$??{_09ed*3Sm3e28gbAa6YyPH~Osvh3*Q%wIuADr?jc3D6~B|RzwUOQi> zG^;+&#;}`T9~Gw*wm0C5HRA!!?VTs)94L_ z+Q$zI+Js_WOP12X!9&I6QJt{|?c7!Ix_{k)#(}JE8GNKQDpOsF8Tf&w3@ks(J?rRl z`iZN~lJeLm)i>H-x5`ESVH^%``;`Tpt*nJ#YL+!RXF zWn=6rUP5GJu93bxL8t5^;i;@{fAu$P?26Ky8dGZ7dk%N=G}YEbl#V@Ui@wBBKl%=qsz#vV!+vbQ2cS9Bg8{Vh5V z$B&8s4v!2;j~3Mmp~+n#u;MumuAM#i2n}H=`eHg?eX8?QRV+Von`~n0|gytw221efyIB9gKI&2Kxj=<*pfp! zfD#n%H#m)0cQkxEo{SeaU1qZcDx;rDI;;<%yjU;EoqMqn?42M?GOA1x^@ijY;c@d> zNr{1fR{U@F_V1=9BsR1{5L{3Dl?PS zD2j+*70D1Wj1sEls~7B)f1ryV`Tn@GQzl{zXEyH9S;hvADJJl7QRl=q{(3C1Whh;%fbzErM)Xn5Vp^8mI6*#mWgpUV z;ky$(oj$~rJ?r_9w(H8fe~4M7rhSN*qAGMOE|s!MmeFZMR6H|mB+w?+(`n4+BKxu$ zZGQdMcQX@RB5KPd6N@P-TdcpWp(t~$3+7!}V~LJ$z};kxD(mNiuvvP!`5HUmwj>Z? zaTp~?%UL*H45#*#;Fnm6>QY-?-h*HC@;h+a(ob$V(SFR$oOnKZhem@+%a(es604n# zzC&hoYp#A!UTmo;zhhq1{7PRk?Dmu?MXtgu;Z|2VxUAdbiwwIw$`lO#QMP*&eO?a_ zsdZ8ES5J+UHbUG?YLg=Q5mF?*5+r#b?Br9N8!2M`nNJQns*lQtD64j7{a%S3?#M>p z5PA%uNF%to2|@Z@!j|6uaL>tMsV9EO5TvtngqIb(iAw$JCNp&7S`2h)0av3~Ru}7E zp}fu3v9q5%UGz|}aqKCR51SBVx>HmZqyVY4am5_DlrfPnK%cSimoKj~dbF$j7C)(= z^`>~v8rvqCcBo{Swr=Fv^z!3uG@m=o952=V;oL~Ma@9E)U(=0r0|@(EtaMMK@6U_$ z!l7O5y3JbM#Za37>KfbFZdke6wy)EUd>dAl!d}>7YtgN(ZL}_|Y_nEtR(7xRSEp<0 zSFEbto1&%8>FWd{ZN~4O<(f1k>zw17i}Aa6!wFhV?YX$Ec!#vati5u^?-sLb-0xUM z+~Oa$MA*pEV4W&zo~zY<%_p)~Dg*zyC9S$3CQcuDOG9M?@17KI>{XeZZ4qsjC$YC((t$Fzow!ti>s_-W5ozlaBj zS@!!fMYEjdY+5>mE!AJjcem-B1(qbvVv3Nn}BmM*4cy!~EGW)UZVDKE?2YQ4vz zGl!V_d}oB}6Faea7XmDtz;Cj2qYEmXzN6)X{WBvqJUH@J#s@|k0qf>vQovmkfLBEAyCZy_JR$OC*}OZ1Wh%CM%tF= zWJHz1+>)poXgE{l*#QxDEw@{bCEC?b`_8=5W!LhGT_NBwsLF^`q5pSo(Y|cGj=~jR zXpeSJM~Yfptw~idY?nH+{g%SH$D`Nx z`P@91kN+9HnZA32^1>g#++m}EvPBzC__}|-x}EKLR~PhB>iH{cG*&7>oeGf>^aQK6 zt}oF7r0A1fHqH*6w8Q_)6QQH_c6qDNV6Jf9k3Nm{`f~V4{HGU)&ucRS4Kc0|-rQd@ zyt$uS{(h*XoK0P`#c=XwzBr#=FDOpLca;vV9^Al_#N}e^?7UjdD;n3*{=h5oxKb-1 zQ;f4qR7W3O#yCBmoKI#==*9FK-l4Di)dG{uMYt($3fUg+=Z~{4)1uG46+GSi)$9>1U-b%<$-o#Ujn zQnu>&ORnYLgyQbCYO(x97xOFr*jF*Gz3;R!t{3<3{cwrLO+X~`>%KXS{N?08OXGUC zlONm3Oi=QbH7;_eM}O-U2s{E;@bIsW$U0r~H9dkg<^(q`0ruf(TsoV{r-n@D95B_`euk;7}@{x@{u<-|6KbY-Ju>H%*fi$=24BHUI zl+i`6J42JOIQrxoa2?40Vp`kQaIZ^l7XCq1yCU9JruR(s>(ii7+BU7Os?YMZ>HgAk zh|bOwg_MPx;d8I)xWsoxks>|Yn}0=V+yDL@Zd~oDq(>K-&BoRL(VfmL(_O_JJVjNT zI+ttr(mZ=%fh@PrvO?EUkwV4ieiY*BHjLt)6*i1c7IbemuEr+rW@Y=*8>4%8Y7xxj zSXGJV@h+s>N34o{;`S$U2Lu%q&1a{$rnoxUm)eVbx0qMIKt9e|^(*UHgy0qrh*g}g zfIrAt(BIK1cx0+H@%FF?cQ~0$u~mCK7^iO!-B?OJsVJeKZX+EzA9McQzitNoGBGhj>T=9^MY}2d#1f|oaJi~6C7NONc?|qepw0tRQ z*}}qIEMaDAYVyUi%2_s5in!b>kI2SN-zKh3J#FtG>#lY~K`xiKoS)u&92M;HisRqV_6>PJ)REQIIo)ROs;H;TNlSo7h;ND)DMRsEBAP>L?v1RwY%3{T*d< zm#^98#q%x3d6~3%SuLga$eXM)u}k|yU?^l@pot!XZprvPp4`dkH`7RE?C0ZjJ|DlG z+`PHEo=pCADJ3UN6eB@s`|mWA{2gX?Bh69?NhQpaszAqWy)xzxi&v$w=VE@9v=yYW+vb&EhrQ~`KgL% z_aCQ^OFSeCS*FWB^RFna#_us>=&7VfL%CPs`a|#Q zdSUO;`8S8_vrwY~lcRi{G7Zz>+Ya&RpCULlxSUJx!mV?9R$uL=SoqFOrclcV7p7$O z9jVn5T+4?1Nz2z&LQap@F zJop`v$Vh7d+Dvg18MP!@-?U6jo*MA3UKbwxBLrG*`vlY`(1Ow(&lbg=uba{drsXQW z)MbXk-JUyIGq7Qc`9RBUmwW9MXz?;tu(kh$Nb00gJ}~yzQFGqr{E^k13myqV68JQ# zIhT(RxA=#3QgbHBih%{&&MFYR<7}PpNXvD$%>pz02wKjg)g#OmnTln>UQ*gt7SqLX z_ZpiN?Hh$P>Yp3$w858d3Ith8h^hXe3j_--XD^p=+A$a=Vh=IDvD)oC9v_Ru)RS^T zAOu9Z$vdZ=N_qEGBdu(hc@+vH*wN5hn&^wZT{-i;`qVR33t2=)7_ic zq!TvjgiSi(eUb`@vXQXHHt7V{l_G@fY}Lqg!fmHh6<&U(@kbuz>T*Rndi4 zIqkE=g>g~LUGd1Hqbof6-IUR!@=vtzTa>joe$~AD_h^p2&M_LF0P_F3_BpZmfAGXd z``n3qFv)^S;~{=NlGU}v>z`EF+Gs!k8zVcLUU9mbGE_^&Bg0E-0-0)ZJ=}NUCQ*=Z zL75+CQ+$L!bM$UJn(&jlzUFu|JpXex<{D{~@K%494-5}p>m|YnEgOKvX7o26;Os_axo!+KoU+TZpe$Ab29r z|INkL$){Lu6@o-#K4d9IqpMF3ioSXLGnOayBIpGi^#|l|agX+z+TF(k=^8w|s<&jb zBzGt&MzNj!-w!LlTPUJJN@rIIZC2zsay_DHh_9xY@V!p|Y!?ct6hc(qoE~cO?NamC z{}r&!h~mwIv-#o!H#r*#>dC3wnEwYWrtxfqDEQaO*8C>>j*Kq8ZQ9G3zzc%(_>V!O zguMSTsmjCW+)MI{yZZPhaV zyW2o{w0eZ&HdP**MWg@yABDSp%zW8e#*L1+WMs(e(OA-_6SIk3A@Ee1)jk|n1*|r) ztM|USy=hJdKdQ#BEi8A~-TA_hV83r-R}9I82-+rgRl2l7c5Q;$!rs?H6(Lek7tH=@ ze!YKj=H0I*0vE$F!XR?~d;q>zdL@sMa4xW783t-LG|(}(pf?`|G^k4>oyI_Xi!wOLoWSyu^h zf4S*OEm|cAX=PXdn|*BMV#!3mD(X>HsVHgtX|R(^doR#C`4ww)S(-LF3~FxOK1-+h zXi9G_G3al^h3hmO-N?glde+vn$mv&9&wq_PZsZWI-fMIwbZM&%oZa^|y+*&iUr1g2 z+r7pm+;5SnissSCgj5B|cm18Q?f*5X{)n6EDwOCvr?S!}+IhEjkTUGyo}I)ytUOOh z5&1~1q71$0TBWb(0H}EK8}}6lKK>BXOVCHc`r7!4UI*!ptm5`IwBQ-+ct(|%wb?0d z<121_#f`7H>A|zo9z5P`*sNpvm#<@bvDyY0S*`Q~#MYHY9-AI73)Y|ikPGKF>tqP6 zTXIp?Eoz~5*H=6!pJw(atfc^7ZLPwOu?AtYPKFy8d~dX8GlN*tW_90LUt>JH5%C#H z0pFo7_}8v^AV=M$|In(|$Mg;`-Pr_~;sODEHGmRma?upMe!?gh__hm)^-qz35FcD? z{_?74q`1EfqBUhudKd8yTmW7Fl;u|lwOeG;qKJ3k%Ynt@2on3Q)(CkErq9BGl-UNp zPOfQQjG%dl4HvW9sf^bci2%h9tC{-K5}a>>?+2bWlrmnOTLcD0^?s+cpi?Vi{zU37 zD9z%TU8+%c8?u5+JbHbD?v^F(G}D`;@f_}UrCb`wqc$^@bdPA8&`P>!X19IGq_L}Q zY%=!OS!%q-q_NkV%~D0Ax{N}jqN6Okd@_V31YSu@I&a~LUTU@|ms}!g>5 zDYRc@TH$p>%p!T(wHeh8bsiHegi>v4_TSQ4$UuF{_6uQl#i<7QNN z?+)k;T zW?U4jjCGBN^{f3!R{)$ zR1F5cJ!2f*4>?cE#TVkN7=|EDPsPo2Uy+=@0Mh4uJGvwuQ)=pCewT zZ;KZ{CjL7-GGNhSGt;6VcRQM`aJQdnBSrW5VmuiwM)So_gI!muq-I3wJ-6AHS_$}TfyN2tIrN{`#$*lS!(!Z8`mzww(lA>~a- zc@t8OcP0)(Arl@+w*@c4^3@eaP`Bld;c^+1@vNk}&R-S(n~i-FQeJi1q`T_sa7TvU zw@rIF6IjeqfBgTk_odBk=Zi>{9 zl)62?{zYZt$OIAv)FmE*KZp`tg+c*&K6&5>A!vdG_+1{gPn|@-GF@MI5_z2SdlA&i zNXjE_nBIzjW@Q5~pjWS;xyH|f8Iyb-f0Y<=Aj5U+5^`Y0Ppd6pN&1tPC)sXrXIW=K zXs48quw6gSlm`h3iUJ0d4Ha;Y*ECF}&%$o22!&d#MatndH$$#VCT0`GKRR|>Jx8!l zLf0IrX3jCn0!R=U>BEFEd4XCpi@#ItAY+KUUE34q(Y*q7TwIN@&Nb= zx+lkWhuh(YVQ4!DlgV}}(SW1x?bMtd=N>9g(M%MI*`VbE#R3PrfdO1VzPyBZ$!bmuN(3syPOnt80JdwdBOp7(~25!+#QO^IIn;V zeRyOxU!LI`QOG0!OX6WLavM_(><_(WI0++Rg_%PP+ls^*>D&Ysh^{!o0!@aB3TYfF zxigqR%`bDxQ+2!~r?t@iXN$@H&?{>Oq1#+fN_eIIx&Eyx<4oaq(S_tJk=$jlR*ra= z(Hd*M%Y>DhftD{ZyiYT2s)bYS%rI`WnA~$W=4nzD^!;%2ZL93p@72!>WzLyT{q=}0 zp(RbUA&|OtW{r7YzV;cz^2Er-JqT{;X7k29Cd2?B>3 zOKQ8GFE!vFWMnh_jAIa~=f_|>ODAEL=SRaza)QCuDj0l^Ie@#B-$#R@6lZ zB2tZ-=?bJ818}X|EMgN?N9@&f8MD#J^i8#(hH6cN zlBO~k%#sG`-OZUTkbU!T!W_A3S!Os=vSt}dl5BEIq@-TK2qNDOlN!yYOw4HFE--x zT#pO~MQY3>k5s_n=vW`%>OEq8U8boe!ihWIraSAmGUIm3`tJ2XdiEkiBhAAU2(|fR zzMX!3b$SYFlr<0A1>HY$*z+XT7|osh2T?QTti_hZzH3xJqsQ*6eKtsdiunl@p|NLey?-v}ci*m6 zscK;EwSJYTZpByeCnvlL_qk$p(gMFKL9Y_@DnYLj^y(q#6&MbsZ)oWoD)bF)j9HoH zmiafDJCcPpwN*7Ud@X=2=92nr@hFua zrjsf?9Y4e|vGX0jZo0ZvB}$|Fjr#GM>TO`FdV)+$c3?{g`|!4U+DOqujz&v0^xaBL z&;6>fv#mYy^rJh2Y_xgUD$dZQRk%mNWCm=>hmR+fiDW`bhQ_p9ny4BPtn?VUC$&eq13x9$OAZ+7RSLV}Mb@~qhD1SgVp(4L zElPXi>*`?1aYcWO;&8RYpTX{x){wlgQjmLo5&cZ|n8s+pkJ8j8y`y zYS;KK3~=LS5zy7)HY|I}eT{eQ#fcrydIC{;ja;}(TTl35AukNLF@ ztJ6ymP;#nE_fkGQCq|>5?9E8G(VWA4E&(WGBk;w5iUzXIJ_x9TJ3cLHp=H-OipQ}> zBZx?Puli48SNouE)}n~T89q~Hh|z7ggzz&kP%_HdjQ*%P<; z`!63fW2lMV+a6;q0_k3cEq_x=(qlGBAWO*tvv)0qZj>CsmLny=h8sDPW%p_|+JvXW zz=X>}fn1dFd>aHNb`g{tG!KObqfa3e6q?=KtsF8B1+xJJgOkWTUpr(b8b~2Myp>5D z646rsd$7T1*hyg>Bg%_F$``0mkw0*0c0u-g7mG%Ow4V$oPmC|9C2mX;WmeKNO zSK~x|CCM&9Z*g?Lda4T7z2hnH?citp?Ybpm`jf$V7&#Hqx!o32-abANx7qO*?@V7f zk=h+3CJ0{byaC*A{MN;J1S-HWKt1TIfpQ@O2EM?mdoTy&3;5c^JlDW*D6#055{u4{ zikXl}iR6|;{>mYLG&pGD#X$^Rf**4=4e&cMbjvm5lev}E0J7fvD#QH50IKb zX`FH~W{4|5$Em*tY$VK+fVv!+O8N zw&)VLIZGafhOt&prpVC0Tw^}YdtokxCDl{_rhPCLq9pRGbPvX21x6piSRg94HVkFN zhJfIl;=B01yW%v6>L5mfUh9r zp6GX{11NLv_HMG+sKairAVR`!!vWJ=nbJmsk!Ccs!Iq*o7JFXO%zU|BlBTG2Z2!0` zNZevu!N@Dxm6ww>y9Y-@S<_>9@Cp{&N(Cvdy-(r6D?D~U(!oFuNE3PDJxXS>VZ^pG zK-GP*hcl!S=depvkxoi@u!IK@BSUV;;S{h9N_bF;!WwB+AEay_e$FwsoY=&xok_EI z;XzNo3hGfX^yYUes;G>-Fz*FPxI;k$C=ipC)1;o^^fFXqB`s>JGTvj5SpRkpJn3-k zo-mIt^anFNzhd_^NQ=5c(1&MF<#bOhEjmhzj;esWq(xDKk@!tYS{1v8txXI?CTUS< zIFz)gmy#Am&!RQPZOu`y^m1$400?RS!lGxieVGN#{y5Qx%+m8<>F-U>hTY8nJsYp? zXr>AsivDhfHMY&@VO9qfN6Ynl!Qm>{#l&`JY@R)%sTh|gXmheW%0$ghI$ zY7D==aw-N(2YUl5M1Ga->0k&t^^gwsCMyz6C~S|@!Lk-b49s&zJADHQAB^A_+hKS< zl@5kW+o&^#I1l#dgXZ{CWC|5GQmEJLr^&r)GKGvR^9kDlovVV^Yq{nW5xdm-vQZ?P zUA|u`y?%pPPj z&Mcq{;71&w?N+dh!oXKF!YrD~4x+|fY9~gqeG?YA(1dRB(Vx|6L64QlQlQiPUP-Re z&A07LRRVplJ}wjpWj^hHoaif0?5)Op!S!3IzOVSQes&Pn%MOz@DQZ9)I)4NG71&}k z1(ZxpdlL#>91V9XW1-C=;SDCcjHXIq=ZZIn6{3ApLbRI-f<$VtLN%)IWX@oeiP0h+ zdyUaaNlh!MX*8Oaqh(5J8fr&@=@J$|>0H5KO;5ivmC9C^iZ7j-_9h1W>?Aeq4Vp$Y z!@8uVsnj%d_l1q^-R^k{J75*4WVjonX&*Xvy~SV(B^YDjferBgZiAlkhu@anpE?=@j=CG6C34C=8HE zdrXmdE72o+@LnD6@wOc9v6~bLq>~aVD4~K9Dkz}>sTC1KlI753?}vM!w^&0#G6V7RTSJ)7xjur)%vQSHI%w0EFKH&&T5{MlUAc_@N_#eW5OU$8% z9F3N0=)0Agp8HiD`@OYCb$)cmD~>h~TSX;16JPjvf^CxtDSq~5hB47dWc2#;6H)1P zN-Iy@;bQ_7_DNwD*VVQ$6&YL>jdh!B1Fa$le#(Ib)^|pSY8!yuPBi@)g#)}=4qsDz zSWG$=l)wH+g-yt2Oh9Y04z4<&WR6}!Qe!0*l_t9Tb`3wr)75M}#YHH_+R+qh?#JJ*YSFG zr(P5r9n}}qVW`G$qA^s<4qRHhG}V!9O%@`WN+Q37@U(aj081GyqT9c$7K>k3s_Tcc zwgRCNIk)$O+*JLlZjDPygPCB<-r7gp&+Z8VSl96uX7J%XF3A$QEa`XGf9UlyUFywF z4VawcXFv)&XcN>1nFfC#J`3KvHbXO&k(f$$3=R_wz6e>Sc0z6)*$Y{r7nLY@dGPq) z3pSssL)Yr&H(trUR00U{m*k0d?`;R~U{bBvYm%e?#3nZ5>z^JMi=P$NvYo9fX2H96 zQJzln2MM-g_X)8ia#5osLs%g@7UWEYfD|`NXx^fJmfmWf4F<;{8o?m%FfMAgbfL}O zsK&N*B(#HzBuMKS+c!^8yP;(*S(>y;Ycz7SRhsp!rmOph^=z{N0Z#Sy(UB$)w%s5w zRN9lKatJE@IY8Pb5&eRRr2IaDe=-qdUoDeD;6cxzVEwmXlg7$rWP>`M<_tsD$f+Id z2a{O~(73s3y*f%y$jy_zf9|FQj}N{uD+E)CsYh&VD=&0WkAOB1%{R1qgyuQzWzY`Q zW3)kXQIA!gt_RhwRP2sfQKRacaN}8!`vfjxw+%{~Ts;Cr5za8za@%M181?9xV9si( z@l3@bP1U2eD)UO$niW(v8XAa*ODU8Oe5Li6O^o2-M$R;z_G-?}wlM7zK)mJn9RjZa z3I5!r$Q!UAE|Xutc7z{_PX__J(Dcl32}pxsXF3Zl9~)Gp!H|F=h!BnKgZK`@u}nsB zAqHb;dz<10^SkZ(arWx;9q3Ya4~|yjCiw4xFpT~d?>2C69J96F1m2Cev3amR%t>h{ zdSvE8bZ~7Edo(y%+mb|YKiJIK=I||0nHg#afSH9`YIY#gb4-{IE&Q@al~tvt4Bo&{ zt{KeQ>PrYmy|f*Kqn-yNn}ZgP zns<*$IQmDTWLl01;b>$rumfQ}C*m7LI9f8F|87i5$O4#MLd?mZR$H)2^e45`jBS7R zI?GPeOG8Ic*>ShvZJoUatEQe|{I`1E9rF!3&=TCrC7C&4I*%(45YD5fNj;2&h`H&% zKHIT9+End8pS=#vmRuQ_vQScaF|j4|xsWZPzrnzJe1pxnnIOd3Ye-~aPhljUI_&7| zbtrCdv_=r_d3eBs*h)zokSZv?n)0L^I1uR!m)%+h;oNwZBGTk$uDraD@Dsksq12T0 z%-E{u#aUp&MjqK0W*cvpEh^LdSQ#^is#Fbf4CFa9VxT537xKKOv_4)A$M6Y?yCi^Fks{)9a|ndOmv^H*$UnA)KF1+_PG?M zE(=t6UOdNEiys6;`DHn|WSM$dCI8dheX5fO+xJFtJi5|skJ}tKDo_;?k~DHU_JmbF z#~*wGZ`1PBX~#^-?(uevAsQAVm_6_pN}~u2kTSV)o1?;$jQH4X^eN)zeqc|nSDOpyIvE;Wp^Kj=*Ndha5cf?6z z#X?;V(?V3^7(ZyZWSC6I+3k$1U3X(#43i8~Z@8ake zjFV#_013!eluQ$Bjgt16PeH|SS3(jN)?b-oxPKARENP_yWxI@ z@?*vxGjXUC*M>VjPPl~_Qzd|**C%?6DKA#l^;p89#yY=2DXvtpLUy5L+@rWpngwz* z;LjlSobe6Y$+Y_v3t}~LpD?nKHkyksnJ?|0dtX}}zL0qQ^OxW`$CxCdH(%;aKb zpyWQq;vS7`y>p;A_F!=rQnubXK+-u%h4>YI!a)u=2)C0l_TED*ww|rCbNkJR+fvP8 z*m_9nhmqrQ4nDYDU;)%LF#EAX<+u_pcCA3=1u9oL;Qe>TF1B8fVC9ti&U0K2$;9ZA zMbF9(+h)<5CWN{f6f-AL)i+p9HY2?Ic?%(TzMLG>*9|CzMXkm3r51fPyI|1BL@$g4 zrr)~V9Ue(8I{@8}X#m60RC+VplQeGS$i@;4W>jK9z+(!e;OOS)uM=d_Yc+!^z1yV z@m3_a(MmX6+D1WAaUN_xZ~@4ExP@mO(A?K}$6oXTJwx7A6TM$O&8l-)l*31Jr2ScI zvS><0ddwyxM;%H=ak>WxS^g66R1J|}5kMld)skg2yM#z9kuh!7z&S`fb}!?LWWe)$ zuRySi0)mx*XV-XdRA%Kc9>_2v$eid!J%VF691g!_44Bsx3`xBOoU(&_QA`K7>gEN5 zV7L-3cCCP51q3S~*n1S8zWb8tIv-uY^Bj|2S~Uej%a0j*ObZDCPkfwk3v)J7eA-uX z+eLN}0#V1l;_XAF`~GsN4Jx%kKv(_QdTpyrc46!BsLD&W^3a@#moDt#Z~pEy-=~N#Sn^c)n+*-B2g>VTHMg1+@)$ieVu(e-ttl6Q?FER^6n4 z^Q0}C8GlJ7?_EG-BnKcVwrpj>GX-LoExH*)Sovs@u~1>(8g3*8*Q}ZTC zAbQ@@1%#q=q;)Z(iV+pDA)~a|hYi^uAc&Pt&DHTPj3}t7&tc1BuYkCKm1tr5W;zz- zM5sKdu`~;cGz%(5RKWvRgohZOHWVI+hdXw~1HJTMHghy1iP7gLW%&_Y0Fum%@a}^t zfarHQPv6BF+LYzoE1_Aek4y@X8gu|KT*)~}~dGp{yrF}dMOrASp7lC#H5oJbu(Fvt6y`!(!X~rF%?J8^$zNJ}woWqG8*?r>525XwR!TcGKu_Q9kYZ|#@K3^>YP#9}UU5{D39Av=<=j-QI(elQB7trAZy`L50}gcM zEM@fV-j1)U#Tr*4i3%Zl+;Qa|D@hiYl$s*6stqI_eZ>9jo`^b4cCsAB$7UlcKrh24 z88`xnmK9iJ_8ACNKa19jr2^`?DY`9&j*0L~#GGo^3WjEi?lc^LR!pWg3NA z%+np!MJF`F~NW{K#1{%jWXC!^3ULIHSF-3ft6yl`+u-u#>kBv;pdAm4k^#N^b zU7T8C^iH`v#aodJgYxy@3qwb47v$;d2|1C-*xZtHV$Xxe2Vb!HRJFIOo8P!Wy;MdM z@Q^&w?#1=+4u&JxYo_Wz$JakSE*3v4XKOoKJA>rtL>R1)-E)96+KVtwz5hL(DnY-R ztt(k4?=WsjwxJM9S+jJcI(pcPJor)(Jphg!bzAq_an%k6zM_Mz1bKCM)id&?8-fob4>Y3BFs_-UzkLNG^de z80&EP2)33Mj1+vD-)+~AGi7FnG8IRI^_2y={~qXR>u>SygNgmcT;p~Jvg3BJ9bkLI zcR-=gY6C)gPi}yA)H)iNE&MVp0l5S~L#B=Ht{JRHE7!~`aJd9Dnyo+EnLX)ycA0g6 z3Smbd2`xNx-;?U72#L|H1p7UBXCg1Yz$|ea~=dyYxN7d9X+0 zq{x2wGC#5(`_cCdd6&7v=~qb!8sP*~){womq`3m^wD*$c3S?G_S_xD6B*SiP#Z);R z7Oy~2vu82NMuAAk!RP%5Woyx4&C^W3WIrU$Rh9*nk~FxqT_g?8gYCy*OB&=|CTSY^ zpv^qRMnj|b!q%F}E29fr__vv~-(uuOK6%=4#(b*d_nbk4SkZI}`?E{W8Ohq%w=djt zCc4H~iM|$m@lK}a-Js3EWEkoT8tI_^Ci-qgD(fXfTZS1!rL79hs~9vvqDbZ1)U-h| zjnIwqZF)(+L3~BRP`jp-}li0ZGEKzz+n!HsJ9)upvEmy`B0>?!#KdX667(V zpcIb(;xP`pnwXtZR`Y%a0|mZ2P_8p>XbMZdJ&>x>z8p_LO+yH|b-HV9%Awe2C0 zGD>@h?Gg^?HN;{{M}cUDq6|6dRMWm{_#%EivK`F`*{;7$ zC)?@QS7&b^36eqkyC`S|2VF?c0s(0nCBQ|@i;vyL^&9B`VF(GAlQ=K_YT_C@?^h`5 zK{mETkX6SRx>2ILOm|xgBsUed(N-W0nAvy^yTA<~oE4i&n}l1&VhqQvwPKdt#WeAFGDs_sV{ld z{OJS;TQfWFD5V$atoD(&y~(f%bEIvi2yjmu)}rtcRz@czZYA3ve#Q{0IQ=aS^}HEF zHOeznWkqdhVP!hRr}}B~>YGdjYf>y|%&B~=$@x{GwWm5^E(u&iT2PTLp{joj;u7>{ zfN)&@*My7|ZK+9{+2Qrrw3Ch<{vJN9piDyhHs0kdXg)fHma3{D0konG)^uo;$-!Ma zn*jFWwLH|h2`L;2Av=dBcw7XuFOl^Nm$`v|zeOAGl?;N%C_!7hG{&V>N z{i>;RK2=^hSh=J}Y0jRRv8ZDHQ_1L$H`PxVLBD|$!fZ-uTa{g|VJy}8cKdbxXnciG ztlRg|etdWZlN7>#QxqWl?>4=&|88fk*^h33QbXhEWTSdyyn!sg8dY!K?FZEhRjo39 z*bc&U)Y70_xofspId4$z2$IEU%-&$vciJ&|bT^+ZT)XfHHfLH}gtA}uu+HZg;um1^$au_E~3g*E>pz+jCtL?{sDJiNysUjrm6%ZPH{4qxz zYTqr;1EVlr!4Yy;U}EJt?TJ>N4;8)m#4DfTib40+@7h?+tzaH{(m*ljyy{YI?iPct zvSaZ8Agzl7TcQ|rO%E3G^T71V7K83DPm?M)aJXZx3^Dg}(2#7<)ffN+09E8^0u&fY zfSF9U^QYOkQoj$sLwuW=nhCr~4azPLLS(mqZoyB3Zq-kuzmtTBHSJqpY|e<=`76dQ z!TyM&`_)s`9N)8>Ke~4u3K{h$gY$s>jz3cGY|913RdKHtO|HqCMSJ?fiPY|3Ky@w= z2d9l=xbT1UWhgxF;j#$NL#sJGoOc1=B)HJnQrq=>`2{7KONp;efKmr1E|uT)INh$+ zPgFUb5b`z0)EO zIy337#mJ>)ONW*%JI$~btl=*k2-#BET#M2l;#rhDarzY0Zptj$T^@B*5$2@&Hp#h3 zIF;QLto#<-eZvVf%*ATNK$**dV@4|!J}G{`Ud)zXwqK8*S-4ROc|R?9rB;S`i6X3y zLVNSw&u0StCv@YfH#U!x$2ug9LvFgTj4fLfL^V)>V_Kz=zZf_MlTm_@4T|PUK4az? zrmcst0mno%R#PAwmX3mzG$`!N(gZ8%*?+h#^k;C#!GxD`AR7>eF9EUvo0*kE8_f&~ zv|fuwqEFo{O2KwS0}_u;b-@dEq1~%!kyW;(THC1LEhV#ljS< z1>!AImvcgHs(w|qKs39=zBA%(xE@-;);>bzZWA$Dmw5{_`0yTISNA4hf9OpxD`0kN z5Xz`qUPiUncNknShdF!-|$v^UU0Hl@P)htm8I*Z#OBWFC;n(N zuwK?iX973!%P`N~e5$<3)#~@hhp`Is9KS~iRV$XJpow}%kCF;I zO@vy!iM0`d%yXIB7#F^sO&8RFtV$RDPX&r+y8OZX9@IccBb%@txH^xUS(AYcM`hGL zFt2&{?)HF(I%SFq5Vg;sXZ#Qq6P=hPfwCn>E#Z|!OZYkInOdUp8p)H($e1RkhYr;S zOfXulXV?GpzjVCe*3u+{xZ)xoW9ep9JGBvokn>g+A(b$Ly@ALfiUIO|Vg zX633xphTqlL8msp{^@bC_<5yN=L`z#a9L|*z2}~j3X1yMxbGj_HfUcPSRrd4U|*X6 zMc_#*@#qYrLw&yWYPOy#RW9!^E-Ld~*dLlRj3Yx2x}ck7=|~ay{px8}5gY3(9bp0Y zwSEwO*yZR+uxp0v__Tn(f-ekd1MjB=Kx#gj7B@Vo1IQhV6*L05J&DQ)k?k!doB5qe zRZ}7e;w-`~=_Pb&nCu~zfKt)=U<-pU*q$<1EVX@6%!N(0O~14OyS!y`T=5?P^+mNZ zsD9x7W9##4sRHcJ7*!Rat7Y*YHQHf2n5%peW$waHRJ%@0p#|uB8Uy-|nNrB9ETt?5 z0SIv=ivMWw?nqE?VE^%eSr!&-uiVFXpe~s{XJ!nV;M#=XmX9jQP_1R&0wBol!O^;# zJ_$`87&^fh%<_r3kMBS~$<9S_AAw>AH4k8yt-0GaVM%W_qr>NQACZoV`&gaX;dHUg zSDEhYGr5obG}g5KBZ^c;iA=t7DGL7MJHT&*Te=I;vwo#$BmXfK`!w>og)erNNgFMO zPApi5NjI8v2|z{;2L)bv-UwvG(ES97#VxbfvV)Y=*|mC-YfXfIQOYnW!u~L){bWBm zhzyac#u6U=zMqo^Glhavsl~mm;`(SpFvXfQJABXrn8+kY<515O9xVa|EB&8SpzNXZ z4bT>&VJ@&!vD2`eb@wSzB=odi>fI^Pg3l%0g)d_TAI4XUVxWwi&+ zH_*{BJKv0*I>bt&Kz+Ml>AMh}XYx#Pc85ziNre>yAr*YVmbsC$n`y_ja7d5;+^XX1 zMtohsrdvqlVS=TTn@tIo?sKR#7jP@iB(s?h+?kYM8r|9aU^#XVoi$MK%(&r1lyugH zOA>n)XYKqoJf=8ns#hH3w+ba3&lo9fTZprBXHC=5F*|F4TM>N0{EReOlgiZ0bUxrg z9h^rDCK|-_#EsO_c?_q`Qk5a zbTm!TN&UuI!u{wE*B*NFc(d7Vs`|Ckf2i~y;^lYOpRpGP2=Z8|tIQ!a4z)^qpatP_ zC!$8tU7FsZ1#Mk1e4v-SZfu>BJLL1wQbStxkUsIx^d-_=wi_)}<6TK>P9#xrjIx1U2Rr4 z#VdOXQFA%r4nSB;bO)RPf=G&B?+jAC!rqUh%49A?80XX0z5Eu!)37X89lrOkN0q=s z9jUeQpMWw-%k0uJTT>2o6GL9}Ou(QCkC6a$lw3yw{6Y@x;;o#fTMV7b#qKccsGRCN z>AiRBKUH4N{p4Xh4anfw9eUXHl$j$mTcGRG$Oe0f#;O4GkE#wEqb3Gv(2}Fm5Jph! zGa!>Gk(@cCgzO-OPqu;50oXuuPv;l6#l;1sR$z?+Yj9hc!x~LQ5DNM>p7%?X`gJFJ z$uLPu4;Q#WAC243X`(0^(Y!UDz3)_l?fMen21qR_aD#Ssz^15&8-Owj+)&^KO)}`s z1L4RCPCcOq^@$*2natzERsm#q?2$aT?v*lFK9T5h`2B3L`a_l8=}#(bZ)bOt$Hi7n z6U=_SsQAI56v&bz>ny;ZJ+`)n2a&d%{?ph*{qn$52%qz)tM$C@A3nT?cM*F({`k0^ z+$?6letQ_JGrCoul=D-NIP7`C!nB?$CNm^9P@HN-bGK@uGdoithQVfvwaRT~`s?EZ zEUB9*fX0K(6iB(%_Wym!78Jb0p{5B5QLSeAxAl7RjZG8?eyE9}jnZc0|97%x(*$B4 zYMNQ1P~@bKDspI<2;Md|M>qgGM7KR@D%y3iv$JE8hY1^;7m>m)42Jn9THCp=@s7W* zSNEI8>a=A08*8@8{=i$T^&Jf$u|>kb6z#!E)eKdIEPqsk@p(syWFk{70RG(iixxv? zE?MCQ)HX;`JH+&KuIXBLhPFQW z9;>=c3ITROp4oLY6p6Kg)K#m$n(|=PJwR8O_F`qAuoMQrR}~r`-bZ{7y~}o(sGK;# zQ5p;t6%06397(3{*8)st?h^Y$YY%g3MSpv73ihO3#OK)yL(rXoJ#9JJsq?O#Ox>ar znc6m7iWb{OD=!)ZV99N4uh}2o#&@vWVQUR8)YjgNKyJm}R5@o=6FRj0s0ElZEek|Y zt(@gzK>(rr=L}ser;in3*=6IFQ^5#$KKpLd$of;(XAmo~n=bIm0EW{r%DN`%0}o6o zYg~$=td-KiT6=8{Q*>O4I=uu8xBoktc}KCT|woKtp4qmd(V zs=L*CJKLzJ{~72b238oSJQk-?_ui~-zo~m6=vWe6k3P}0P3CC;b9vRS&s$Hb-yq-T zGfAY4YAhPQb1<9n^>nlSeY{mE9kf)LRhi<1=+E$}pK1Mc@Dv5I$Z4=$wxlE)lcQ~; zD|{%4#wS6>Gcu6>J)_K!4nI;l4E{T_I2#oduAHV&_)y2MD3fZmQ9z~oH0aCyruuTa zx~)3RkCdtcWI6F0OGsWl#SX01)=El0C#;nqk1+VcybPGvVq)P^DPJ%HWRSOeRw2BD zfuOi8RL)LTd4YfZ4pxXecq#4{%K7wGGh6>z55LlM^h_x^#_6TLk4ECN(QV)gr2EZW zG5JUM317(Ez{wYGhaZNW?I7$zyR}hv_F?UsEIId3d5UH#3Mf+VqkhU$Nq8%PAPK(6 zk}T)&#_~jqWC6>oG>bIxtysRF(ojf8OtKK(b@>)7zr%ekvp|#^+2tHmtb$igRIm#z zmq%Gk<^wiJUeM|cB%FtXQN+<|4GIEfnB43f<* zm4H8(dk-`UjFKCd#&kJx1pY;lk0VREH*7K4A9`iYAatAMp^RkKNKliwRI>y1%vK31 z81ko&c$+aly!kd0UTW5j>?JzIwE1AQaH`va!BAB8Ii9ct5*GZxry5&{W6S*#6Kko^C(It#du46D5t+3$Y3E zDbc=Iqr3TRq5ReGxgIQ*>eyxn%7=AQLmJUD%5DM&d;H-&lX&b8Py8`+J7MF-8~kQ# zVzUhdk-1x~r>ng_KZk&vycoqie{S;}jG>ULHyAa=|2Wx8#~BIy^MRH>`U9RO3k4bkV`4s#2{(UMzB&AaXLg;qFvS`PVDev7wsl^QMk}3J+yF*27QXI|@dm};c!nNs ze0vSAr04#Y@#U5tF@1elIeLvmQ!^-~(a=SJ=rZ7!R~-duQb#I5avS3wump)CrsuG; z7M7A&+}zv*8IP*%=Jiogf2{dXQ8Oj4)C$+i3x61Vyv{XXz!LjI+ec;)6vuT#8Mg(m zG+RSm3k*i%vAuR*h5a!1Hrdw1UI2$^uhPW`mHI7>h zEl8A}T;Ql==)4?_=3Jb~?Oz~G%yTATdydo;%p{Nlwgfe&=fPZ!1e38R1Y1Nnc!wz# zdpJ&ywn&qqq7o<5E^K>z3^N7UA6nEI^8j~ER=Sc@tl10LQ)`6@1aIzbvVc)S!37+| zF2r0U7C$h?F+2~~)8xVkR0)$lUb7IB&h`2bpu3S-_`g!~L&m#Een)2%=JOim%^S!J zfS)jc#G811*mrTe{x+R#r(adpA!hmP8lisX(2$On5K8h_6Zgc$W@sK8WBu3?k4LNZ ztO^ZI?;nC7fEfXA)<5&~031w^NWxeLYFT;$4hv-hy)6P`y@PVxVUr33VGVVPDFoFW5!7+nGvEa zR>Z%eA(k0H^4(F+w`}VvA=r}efo(KYDBzM>BhLHB%eRjJJ+*6mzXLHDLPP(g;iO)*i#Sl^QyyNXtSa(Eb6p_*OMST7QCPiHPV-)*Nk{ilp#yRSen^ZwkyMZ9%CM38 z0)p}~&>~kc`%7o}g){KllfqbT0@KwJDy}1s7=9vs*M9?YgG$sHDCjKdm?^{%@Cp~4 zklZPBgNTA^sg(ts>2Q>?zeJfzIcod~O8KA5hspGJUidLpNC1FbMAYZ(nHh`hnxE9F z-@v^Xo09e8n!Lf^RgT#9>-y2?r;rQn`xspk@_fBF#W*n>pUDE@za?MH&00GS6em zH~^K1&!y03u=ZRz0=TI$u=?Am%FRbfnYGm5wsubov>1Ioy|6rr;h)9mj8Sw%GOg@J z*}Gk^FEk%J##4Q&DJJD#ziVT+7n4%KNKs2PU`8z)Gs@$ONm=R7Vp0ac?I+uY0(~jC z;6=8MFfK%}L`R5;qG@^5CMAK%TACnDU^4PF0ZNU;At%%A{Ao6>jE9Hcw-x_sfsP*u zyiN_uE)OEDw_r)bPcW^ON?Vvmk}{{^GMaqsi~D7`x;I#v0wPo~G&`D9>{L z=-x5@c3&2&o5^C_%iEcGz<$Rcsdu*JBXL#St4RnxOdxL-?dc0AQoDlz)wv`hoHmZ( zAgAcd(C~)m17dgsy1wh-ybCzR#M;A$XDKt+1qXpOm0w)DE^a;Wj^n6<$;>qfB1e%J zgd+pCHwl@hBIfM!fV$k;BOUowWgxRnb`K6k%e@dz4m3P~2>8FA`kLAkk1&0uS|W0yu-)_tgeIo zp_$GNa=&_-Riwn@z^V@|(vgy6?h3|TjtLke+c(d&Q1zHiDh!7kIWuJd=N!%SRkFC8 z-#$_5G7LeRXlI<__qy>*LV`2-3_?0?g?BUjk@@R=6PQ2;^ck4Jm@6b1xlu8Xr}G7s z4bzl`S*BjiP0-(q1-YHwO&%AjqbBwk7-m62^GGb<=(-@l4qYi^?8N2#-?Q=Rt~$u4 z^7L!s6eXV}o}=r$+5JzTK$j=>Gq9SjKO+td|Dgg4bCve=^Qh{CtdfXK3uN7BA1Xl+kx>cBJ+|X!lN&5BcV6iqQHnkX!T-yj^>3 zBlZe8&1agb?nlIaWfNx@7lV4g>HdfTh>OGulMP3P`N59Z{E zy$&Gi0l73H&iB8o`R%Vtla8O%FKU`#jk2|6RUTn%~bBt3Om; zt^TAs!P?`OAZSz5tQVWrkvVSmY%I>wd;w4Zvdd9Cl2BExuh(i(#J!_oh|4UPHgZ(1 zx2}zj$_*#M)(aPk?1Wo#6dVzSvLCS!wOw=(+cfUBWB&=ocj zsEH9!S!TwmZShN=A9`Nn5};N<7T`bBBVwFJY#@KNm~fH>KDz|db--}-a&@Q(vXJ!^ zh*35I+x}#j`YQC@akK`)?EeX}+=`^;7?{14t?|yKnYZR_nQ_Tj>@j6pB!|wq2VqWxa$PY;0ka~43%Eh5 za~NDFDx6n%7q`zNS9#`m>?&z2yf$*j^`UeO$jCUj0yx^z&GBX&TwzQ4l5wD_R#mJK zv~g|50a8ZEIM^;B|J-08CO7>V>}%)gV&sbXySRhaGk&&=1AHPn08Zmd6Qmw?m<{B5 zo?eByK)Y~Kp*@G-ICAws8Y|aM($j0?HbY3bJj5#YR}x;-(}&dpJ~Oz$Wy=O*+%ZN9u_}LH{wIZm$=s)$flFV17}x zN9pzj_i_imwj2O1r)}YGe-SD{=<-oZTyucVgQMx1wk=7J9OvpHm7V-J-QLco3!2m0 zsiBxj7p^|XkDFPOfekw=cw$wx{KL13I1fa)(Zva@cCQ^E1IX8_J}j z#}qN(CYv4eRgjMgW~!h}x#ym?b8IuMb~~!RK*x$K2kvW`>{Gd*vBx=f zVvkE%9f`Okd)m%{WZz!Za{g|$28l&O^rikMUjO4rFf_Go3qt%a5|5K=h3X+1u}6QP ze`#_$!co@jzJ zA`1FMSF^fRmFlVuofZm--zYq%EJ54GvpU!aI}L*ixT_BJO~f<0ebFvSebit0b_G*m zh=!y!a!Ms-`gQOi(^Z~|e${924U9n&?a_~Sf=}|}%~<3O{B*KR9p2;(T(5Ur%upoT z_c^={^>0AP)~IRwZly!Ls%LH;c5{T?wD%!u)=-Yd76Wlg2buICzHx%C<3XSpIJ+PkmM2@7s+nIgT zu**#l50NFnaQ}9*nJu^0Gom=IGd`8Um`WRWZMNTlL5%{d7OwLw?_8A6i@N9D_Uz5s zJ?Tm2sTY>}Vz9vd2fyAx*7VLZG+%P?E7dPQK;42b3@4%&)Fk+oi@4EPY9^q1mWimg z7&_C(Xfbl5IhW|y`x8=9z)sGffhz)}Sp;ea(JwUj10(K}*hW1txjkD)*JWh~$a)9m zvY#Bhu!4F@Xihnt4$?=ti6_guSc3Mc@WT>rRQ7}hl!NSFsFsCwC6JO2p=D~4TqWfm zrv@G*Aqk;0|1}t)?$#KR}Ob({-QB{_FZ82|b-D5KrYlYSy;OT~24 z>Ml^)16J@OxC*R=l3a3q_; z@xzfNs$4Sp1AD#Ls9N@bF-$DtVxuY~FY^GVVx#i11++JDRc^shgP-t)VAcWE6&v-# zd%iYsij)vCcTHSZ)9K@SQ$>nu?9`GC7te-siIW5{Q+yvuE!ru9hjR*l1}g3P^Aimo z#&YUh8z>c}l(4P9UW(rl%wazRlzLyEe|OardqlI4*-=x;P@c=yhTGRr7EIPz@YQl84Tw%YGVTz2G)6CoE95_oUG1kyoMNeQVpYU5lzx2N zPHq;nU%x$!RXFK!_Uil{%$PloPahVh^;9ts)n=*{&FS-fai(vZo@Lo=Q89g_%}jrN ze1IiYClTwJl5`=>M}4j-s^>N{{rghwyqjj$(VK0z9(Ic-^B9=|LjqliMEbY&dh(4; z6vV(#G@2!2#EH@agx^A2`qUg#rx>G{VqFBxqQzX-7)vvX!+8<$=8NixA+;Z88KaI- zc~|$Vr&%QzRQRn)aek&yrdqovwQ4r6#n4fkmC=|!CYdcq$`%-IIY90)!s~SWnZsfj zN9-a4la%L^^m1ojfdNM@>QKX9PCnD8T7bAxjJVr^st!4!z`{lak|N!R7Eo};=8ySy z`t{ZMTeO>~N3aX3f9`HFF&!7N1(~FI*(z>0Z86-)seU~jlxHJFbhyg1Co5$aqJ`+n z!4)dFc41{D2h?e!9j`OK?oYIyZ@+#&UqanUv*G}xbU_mki`@5CGxs)=(&0fxp%#UT zX~Yr8;w30l8LI5;R24N@9Y<#!qP_AlGev0%BQI)FX}zK*yG1gyBaX%I3^O2jSqmvY ztz}Ja=Ydb_8F(?jQ^Lz%f{xSz(}BcHJz62(UgxDo&a`SDaqviVY^ll zIU&!45#5Qomv|4GxsuHv;U`F#94p891oG9nY7@AU&k%2AR6{DZ%)oZe>_SUkQ5!;@ zE*peAO}^U90nTWt5Oa3BokJ$MxTQ4XCf+T|I`VAU3c&}Q4KM$$Kb;X?DD2~xwLl-<<5n;62xRt$rVDc( zGRo8(GHfAzqb$vdv&G1njEW;8pkr7yAp%~yTL9ucBw%h;9jIGyfE{r=_yxct0t#8; zQA!gu@e3juD$E8#iAGE`XY*nQ87+R?*LX)sxd2NxYLTVHafhf;)F|t$o%Txa2^p!% zhB#0j-&_sZ^yKzJR^3=iOUQ28qQER zetvg1S!}Az^{kYBK40c#S{O2yD3vF?G2vC8 zOENSxi4N#W^&rA9TCHc-|MReA_4>h8YT-BEmfOjn*IsGA5QktIW*7rWtVeQNmsKjK zr%BBfYZ`^71#XQ-=CSPBQ8C*$D-4@KH8mTZNmfx?1E&`p_ZfV_Hm*uKU)}u1gGkF) z7q1~cD|3DKiq{$|)0xNO?vJm3dR#1iUa94ut(}HCI_L&g$kG^?s_l^*Cf-i4)^#;o zPgRIe-eDvqX63LyG$$C7ht{l7MzeIJh$*E{qh|!- zvP2G!s{N}Ji0xH@SOi*}->G^or7cjPOAcr(>&#Q*Ohhq7O9cvhT$5YQ--T)upVA@H z2tyjIq*T7~Sb393+Jc!y0U=?Tv9}QZhocxnLAZm#%ro04^RXjic_|ujl|PXq z?Bv+Nf{o?MH9BD3wUxb}Ew}U4a;(6^*{0&jV)Pun(L*IO{XO9H{O+<9`OKD2*Y$^A z9-pi_LfK>j{?ry;E4%<)XN&cK7(!WbTwP`_USk(S2^jICWqy)CJVx8CHj{xXq{>Ez zA6)o}U1;|z#8EW3iAb{l7HklF8Y0Omb>Ux+%Ks24qeXH6je!NCA!1+&Wi?+_b()FL z*>^^C3tvU6wAx4TvY6U}7p&%C;5z$5ub)BMfoO)JjJjBW@!W9&1`UryoBJeCoprQ& zGO-FF8tYHP=EvMV-rs*4FILm>>S61mMfzp<{xa05qG>sw0Tw`BsppK;{^#`eZZ^H% z%)Zcwu6C*@U9*iFCQO@=$9C~LtGlJ1ER_j5SbOmrvWD0uA#WgoWUT$?#~XFbZUuQX z%^W(hx0=S+Jdqfea~6nw?W7 zSad}(BQ|bd(MfeQvC!2Etf0I!BggLv<+b}-XZu3jrVDwsw^HZy;xwppj@gU7Sx7lt z0+UDtkmJeNsQ|JzEdSNjw7(-0F7S+yp(dIo z5=G`GWrY{04^nfCXh%<~GMC7>qA~qgCxI9zXDa=uMb7qJA7h}MwcA?-G*T5>#nDXYV}Adz+HB_P6dMR_(`g=^ zD+~hG`bfZap?S^o5(HP6E=yy4M@m-f>{ElTH^u`=4o|CCvznuNy4cmH1q(-MH+p@sZw2(E^7cOC&(9k2L8vO?S}_MdNXNzkJM6! z)F}oH5ba(xU6svESS;nIMErRsZ;sSO8C=mhPmZYwRmTIHWt*XiYW0Ktv0v%1S zKq`Ql+Z(3E(2bN^Begl4eKOpfQ9Cf+tFe5dy8ctu_I9o+bjQ=<_g?{ZwJe2rPYVrC z4WA&cMxX&XE0T%a)KIY+e?e+@_JXlt_#kd@CL8RLy#Z5^LuN`6`Yw8JK{^!vLF6g; z!boVCW~?*78JMc_OO(z45Emz|LjV2*s1+-2+QMaetGT3q*3JM(M*?s=xK*@6Ya^3yt9b>|FJoIQGr_%|`Bbrgu;@t@70sS82k)>ZykeF6{kZ15gGIi~7_#p7I1#et_kpS~1K8Sy2qApQkGVe(7#!NS4xG>d#v9&oP`PrELpp}QIwyl^1Q_)1#Alv8J;RhuVCEen1C@dmv)@3 z(W7CPvy2Jt@uc`DQ#|Y0n%0Urg1?yfs>d`|%VG6iDa@nE9G12dlAOtD5Y~#acWdHD z65aIqKRWe*){2{VfEz_sj0Y-fWzHC#aX1w#>~X>^#SsZT3NTXiIXls!l<>CJj)>PH zMcAbSK08QvMJR3bl{PdSew+u(O!4fXpztfwxtg_#M^6lY2Aoye?o)~GH4s2w)11pv zzm5lT$c*kbjyi1s1=aFskS{Yl7DRQzTUT`%MWh|QNTci&{w@eAsMo>Z_K5c3H|E?* zH}78EnhyP=yZLOP{t6@F*NW@Oc)!dhxBs_R70F7E(BJIY*0 zoPh^!{Ve$FD9_ zeK4LbCYudw(M|$DM!&Uqu||k!T^_TJAuVTrjAv@X$6xRZFp1^oj>Z4VKv*c4uY;2S z3eS)iX@W(9Pl7tgfwrUKnRAs>K0~RQY;}NqAP2Jer1470q`(6j&LG{y?y`XLn|l?z zet56mypzJiQzM!T73JT&11RwQTIfc0G8X$oFR+Odbs%XVJb|QJ`n_RCg5voW&zCoT zzWz)+)zG=~Tv_AR#SlaJ{1Ah2NgCfYot8UGnL7gq(VJH|@rYMyyi@ZT@hpp%x(4-p zVh?l|e?xJC^3SLR&r3ZAAD!!yqx4eGF^VL`1)ZTFEA*kv2d??uEV?Rf`7MN})q1H& z87(w1@dHMINGV?GhwMHw4yM4|EMW(=gEde{o>8Vl00KTh;eCwgSb$t?3CP{Tp47jX}L`qy$Xl~aYlD<>Vi6)7ypMGU?$lm&CK zESBR)MZKRmD`mdbd7}bJHtE!Y4L51QJ*7Kg)){9T1#|vmKHXj`yzyrC0#F=okKrjs zDf5}OnP+Kkqbk{-NnpL?+R?=D6Vj%MO@%t!0e{MPe4K7?XVV2m{%C}Y{%%LC3!oVX zqxu{_Ze~peHuR$o?M;W#a_b&E*M;RN*#~w7#|G@QpW7FM*_*q~=p(0^`7)o$%?#~s zcj#WBU(mC(K7aolFdg<3t~p=~!!_IW+|+@Z{QYx)(b+vXT8$#}-vjNR-B?eI406!d zcqwHLz6p=3&)_`AY;DO?OJ;E5b~f9MOa;74vmz7mrWShUTNSgSm=#W_aa8Wv%nEh( z&uUq$iq0XoVpV8G4q9fHRz=nUxph`W=K2?VdRb))zgb+pve0>;>Vi@%I0MFf)XZdbTTC8lIZAw{YWKC*@tMiZrobo#M#)bx@hv0H z_q&EsF1j{Xx&Ur%ro|50q=<&{`!Xpo1-ccCg1aef28JzghBqo~cY+5a!WWz%X@CRg zK+_qw>u=M^cKY?z#YGrGs_k{>m44@jioG((+}phD)$H5QoI@{G3`^tk5OdjIOf`Edgo8Su*%7wY;ozE+0z#cG+(rUV)v7z1CFv_#3Y8O$Xon7C? zf{gNR=aCy(z4d0^Vzv7H@u9Yu>bKQgWu4%pL&$~kjPTzeu}zN{{pPx@2cvn)&(n#L z8hn*T@SSkSpy#i0;}A$?G*g8;-zq=a{E`T(rE=+ocfvHEMmHMf8rWFU!otZRwo8_) zqNHDhyrLW=b+cn24vBpBO}I??oGT;UrO%p;zfGQwGQ38IURY0~SsGJn-Q%f{Vl;Hp^y%Ta4&#L7|Y`#nK8<;nJsUXT?1jkY9=aE2U&^A!9W5WiCq&z zt9*m6l+U#)pK}&osE@B_e=bP{8tFGY+B|I4;~I3E`mYHY$&`-u9z!VuD_a9s)htC2 zEs9j(ME5dMhF6LcrS?Ledly+dW(S+P&6EI#u}<{1ravQ(9)5x-8jQs4)v^YL`DnW=q%XlsqdWZ&%vReGcN7C$V1(oyjxf7n##l~q^jU*v0H z&gPQ(Yw`7eKvy0=#4*v#<=3IK0nwx31KeR@6)CPJ{I|5o`He;$K{l*(yNZu#&kDgOjsPzGnaH55l(e>>tYpX+9KVXtB?_{tLAPON_sL%b|&1SaTTG2!y zDmPnGllRkV3uSrwliCu_Lb(KM6II7Cqg=J`7N}Cd`Z*-b16`1+*C?iyO2{(%rw4^o zNJ1*WO&qUhcdsrk(Pol}fDV%BbsT@ptmLA3-EHu0PkSjY}>oahcU>iFlY zwBmIC^}9C6Msd0;^fcDhKCl#u(_Nv>V5Vz)-I&kC>DIwxs|>-^wtpcYhV;XXOZJB! zWhzeh-E4ZjnSEh3XBxx_fCFNFy{z7vaTGvy!jIbNCYGK9iqYxLbdH*YWXRJ5yD}1- znoPIzr`fnNVjh0qR;aU9EcjuChCC59D7&1*!!Yo3`=^r9AK@o_5zjKx6Pa$dzxOi> zf_V6g&*qF1^;e8tf>m;(`_&s$MAEqgqNX_XrNrc%HVdA!KH`uF5tt}BsGy-uNq&WMBS8m zH(IJ}Q{P ze^8DSd|_}<)bI3ca;J-#%1fAvHl^7N+#*zvsyga0v~1t@G3g%5b4r(7h7!}v9y|;F z^I|@&vJf%2K|>r=O=>dH##iZ*8%9aI$UiGeGECMvKvMeIY555vHrHMCi%~#unNy3H zK+f*%atU=O>|PE9rn=-_LP0LO2S=+>Wd3^~GwW~h?q@q=0H%qG4(pGc1Ibp!s^^Jv0 z;hdX}%v|V5u0gjTxcnMiOz+y}xq>`vHm7Db&lSKR>>eELSZ$uDSKkwmas_GOtfz_r zFq1Y9?q;*i$YiTs#%H2tCZpS8@<_`Ot+yzr7zPz{JiIKFS_~7PDP3}_#OMN}90`na z1+f(-i zG{&A7?#KxP8G%ub_8#iZ<&18_fl+ygmB|1KxZD8{&SB2Yj?85x*vp~c#bQ=i zaj)k}V0LGY#f4XP*Rz>&8XlB>S}MskG^&|as86T;Nj3Od-TcP8=RK;1tFJEJfUuC) z?Zn>KHdze8FqUMc&8Sqe*niAWP|R+}Ppi3VR{RFWVnUy51qkm(0+P40yUF8Xt46`J zuO=bM$V}M7aazC3?pIH*dV!0Et#2-AffQT+cb8lZ&~T{Rw;>V z)#ck2Hy%vq!;+Hkx%JiT#f>; z?)|B~rLdzEc4(%xw!DmKsI*CR^!;2-!O<`Hssd{io@9Ymy+)u4Elx`Hu*O%xgpu)- zy~0=D0*$bp=Z;<;z6!QcBa1D-S8*on;W#Y{oyJzWDZU!ky%v16!pTD$7x#deeg$83 zE5P{1iruKCDa!*k$PwVHz*_}hE%<8FmL~^m!j{8xR4OncwSzB3fma#qgSf+9JCEO*vnwx)I@YRB^szfPrdV=+vfV1phr#nbsEhX-# z0~1+D)e)Zmx1MjS9*FdJbFPR99%LNV$$I|PVlbYi!)~+>EcIx8)}!@^{_ZL-jm}^{wui7c&P1@=>cif+(hbAj zZ%f#lCJ@-NBHTw|4+y9E-ox|q^{(6}SOD{?rL3q*F{2bjO7@VqO4z%Ey_<%;57dAi zJkoo3r9XqsxCFhK*!7^`67;S-Kqv|qmVy^+N8uu1<%niwTo1E~8xAp0<9e?pKT&_cx5fTUAYo>K~y_`$Pc;6VYVHm8gCY%;3X& zeBCAVJJRp2|IoW-R=|+#Ae2$Jyq1+#>RbYyaP}+&iq9f^;s~^nDU}G>T3FuvVm{?M zHM;^ztABJeV!UO=Ta;zkC&@+QwJqR1zzV#Cw9V68qT>yi^FI!EMRC- z@U--2>)L*>B_=H0?WptQGI`lqBtzLA*){?)e~v=JE4%B5#%m-`uB#P=(zRas)I^Kl z0VX3)ZLf-?-$C4z=@M=rWfe)k0~o1Wku=VPJshW1A1%g}ZWt|o2c`g`|*yu{qL zkgV5VzL@j;+uWydy#oy{^)c>7mB(5P*=RBDqp$~rulFljjA^B;t5B^6l1Trio`_jB zCXOpR)2#sGo3x!N%L6tDaRlY7wP5SXY;^Bd`?Qp0oe$kGl6}b^A@-3mHNMU_E+(RRg7nP6imqC-Kloif3qw z(v(7hHdAF1Iu%}Akd~&<-|YZ*^VvexbcXnd5`g-=Y}JJKn!qD;82Y;z((pl65K{Kd|*nLhBH^Tc8 zpj%)G1Ya1r15*)QuH~!C*MRTKHcWSa3(Ol6mF4DIqIKo?`lrXm;^&pJ4QK00*y7#0 za5eyl`gJl_ko8B`>)INv7%d|gY2N{-N>MDSz7==co8oF%MNpseys(=5=FCS<|_ zpu9xlUzD|EK4AJEp*&eAP_&8dMIaMy{;aC&;3quudGSBLI0x2)Tf!7>Es%)HNn!hG zn#bt`v?#tznhW?l&WI|9?wn8Z(`pMH_4TKxnJ8?D57~vi1c3@cl3~Uc{zrq@3D5c? zu@lXRe0LzaC9`OCbnxZtSVyZ4Nxm3+(ws$3j&P?u;bsVHLSIh!Z@OgRf3s+m@Za=q zV-dJH35!HZ#4w;%ud zq_>a49uULNl}9$~y|efgf2J5!7%o)(DF;0q%ynp^eFl3^pF!4!{N zo?>{ei6Aq@BbQjlQV`aoM_7-fcw{JYs-RZnmVu>;x5EJ?EvXMDz_*~6_%K{{fN zYMKQcXLL97884>>PT=&0bM%;{E`_Jj4Ag95P(CYlDW z&~;r3o<&)Y+Bkl1q+waGGD^Km zsdv$o(^ookB90b>q=PSv@FQzDDjAMK3sz?VDcip=vlPV(V@mdiX*NV*X8OB%4MVM^ zzUISwyTqBY!$IakJ-OwI39YmI7va9MFQcr#qW{VJRgR_)i-=J8&6 z)V&L{6FssKT8sf9Rywab_YS0UyUjThd$tB__M;vz+tWn_?h{>Gj zAdRv2M)&IE@hbj#t9t96j8w%WG-*iuhR8h1u}A}}GpNn5<_?$+M+xYkVg8Mz9Go$Y z#&Bt1RW-0e$|!(-0ra&Y)E9n|a8CmKKKQ~2MYKtq0Q!kXn}M~fOGi$D?gh_Z{r>py z($1a%dTQYv(u7-z6=?$036t2Camo^QW?U*+m3g|93o_^Cled)fse zqX8HS)f+;hETxnLZ6qzmQUUh~%PMrZNl7qyzvp!SnSSPdD4&D+f?! zCYg`bh$s|&i<%9h(@z!Tx&69+G*3wQUBZ9U!=1waW@oYp|4r{ULM8KPs7o|Bv|ne? zTe_AQQ2b%DolJjM%Wi(LNH4nzZPHW2UR;;jS3&)m}uef>kw@Q=vfy&;I0U3 zgSim+AHsi2OLv^L6uPtBi+AcvPqkh5R6D{L);bfQf6(D7mA$>6r#`QOny{&RYJH=AB>W?$F?f_LRfQF=I2wCg@`)xx`(k)Wkla1}Gs%p!$XNn_F7 z3huc6NcSoD@61#EgvY`hpEss|4%4u1EI^S*8Ph*Uvy;MjM))|tMfNX-Le>Sp#duG9 zP^5>HQDXWfrmyLgM!Nk#7I zAr@lz0?mP!kuWCcGd2|6y_y}FE;nrVq`h#n(J*dm3A$tA@#v7tCrKBF1`Y(_2E6^ zMfQg#P4l)?Quwk}BIZVXA`x_)PZ|v!r2uKt>0+Ph>i%Ip+iYN!sbRf~`1qjRn!$Il zBByO8m+?0`ryaorjI~AD0}@-skC7PFn-J1bal3I^(C+{Pxx51th7kP2!Rc25MXL3shm~ zjXfk)^E>)FS<`6(Be4;BoF*!$HjDWgl8xv@K~xSk(Hhe|k(2fPYPFb6=+2*KC1thU zc_wMta*&<>ezsWsp=??GNliAJEp(nmplJc-)GeEhmCczd4jx1n#TS=sa7|J68E7iD z;)hDm+#;P#7F8+x1@IPoTs=oLn<;``ZCK*h$A`s?%@o*cu$jWfwOZ4^Yj)5J3L&Z8^o*)H=nrIf|ii+hQ%PpPeIgFH=W+aW$WB)3(S|J&1uxO%TgjE=K z;7CMjIPp-MIQIAT>K?i;oY>!(z0dx@hgj}!Vj|V2qoQe{CVIbmnpLl>oK?s65NT7{I zui9wK*vKg=10fQBDKlj~CQZCdjl3swj(W@gF`sU)6%=uUQGX@^kIql#!9+YMCrt6; zv0rf9KQHDLjC6u4Lf0h@|1>RmQg-R%bX%&PI>Qf$P#G(zSNtIt`~umA0&drbMG-H; zsYDtz(84Y7BEq1H5rYXQ&^98u$g<7!sMIY}nIgBk8P_Ga2uw8RT2(30yh~NqNNX2W zrb0tX`r_f)YXi^S~SD~sfRed>~yLRB37-Kx2}`D{@gSwcNXyCxtj>|Y3zBR?@ynEjF4 zE!xhYDB9)rFHe;5KcjEE%9e-rCQyM zc4>!ogf&+=sm^xET2WEbFG4ZqT;@d#!`TxH?bSe{(&<3RkxRj%QA_c$|f;#h|$d&<)l*0KZA22xZA89{D6PQZy5 zG-xm_J(5|XtVC5Pscb|{`g>_MVyrutZ#c-GLN?>TFSBhmJK@#0MDG}1SACJjRSbsi9P7@3>q38qOa7VeWB$B6 z3XyI!VWQGUAXs$|eSjTujSP_M;(z0Y27_ky5F_WmGMLD-G(oj*AEVbPu{i1&^yi%B#RkkSUnNxU73nYmDf$x zaUkOD=9%M$P%>9m2`~(N7`w!N2E%wTQRMdEw&3ZSxiSnNg^#M2f zDY3;Kt}D+*^^8!braQ!)ObFWb7>h1gUe8%D*WoJ!2G%Dr9vWPxRBSQvze}wGHS6i` zhJ}iWFW2N36W{nj0XKIu@wMZPy#M?{Og*v3)?(JU(Aj>dHb@2;n0v(NV@6vT5P&j? zzujzR%dM4}6x6faYOA7uT5X|zO@C6`RjFNLIzhA=pIsYe@wPTMIK!+yAx;IcuFzQY z+`8PSF$*>SX@QQ1S@B#-jyFTKxDI~ib$7w~UOo!i`E27^mRd*!xQXNS>`qy|SZgT} zL5r|#E{NY^=uA(m#mJ2!zKyo4*>XgK$w~(sb+MhT*XSY99oq%Z4H_GUaD#PGaTx{v zW`K+kDz@URYv2|GfI?t}_}?!FHi-m_Hh;`jLJ`J`Fc7tC8htLkVCXCsFY+~I0U6XG zuatR1-sYBq)^7Rziwnr0Pz|$c>4K30GE}HX0U5p%@U8z-2lrJ)!;*JzynKxr1sKYw zexMfmX!^T(d;>~1*ibN42XvG;Vhs6;F5YqUBag=o5mfO1e`iRkkyZV~6=jftShh}? zvdEGHYMJwC&Gt->4IV_ECfL!DDB)zfoj=XSm38&-`?iArn@mIv$}SHQ*=`}A20u+e zt$rdck6RLEK@c8)@!8cF`SVwdU4k9NM)#|ysp`?}!8sq@J1Q6<@^;-0lkz8n^ML)1 zKT_{(%LT<%aj%9{%O~QfE;)<#^o0|t-NAtB99QAAaSRuwiN1_dV5UQ?j|!3;KRuim zE9Pqxb7KMruf~rl5m1%IK3cBk%e&Qw_o|Y7YsFIuxnY;$x?aT!@1nU`CRTuYIP?}h zv(?xsC9AFqZX=g+&{&7w%|aDx_mfJ}UsjD(K} zSBiz7g#Ui#1SwtuP&9t4Om}C#k(?Hry&gos2R!;JJIO|Im2{lIr+yF}BRj)aFZuDx zaMbDf1=lLcGxOyaZ0^7UCdMER_=m?$5TWvMtJvR1f@e_;9tl;>?;uQh5l?=I`M_RR zKxS1lFPdh`u*V6vG)gMSEk@m|Oo&V@aOgSLrC2B*{qEcK!)mjc-z>h3mye4DWEAf@ zh&^bc1sIbSv2ERd&nl{}i58gZF-W3CZZb{|GT03gMGUSUwS`&a!oYshQ{XnUh)gPx z^CZN!^j5rGL3p0s%LLa6B(Ldx=dw;U09NG`S0T&eXz*q-GK#5eBmx6SX38U%?wRB5po1+?qox} zxCS*Pc3mntIM`T*l6W|BkpsZG z6#^JNYM9ujJ{>Sh%pq17D}D%{GfIJU*zlZ=l13>cma}_sv>H0=hpGeXDfoh!Q`I>| ztDE2Wq1sDjRYQ@oyr$jHNfcPf%qp=(FS8nY%qHhf zHCm3SZ}h1OTK#+e@Zy{z&=xJ^6P4RFUXGU#kzXt6u{Dym%#rh1O`g4 zL_Lv#B_i|HsTH&EluZ%*rn+N1UoL0s>$B^-fSdRLKPoJErBY7m&8ofYS{X60dRA)) zYiaH!qTFvQne!%Y`Q@u~O}oY(iy3(EUXf&hoDN0*$jl+p!3f)bb4vtoWxA2%shWWk zFBe@jc`ss5ldr~aeVP!)K=bII%lW?^RS`>!7EkzP&VKr$8j|=6i%(&87yCoc!VJ=m z@ysG8c2D3PY4dOnhO2~CfCBy;j)W{@Jvp~9Y~k%NQ3}9(F_Gy>9 zGj6PXpCT4YH$fq`eA%NW|!9s5V6H8eT4$!KN#N+^3UPD$?Igc9;DOhL#hE=@|5P zGmc^BjHY7rcKW*+%6>M=0&KXV0Udi@0i78A-A>Llu?^PG)pB|s3LS?2ZiZ~IggOI@ zjzWKLHDZ8eVFQDP>l)cJ5$R6tLFQYUx2-Gb@XBiGV$t8tP>r^tpM^^;2AwAT-Avhj z))3@$@-VrXFXr2Cw7!LoMt`>>?wHic`r3Wzn(8xPPnFbpI_4dmf2d4|L8>3oPZ`WT z&$N7U*Q15GmWzbo&~Xps4M65~u7kOkSkAuxC;N)raCCbvU!A$6x$o^BX9iBV%~AVT zo^G?3!!i726UIn6RJdI&9ICb<*@#m6)K58rR&zAiSxwYdXe=hG;V*S7pWsM=A8x)? ztuek=KQC0M)O_l%S$kVc*|5J_0O@L)$h%pEt=bKPL!iqh12u({;f-zUs{T&YK&^O)#vze!%keHNC*AhpdN-*GXPDNK~M{pdmtmb z(LS&%wlQGVa;k2_yaUKuqosC>ZJ1f`(;b+Yu(~oTuVl5s2EiCLin5%`@yZ&dS>SU@E zWF(}Lf~+7+1s)=gruASeUM4!u7p0lNg9LF&<~}@$JkB{)eOB#!5@FwlIyJyiJ=2hy zr1W=Vp4!hE6V&r~2b7g*6=S`hDa{MVtV3=6}j4oUY&)jTW$zT8Q-ma!h z?8+@gJ2E3cUJqBk0h(_ISH1yxC5W3EHI@li%CBt^)xZD`zA(g;QmZ{_@!Z? z7vU`g^NWk1>fgc0U?3BsQ_txlfM9Dc$rxumZ?3^i& z)|kHC%J`Iv5IjHlf=wtYT!5U5aM=^#0(jVA3Kt-xLlG_sZa1tGVA>IXzo)oi$}aH{ zbS#W#SqT(D&-bMF(C>W{rT^&<_Qb~?9I2O6~ zlZg5?kQo@EE6JYQwe2cAz?$w&vS$b*CKGmieT9f$R9{61RgC!7(5asXr|FbvMlQ6C zbfiiI6Bys}LS8Ac$lTl{gE4N}H_uFsqLa;s5)=mt-sVF^ks2H-l`V}niUVv)MM5&^C;d3XFrO9pNs{jv7ZW}B`i)|yU zSy%H`l3WeK>%arF-h|DnM|j_MNJvG?oEhWzd}3bu6EqfFA|X}a9=sDD5I`Kk7e=xp zgV5ecZbode>KK;SIs)68qJD)7(|He+iENO#U6G48SceWL2N>P6s>&6Ra#tGhl zKL-uP^FBvsFqK`3lL5PKTx1gY0gDp@j0nIRuN)Rk5p!=7+%3AInWz+=Z^ls&11Qx& zNfh3AHjJ}+DCtUYj2#zh9HbfycwYeZTn2Apy61XHIJ9AWIHK5m1q90rja|Ia>Ex2kZuL1`J#xi!URxI3PaS%;>HWgkWh} ztXLY>+}y+zVIt#2{>30^uh{9Ub$64RtZBmh2vhfcV#|v+vJ%T*ksg0XaU%2-! z{!V&kO`@Ywsnj_ZAV}bLfZK~M+-uuX$EeMHhU{}KPv$S~w);z5^rEN{#6|oTY)dBG z(m`5Bk9m#=N_8^x=8ySy`t{Z2d6<+^4@G0i0Rq4*lwE&Y0>O@i&NuCz#*fUUc)8(_ zm?HS~@nJFhwd(ZrG+9)KfM3Ki+6nsNj6lo`ir9vKU$5>rj|vn#vA;1Znf-y1*jlzx zlaxzrJ*j)~%)Cc9y67D8D_PA*Q+vFbF02BsjNFqEsZbS;X$S_wFF_dB>?}^snEljT z!u-;A*|~#chj)mtwUU=R$JaCqqylugl^(Da@WTw801u=5lH+x{r^#2#n5)2v$kFkm zn~)Y;_-ss6us`%H+Pd0}SUV1-@U_r`9-xKVY&kOXq;Sw^wE0s7;=ar?bbxDH#FnY) z(XfS%gm@U=vU|2?( zRLAHG@ym=Rty^oFY*51|1wwru2$j(Rfl!M#tbETV-tga-%oPUqV6Gf88E~`AABX7* z8z&=CIfJkZTaXsMExM@KpY0u@{}Ob4xJGo;(p3t@m|EMH*q;qFTMKiyH=7D$>PBM{ zNl7@`Z)}#y8&Ly5(96ae*sV!bv_nJf@VE7P@{K77Kx>&iJZ=Ti+H~biAc*$V$eHYg zBNJ}kuUHQ zH5m$oCllh@t3>RK!&9;MYrz{Vj>P`Zg5N-4C`@ICU(yj~J%zdQVu$D13*&B7;xJ`r zMfbwjctBLflj4g2&r9waW;*c8%!1b~HI-;JQmSB&4Y0?ED~S1)SFbUbMFMGJX(i@l z0d3+j=0;*U3=6GkTd*40|LY2KVKlEK4K$Dq)yKl zbD9q!SHYL@xXex6)lZ^V6O@JrU19<=(yTI_qS@61Qk~l;A<2cMIp1%}YmTxC>rEBJ#yq1-_P@`jomC8UMMZ8Z{3$U zo8C6iep=v!AkUDiO#SPKUynfoS>< zx=Sqj$lw>ezk_$gNQXE_kvM2+9}tW4{$siQl{61H55+s~6#%8OMXr$*AH=dP1NO6d zKxEdMjIB{Q93B+aew?|`^Np*6(`w$Kk)${Wik0=L5ty>xC6c42zy{Lls_lCUTx z6D?%oj>s^~><eg-oRZ4sD^kb%{1 z_;ke(b`7mZqSE9^Mw+MO6P_H9BcBM#_~JHO(33HILdYb$+=v@I34WM?+a`=X(niZrlfkDyKH((W0H5cu&EBi^lx3RS zHWYdMGPB}!%tNGA9U|WwLIK%>bra$;vrqZ{X%Dguo25Y*tCT8L#9UALi6nx4Y-Q3) z*e!~;=Adg-jy4}EDmt4F6}6afD9%8!b#EaaD?V!$^ujyzeS4T~$n<)`PK|D0jwFBw zMesI_{9zUaJ8a6xkr*Cj>kiO3W%ejjLzD>E3^cEMYXJT%BKW-V#;;& zb+X224+>$xkOX}>tPpM_9uL2*q|BlX&(iTA`yIm43>rMk7=%_mg4+-RE ziC$^=KJZ-fC3HLsF&{Q_oO-fBvaas!=46<=J zmk7*q%80&QtroKh_4%EKdj0jn+&l$iIYnpRTj3b%_(Ue|Mu1&83juDU(WHI0c_ zaKZidFU(y*@yy&0?2nNiYUDKD*+Y4Y$k9t%^ptPSGnTgq31<_zUFVX8R_;|0(O`c? zV$vXQjR`G-(HOr#-+{p5V2g%qQC+mJ&PNP|*0Lf|?MKQ^TaA>Z=DO)ccXpCjRj5pt z&8mpm32H`Q5Kt469s}j|*P3DW5tjC{Q!uI@L+KiPtkae)TuH(m^?&4)2&WxTy z!uLegsd-#LI439Mc2K^KJ~mOP_KOIou^K+!HR)yXGeNIG-xJ@h8grJCyjmJMkI7ZB zA{w>#`EWHq=^+~eH(+UQz@JaahVazTPWVvJ?avXL8BI^ihQ3qnPPU)ck8j`L6eAlW z{5Qo&|G#Hc01E$2pEkUzoRhvnT^XLzsk;!|t=7=xhu({#?fcDkGX1Rb6wHrn@m`pT zxuyDET-5i|Ns-}%U*edkH1PYTtJ|V!!o8|_yZ&`GUxoo+2XuPK>+7W&`fjDBSN>G} zq_*lXW!&d?H;|oGd|q$V$FwQYdbOE-WVd@e&-Ft+fsywefK(!4P%dGB6YZX$L-%LM^3Ib&sRFk;7U z$2_hE>A4~O{^!|Nu{rKe#A^b7m(OY+n0(PUSZ~*1GI?Ymp--Qb+hM`fL2j@S+ZsKr zS4#FE$gl6_vqd3^BvxO|v45UTZvVNSZwucos=;{#9MZY6uf= z8zR!-iKN0IN&%rS;ri$TdxTg{4=fm&m`JCg>nxX6<3x!#4rGb6`P5;e2Ie@h3c<}?(;T0!?jP2( z&8F~t4d(b@j`PMq$sSQ+8znv{szbRj_{-)tzZH@_>@=pB5*Yy;9rN0{nUfD8&(cR_oQsV=nK7d;PNe5<=2$^a;$-MG+V zTPZJ*-7+fG(jQk_&}{TKRZDG`z%!sdHUcA?XbbBzi1D&#a5RDeHSsO0Lg1;{j|0(8 z08*0x1K=+&v+YkMK+jl z)u^xVpCgbCPNM<{@jb7vQ8BFF?Ge$4N4+;A24(+^MaDtdXN-BDlzmhwWvc4XsnCy;ry<8z#aIYEyJ4X_cg{b=8iP-dzT zT34rdDo|-16(e$ASDU4!D6dm?M{5x|4(%{Ha0VJx8g_rafD94Cb0R%VQhQ(mY30z@ zJ_zZc@Zx)#y|&=E@>Yc2Cf|)$+h85i0SPIr1DTCZu4jMfm6$kVS@YQJFjCy1<~0}UOK_nYb<+-X#|{=hd|M`omw*Kp+$2Y zdlaI05nkGw3<317!eT=Vz`10GAu|_7C&U_%?>=Z5B7;o;P?TU{yBZR#2~*zi`cE4| zB)%oFTasMOsmnkGZewHy+}r=tV1n{WrgQ=A8$|n%AhZZ1od&X^ zpH8%|C`lRue$P7K7YLs58hP4z&Wcc=2KyJ*o*>Gz&^Pzr91Y_;tg=Ibfob!wSgC})CN99~Cy(B|SP2F{7~Yy{3ipw|Q`IxHQOLM+L3LNWu4y@b>rrnJPkIG~moNk9<`&HlFB;?`k1jEh!U zO(1ART`kmVLYxOf8{BK+7Ie`j;5pxqJ`+g03cv4dpNWd4xJZv?9do>!6ARln3Z?Tp z8+c3@U#aUc5m}6Y9u&Sb{y-V%unFaEX1T3w(mv9+0R9Y=Msw~Nqcm{>ews}`7tMa= zw{XtP&@fXSdjg5%Cc5~%&ad!t=o7R5<>Mc!&TLSnRI}Og%T@K!K_;N$1aRJ~E|FJ- zRHofIeOu&8@nKTC;I!||8r$&+#y8Wu2 z2E}w}SFT*Khpl~uYA6E{4pwJ~YC%9b7_>Fe)ADn9`l_uR@JW(;5-pMn)LIHeMllF2TFqRjfd8-k4cMVg=8$u^FhcG4>V? z({XIU@Cmeus807~d4B8jEg+q1fgX={IfeMSb$M9!;npJ+Tg$hTPu66}GrBcHuvgHyDG3@CE zeNa-Eh%AYm5FzA7r4xlvB2~hR;6si3w6S%v=91+g+!EIuD+lxBtarf>KFlR3cmZz) z!E0Uk*DH0D_XxQKeuAL;WinS7_3j1=Pm0fy@_^NloBW4AEap>nJo3AegzVFR+V-!8 zJ=&jA@M7GQism>Cl%yd`V;GY!3aN+PPKsw5)0RvK!s|3~(V(s8(_u5mF$hfD-qnrg zF3gg=u7tJBz6BjQq1E?$F{qLq_@HGmjBSy$8y8+|xheZrs`?eZzCG6(zUC>k2jE14 zM8~p5{Ev~(u0oHYn^Aa71Ds)Nu#-8!XSP&%QXNcB*) zG6bqA$39@!XF3ZOI1FbM)75gLR3aUi_rC*v<)LV%t)iJrp_9cy7R_9O{wpS8f+j=M z8Nrh5#NuOGhOcRsS{1v|D)d1O78MqE6>JcETV>E0F=Gfi4?$;*Iq2jgbW$Q{x84V) zwTsRM%xQfEc8093H!li6{=u>|&BRLWKPB7#5?tVn90mQDz}cl_ZOxV7XJc9ZkmK1R zwvhWaHn{7UYxMxiSD+iP_X&WhJyo=AA{!`H1{64VOKSsO0q^Z>rl=#;ndvW&4{)ZZ zDnP+1hF2ipJDaIs+51Ex!t}c(187nOO5Lr;^r<@V{PTJ}`N}2=Skx6fv6oYwiT;nN zJJQ>pTX>X3^+~Ky+;w_3;Hy>&Y>=CZdR-{R$v?{#IS*qtxW$AEm+Y2FyYezVu^%-y zC<@g??`@3@if>suGJMu2w8ocSx}4ebwFFT7Z7!fSpGkdNjSvQE6|XzhDloQYB3lfO znMHiB4oR(G&bkm(L=L0&dA3ziRhY|&joKn*7ekhG{l@NB>*b-jMHj+ z%pyR2qt~8I{5r|r)%#Al@BD7New-a$ybeW3$xgaa?Zll-WNGC_$~Hv;)ShD!K#2lL zCVCYLfc?xQ1fp%00Lzn&rX?*!N$l?a6 zaI-%&AYo?EL|mST-x|D}D%*=V7A1o?U|r7dAm!yXVjtC`jMxwPu^d*?loRBYe;||b z%>(2NN@-0s6yhKqvklYUt1KRvNR~KeP#;$4rb_RjiGVt(e(l*tmFnLN zqZ2|3RrO;$or36$_!vnYZ9l&hrfdo1>9fR91s$|olpmtzKv<2yE+QNXopKU2$2k<} zyTU7tQs5~RmoEXeEqVzwYMv!xsN~`(aFUrb3Jl!}GIGRmkP*2G1?pA?Avv3Z7m}H~ z0`$-|3cbIMMu99=me~lu9Vibt39>gXRweFDzDX>jYvSvIXc>0aq!4kIVVp*Q@Ds zFC1)G?7P>6gRvqyW4)DvUU9Kb@<@>nyM9~FcVHGBR8LfBnLbCvDK(e{SEz{pI-t2K zb$9dId^`Pgbnyx}5{undTUOl|WT6V0dT(=DxDoTuU|&cxcVgOtW}YQTI+RhVbXJGo zXGtld5Y|-IQ=tTJV(>3t$jI7SN8JhX?ncf6K7&Rqlsu>iZ>o9QQ+ka+ELg-ne)6{c z9rhORC7_tmutyrgcdApv_S5>&96k|=atr@W1)2ZfGm3e_f77Rp2rK6#C>x%HAKKMD zmv(iu-Mhhz)xGM4aQ(1esRkn`DX;J5vxWLAqX*m1v&rp0*Ym9!N_i2>-)1O-nEE|c zr(Xv<4iByMAU=XDz-fe>9uxFAh@%njX?>|%=*l`O`aQ4T5m*UJW=vBLaqHn8$QY#Oh5mXUWk7#R<60m? zorBe36HTOjvg-E6Ldvr1y0aJ)x3Gqfp}RTf?P1(oefPLnz@Z!CrK)#4{PYDUs~4n> zAYG}&G7_?Stv1-S$z6xZxrs=)M}{a;T1tcw?<4}OX0$dNM)KrLP!*cF6^^P#4m-9Z zNKK}|te{Qwr_|umd67Rh>6kMz*5%mnj~cT+ z>rQN?x~gl;y1PjyB3_L-=gHbE{K!MSxEObX&YQ!R}MWd>Mqs*NEQRi zVw(G+_XN47DviL5xS~K}TsY-b72|ZfT8}4+l)oN6Cv|)cQ0r5JleWeNJM=ZA=IiW} zhO$z2IW#WITf3KTHkmBjRAEY?cW@-W(Y8qIqZ>R1!+CSZ;f}F zo;%dyh7NI;FCnO5F+?CS^DF^$na)z5=F?b!8dVWxCjj zwlgf=sw&jX3-8}Dht`)#@uKf(->T0umGWS|_2p|?3}h>p^oGDGhF!#h<>xXcg(O=m)DR4(A3&YAN;j$uk1F%vtmU(feKp-@q?}qIZbo zEI+(%|{QN=_? zHl3JBD)t9HVx?+ClQJ5Kp47b+U=;iCSnC!}2uEpv)Iwj-)d{hA=6jY#oXE*)NY*`( z*;&xYbHN1Ff%nh#fLN#{sv+|$b^Dzp#~_Ib@`JFsn#w{B>w>pExw1e!1SJor*#er~ zy6z4%!t_;S;D#|rOhsFjblkM~Q%C^8ndC8o#K;}xi@rQS*?`5LS{s#AEILS{ziMFH zL+vL78Jor8o!Za3?AsFg9(Mxl9~TZ-F2;s}1+imY%r=BHdxyyiI=vo&>a^^2!Xph# zHeA6_$k8}o;GIfV8!L8DoR9u0bT~FqWL%*h@b6hGZ9na`z^pY6=UR>^gb9$IV_Z!ak0%)Rc#ySxiV zromi70WgG^93B}0V37GtvNhX351R+T(3xDr3!wL!k>+kAV2DUV1PqRYW<0jn32#A1 zofEfHD%AWII>8@)v<{SSt0dJfYcu>=gSdWw$_^8n4$kz0h+9q5&-WZLI+skpP=IeR z{k$Tf4fqXhkx**-O8r!Edz|TIXqU=UyFA$rJ3I|1+tCB6epUHY zs;umKslM-4#UWMpr)o92wP{vF&C_RGJ=tMfc*NGb(pn%Bf>XGL)8bTFqEA|0m`6xC-8YpS?IG?&sN7F$Qiu;We$KQQ1CrRNuSD`D7=V7 z1s*u>!E;_@DWOkK!MXYXrV(K7d(q+*id$#;?^>MMfnJmV|Pka|tq6bC9U#GWs zv+31l_A%*ph4-7U%kAX%t2@AW_K>&SL_Ue&^Pi@glYxMD7gM0ZcS&Q>ZS%&x>f`B| zrd0;zRqLb66CSAqBLHtsEB*1)|>*m{BH{a&Ev1aw4y1aj6CIwE& z#IQp!{i!-l-~_i>N)v#$6Q8V*j?zhvDYFkrD~m)x1g&rM*Sf0^-iCRB|5(od^{Cn% zh?vo^+s;C!W@h+IfBIeNM80C~0#H|z2cP+7Cj~N)D*uqjH z@AFpXT`)7sUuuUK6BHZ_=ym+@5qKGB)WP!Z+U&&;n$LRh^xNY%Em=CA{wGiJ>d@1l zNIO99Nc0_vq-5QuIU9`Stb1I~yI@GD1{=(x`#?h5-D<6r_N`L#0aLT$8tU1b)$LdH zETlaR{)&V5S7_%aTE3oA{T@y!bbHiM9bF^4#uREiUf<1U3w5fV?=5xwus_cxxBpa; zPGyX!RDtrh9l#9aO|Nu5IGI*-82Z}`X-M=MCtPVEXm4rAOB^t5n2(kE6#~AE!4Xu1 zUR=j{09Z~~FmP9WN02)awwZ_F7}8oPWTucv5>tw;$`g}`hMYQAb(@R#SjwI!&Akg| zvH9XX%V21})RPOc+(EE1tD$07rET=4)f7HC3Uk&z12M2 z1*Z!d*O$D*$S|18jQyb*N;$~ASWB$hWgum;WWhKM#C#|@77SyFv4MkbSjIu?xD^to zT!YtU@gD|_GSz@BZx;e$a9Fh5Ar>&VK8H{dwW7&%gJiW19TtgH348@URM4eESAIe; zM)O@7g;7TMboIC_EO_4^fb3|J_@nD%LP_G`_3y&upLap8V!S$q*S7?-+ab40^!rgI( zR4I5@Hk9~rwS@!4^|vFLsCyV+QrT>zqyB@}MJBkQ!|TKVRl{& zJ`n$=MvFd{j;0M*`~YNT;(f<)Ly9Om~OV83qH@(YYtRqfE1ad%F)Sg*V;JI z?8=~rad_6WOkRJts{Ur~He zaQw8M->)8YAFs<(u&mes0!-@8GHL6?St?J)W@{%29@^L>19^YW_Gjh&Nw8wz!8s9> zDRG0bit;i(u@^nhG~F`yF>{=_v)1y+@{>eF7ecbJkt3YSxchKq0)E%um&mCT7XmLL zZ!1@Uo9z-Q3US7{63*p$FM4=A>0c)KIkv~sxJN}QwyhvfAy5&cV59GlgL#ylbh}-v zj)RwHz)WgU-+l!B&LAbeD=8`9nxT@2KzN?#=kzz|hj_%eHsMJD4p_1(Mue7?CogSgt%JefMD78nF9ir4IEHqbc2IZotCBIb^fu|zf9)JC?%bBESaKjhvZHA4_BOt z4Kl)x)ng6_cogD*C96ZNOzVK<lip86Tf$kUkh9AJh^(R>DY#B>ZsT_q97>I4WHK7Uk1*zG+ogs93^QXL}izFrNd zlrFa~@c10ShFDSn)m?b`QS;&gWcDUt@XSvsWrQ3PTL&s8fin+cauAct$$imb0Mj4z zA6iD0Nf7s;>clg^BPJ%}nqo|L?nqMvEqWk&++Ss~Ue<%~jpc1h3i!g_*5*VZ z{BJBU_Lf-6mICSTD|PfDfb%>lm4ak!mGTGOb?nt1IP0fusackj1ewU*)Fvn~3|4c` zC(By{%G~6)EKs6mTxXI$tn3Vd4!eS5{UpM_%)b~I!XRDE5CeU+U^rhsZp<8En6Bnn zpJINQ-_PtEfjx9J$NB{G)1m|;tTZ;oSQ_RzaM`Rh%z#K?90jLEm2jH$N@XT9RvXl^ zVS2Y==)|R#Rm`T=CJ3QJS(+=h4rN`ApgWXhxuWqP*5yjmgIJU+fDd9(KX@|xAeQBe z`GZ)Nrx_f?s%*Ik>M6pSdVJF1Ov@O6h=ukBRzzBP!GVk5` zQemg^L)m*?*qPv9Z>{`Yj=m_#i>{g)>7D4sxd3mTypk3oAJ8Ty;&Du=Hor)6=n)Ka z!%>NK3lT7zs13Y`L$}>|++h(d%h{3W^xr5r0@N9_TJeqrTN^oIs z-4-d9*QygLG>MP=Lgs=3xq+>~VyAoT%mA6Jk3fT%)pkV`{0~2Uy7!CsM*$TA5 z?^bMmNQN~(E$smPY%ISyRzL$r1!I~O=QZQV)rZ$#U}hC{Wf>N(Cq|Exj!iGoey%Zc z*21gG=}hzQ?QFdU_XJ(~Q+V-61B&*$RDgkHT@L_6-_gybc~2|n5>XFaH}IicLQi`v zPY3E~mT$r9RW;}>2d}sIW+i4bB2zd(=}LUUZo;v^0DgR2EPh%kemPr@A3kpzcfT#U z4Z7G}N3k%4@%Z2B-*__J&cDpYv)`AUOyu@=rKY`uPncPBBCRw725=C+xw|9GZ!SZO z-GaIP*Y~S0)9KqcPP-6X9ejpY_i@o)t79_{SY`N?dS`0_6rtj7ABRebb+zom-KyA297%t?O7+^;>mD?Cqvdg9m9iNW-o0GmY~1QfGSE?h1To%N!a8$fUneRVO!7z zJiLYG7MtvEdnxp{|7xrt(xhbCgHdR-5|$2>|s#QNH(V{^G*VsYm4W)qLDZ2X#|$9+ApbP}luf~Pp_%l^`1#ZQ_5lQH0oTEW)hezZ zluA?JdjApq9Nd3N=Bwy}NkVkMB!@z>nL57SR z>DYoS4b@=-9gIyub;Kp-!F*Jg@uvk;2gRsWo-R+dS4jvK)m5{B@q9UlbpAejWG+r& zS2{*@z@_~~b!frSCaPn+%@aVqLiHEx*}op=D$7P?*vz+c6>$CcY~6d#j-Kr#bQgoB zNt_+$U zY(D|7Nt?{QXUNX{12!4ahS^~?{I4g^g}gWcJ1dDx!~AQYE#igqV7B@#<3bBq+d7Xv zgAx19qi4TQXAi)k3Af%DMdhIMl#_JMjXUdLati0h%rob~Y!V0{R&yG7*}Ag@2JANp zEN8!sA)USVx(Obkn#AK_-yA$mL2$$%=fP|Qm$9ZXexJVv%ezg@1dk2rH-f7SU)7&P zDd7*;EakA_OmUw_nhTZmWK)_%w(*d0cEEka{2irt^$SR?!8S zqR+Hb$r?bV=2(xut2VXyKxT5UI6T36rPe=Ax8sS@?+>7J!TCwL3ST$F{Ty+sDWebI zQFD~XG_F=r{s8Kle3VyBL3^*mqqdNw*VIiRI7fVHjP5cHHAnZC?PtuuhzFzBXbz8G zy~lVgDow+9*hdF(Q*;_cDd)j#9}S8k4~-gQ{MqZ)yK|sK^QPK!okn%4S@oX9pS^}) zc^bk?zQqKD*BI3p;o z2I`$Uon>4&W5$K=z4shG#|c?ie`(1HDPvJ{9*=e9ZR#(9g#3OY{Kwf6@m)c>_vn6- zUc*$^BXcJ9Dah7n0yxwh-LYL+o9K=_#Ir1Vh3@}Ys`{nX;>+yzm#=@jEm{Q^?ZlLY z3b@~YO&){vNp>o3r=y>wsPbYTzHGz~U{iC%e+jx>n~0BT(fvkztJoYCzP|)*Aqn5b z7s z(~MNfy6t98+ewuSU{7;Izi2=)p9Ok(VHb(>Q} zsHTj)Gu`H_b=@YA#oSNEzEK@G#;d!+cI`d3pJljgYLq ztt9oOwz`g()ELoa>}dhfhqMCW+`fZND>y|@VG_#M&3iw6xz}k0C$C!fzGGPVKrB?q zC?Kgcs&&W5-n06XSLt4OF-agnr_qSjl+_22sWGcRY2O7I2$1#DQWfxizWv&FW&R|? zC0RGzsxhlE;)B@mxM&NA57uz2%KRZ2tq^?bJ)U0z_9Q7bb&b}OOUbLVH%9BpS?gU2 z0$$U8GI$KKPT^n!WHmiLmFxuj8YSU-UDvxtsHS{9fJBY?`thswUGIk!YNGg#>*_JS zAHPa>O4h}9jZpiG@2!Vc1L+RE;_oV9CX_~Vc5(T~O-Fx7)0fpe8fcK(Mm^>y;<07w7?S(4UMp9BEM_Lc-LTi5ar zo&?VSc%r1h`S~AD4?qf_8?RduKpfj!62PN62PyFOjcV30S7*?xqlf?WO{F4q22$YS z>>#AT1vDS+mIM&T_Ll@e7i^UT2d#kqL?t&trS2U+gmAc+H^T9y$q}zr@Jc%flc6@UwJF0_~hl7{IIMictH-iD$C& z>TsmMtHLLG_Ua&fq8Vk2xUUW`PqBs(#heH8O#vCL8dCr!hzyoMx+U;Zg#(m7NPQiE z8ie_for(a&u*Ue`kwdio%B}+!0H?2C{qf=t#pm(s-W$RhC0=#;KO$IT^zVrMV+lm7 z>|dVg!RS9PPZau}sm}vI|I3WjIM`MPjT6)#L@?*Mz0rQ!Kcw@}x&JHW2mQlo@p;yJ z?w@gTXI<{!&cU4lq-w1EV?4Y~?k{U;y<-0V+4s7}^YlcYnh{>ao2E=%#+Bws4dJ~u zk@^5CcVEmm{Xc0L_94wyEoj)m&je+qo#T9DbZL&)Kre0LHB<$7#q+&l)kgK{M0;h4q89V=3Xdf@1Uv^_BZ;5a zMRP!;y~FacwYzQ~@9)2k7pv)b^#E)uPjff;J9&6``{r7aFHbVLUaeFwQYdbOE-r1y=kVbGsvliPo;=iAxaHzWG{TEBXvfqVV1UfrsDaQd?A&yMzB&&hNfn z&3<1NFxI<=bN0EQy33x5dkz0>z^K4CSF8Q$TI+Y;e%;Q_6v)iC{g08pv)Bv0CF;uz zqc=_OLIR{esIvhc@73mJPheypO@f^S&(U_Wc>ACK(bs=!_9y@K`~9NJ`5(W0d3kyM z;y+%zcyW1g#=c)1U!I*EpTAW9bMo@!^z8ED^y0;f;12n?lN{LtBr)Q~Q0vYu>&wEXI86%bIB;58+H&zmPF` z_J`s4^!!{5*DiPWo-OO>O7e6tMKS0kZw}RSS+T^kMqj4iHfscY~w>)owR0m-Yo2l-l*2O-aH08>QiY?uz5# z;0>{z4dUWCo_06N#&d+{-Q#jvNSz#@T154>>2J4J^V?cO)lsplOdmVPC&lBy%SF>K z7XrCbCoq@)ovhcBuUEh07m>=oEox=Oe5p9vqlY)SGxE^KzF@naEH`&5EJWr*Stb2# zcRIp0MWBbLP;!M4LLzr{EyDW%=MsR>aAo<1m#i8eC{(R0ps}HXfVLTdh+$J1pbq z)6`Yk)v9E9eVVyK7VF_BzJNp;w?ecAqI;Cj2!FMDF?m96=-pyw~SR?F3D7XCB@e zD+CD=QfWYcTaq*)CIAo_`@(aOf+D#q3{0gCo_ty@W;^E-RLF8itmu5rUhdP;ejN<6 z?oM2MmA~@|5v2j%Itvt?wvKs~p$~He3KJfALqi?dAIa5%_s4r{rIMo^QC3HPdu-1G z=r5DS)$L?E8G9mzjzfQ&F;nn>@uS*Uu#fQ^VFRL(kR3;J0*p3S#CeE8*j5)yh7vQk zf;KV!*3yzA+%dXpOB(e;Uh`7duT3)^pOObAwoCn%6slD-`t@o|%t zwPLpqMYt&8+ZQ^ypJuGCN$fXMiwgr=<>0GS(0VDwt&Zw!W0fXW$hRm} z*I?gGcv%Y@qQl390=>%0!8bC#!U;WWb6LHF(c5}C7sECql2H|t{5zvCD zroPhx@?cHLK+PpJLo?Q4Lkzu{?=amdg>81LYVR=dGcnk+2h#Zx9EUq$+qph0$^q%` zZn>cyJ-pX77{wm!wAchQ(WcDgvBOTQ_8D>CXMxmRXxw6_yl!^7RU+wz^!haF9REARS4_@_B5#+$dodz!mb-Gu@5_Pzz(tH?6;hX(tX&CCf=}YgzrMRIDTSB z&z{-p34B9}q)>Z?{x-wy1FG)ODbo*wVIMYSaLg24BMq6@?@9#v&;NX*Tv7Vz8+mEYkSARo8kcP5>guzJdC!<)}#+f zLzT@R$<^9UklX6ZGGmfvYSgx8-aUKgs;5RJuC?L5OH zI=%nePXQw}CrPyci6Hf1&KYUqYb_h9C>UUokPJ>^b*Zcd5%z5jxsE%I$8+jbdWd>& z(zbdSSbwNMsZgD&ThHm3h&Bm?k})!TjMPCr&~v{twI@xB&&)) za(62L*)Zm0g12o@Sk&;S$EqpW`?h=VCNBMyy1fN8P8g+F9ALv{j#q-6iU@>v#c8JnsQ!BC_f5RtbwG2;aq~HF3mb^21$# zs7jP1@J|8$R5;V$cC?6~&Mnh+s%FXdC#@|2

  • ZfJmH)!+EO&b^*>VaJq?OE)%E2 zq%3jWq#fMpkitGwB&WYUgbz=&?NMvjzwQAtHu12whKx-%B#WbD&iDyb}3**4}p5oo;W) z47roU((QQlveAu~uXfnM(w306$1z~YMpvh%Oyxcj~d~VCSJ=hotO-9 zr?++&Zr2@py0X;f*oNlnnSqyQ8eN71=$KYZlgRb+*PDf9eoxmQW{2*koc+$#vU0aQ zOz6%|QpJh>HllS8ws0?ucbh&;V#^J~G?_bLc0@}lTjcw#s`WineA;~5|46O=-GeP| zWW|PYUD~uRB--%~t-hF)qC`x8Tl6tjQGi`N8_A%%{MT##guwInVo2dt>N8rIL}YbB zb4}8Sa>Z{!NS!RANh(n%i?G6=w!KPPgJxK*`n^wHJlQ!{>;jFNxP-D6>@&Pz{(Kkf zY(nx`>kO+Dj?$LHBY^$ z{9K`uJNK-~WG*y!?o>UdzdcnsOfwEUprg>=W<<*Z;tr8Sz-Ej{Cq#b-CMV{?o0C;C z*pKx<)96ksa=wY!i%GHqK>H@bGRYxjiAnsXZMw?D+2}Ud@oMO-w|C>&@=GV}TlCOP zgef1w6{9CuB3LO#k>C2L;25g8Tb#(93~RaGlWF8Gld?(r+pdc{Q=S~mZLiN#mX=ow zXkUe&R6dL`&qqna@#gVnvmGl)%qPoj5lzyp{%*CNEEeidQew3Mk?{RyJDGk~RBnD; zi}wPzH@8&ZJHv5Rm2xpEe2HVua7=dJbakt`E?(cO_KDZOuI9_PZ*aM&s?piwTSoXF z!hfT9(kdPfGSWj{UoX|rcPll$@~2Xpw+dc(=ewI??(liNdDyPiY>H6CG>k^}J3xeq9D$D=7)#O_a`jo9pJ=TsPn5x>Hd#rkh9igWhZ(-w( z;N9@?^X#K~{}kR2pKw79?#TV|c>HhmZ;VGVkEfgM=YkWlVI$L3BeL7cEE2&03zEPf zuR?g+cdPZs+4djH`M(|&N>R9JtQhe-u2-}5RNaX$D7R?@7w`o?uNI46R$$2ZiOS=v z0#swgXW99tBl1x3Q%MraAvq*9B56_KF?0Hgbjt;1@b(QZcR9H)D2)A~p=Tgq(6tl1 zr~slX)3P3)a;{lj8jFl2(k_GvJf>|cL3oaEl#HQ* zTmgsT>IGFm%GK4-L73!bf9Sn1V!Xt{Yiq7 zyGb#a!QHg@(>GVl-Q;P!A@_y$e;JPloNc73l5Alw3X!`X((yToo8!B~HvDo^^4tL?jgDLqwxQwwH(cQ;vVW=B|C!3N;g z0SqJ<2lG4C?Oi?Z+fX=I1yUuVaF#Fbscj_=PX+20U!2ibQu$vxNl>3yD-t!G!dWxu@@f{9KSq1 zEM_WWWTB|_1YCaXO#(=2PZAx9btYLL&8?Xvu#~PQi3(SpIsRk$pd6yJ+dr?@ldpQN z$1q%1b46Xc&Rp3XVVJJwh~?&W=J?&x01?LMYJy4ZM5LVmnXJtW!IK%A;ol1(j+PH4 zK+cZ}ue9PSY^)JHDF|7gb{VAt1q*HwmC~8LPWXa%X?7ymE{T_-cysyC8C2rQOEGLmr+nnM*8O~x!U~u=mao~ zao2SvlZb)PaA5_BNW3d6*h0P=TC{tC*j9s8>F2JY8tpz(~Thk%J z$^{@pnUqa|u~IeX6DXHr?_k7<>V-hmT5~>u5Ey#|2de;ZB35HGiwzw{>js)LlTRdQ zmK9|%nlr+aO_~9hvefxqp*iCTO~Mn+As8VlMi7RS=>&lfhWXbGvQ1J4u98xEw_*<8IJVZ zaNZ#k2FLlZVO8d@_#T3P*vl-~X;!W(Mf*1M8$4X*G+ov#fmw=ZK9e$?_G@{#hYK@;dhFtg6I)3DGXEnM zR6V~JOo6f7l0Yh6naaJ`$T3E>?+!IoCTwq%=jK*BhA?D(a#M-_MM-mmC}4-RbNND2 ziNj9HjXU!p5pGkmmBZI0FtAmwxCKoDTwWxwFlZ9WRceZ7sq>h6)IIBD&?FSlAc!jq z+CXA?P7$$E1zMov>82#GgnrwQV>%hcJlF{Efh9;B>Z1=^-6>1BI#3TY_}R5${W z8>N8Ypi(>&_m92PZeJo1l%SFqA zBM7JpR#bY;kT1KO5@eN0eWx5L)GOxvMJX|H7AZ&rkHd#y19cTDcg
    q6!v^4mF@pvzcYc?FcNL_&w_eOF8W=FJu>u>*vR^P( zA~g&d?3m3~xvP*Y0o!DZNfZrSoV3@eHW0hNWPn%swZc<&X=~Oz)JDrSjpEBr79vs37aiA1WvTlRDX-VUMhQUG51KZ~$7>Zq(V-$W9mv zZ&d-u3SF(1GhD;)1>X(d5`j6_Fq)o^{_Quxdj@)UBJT+@433GKZ|CUDL0XEA9CM;~ zH*#J$fC;nMq_c!s*2wj@u+=#a`tfr60$Q1H-w<|IK2ZHw(x#B}a4HAMX~gHDcw-?q za>pu2j*1#&3~9#oN0#;aQN4Q-2J=tsAi{`R&!c^By@^tTx?WBVL3O`bx`#@m;p z_S`l_=I;Y+R|I;7tp8{SSXz#)_!ciH?zq^k#`t@nrt?Q^2rSvso>qc zJ#K?}5@Ru~2(B9zs%?c(!0ns|+yuv`3C$4dbSoUHAZ6wOkHd$d<+dtVFb)=u$&iVc zFwF{@Mts;F7x6AIAQTl6-Vt329{`hOP!4Pbw8vS~a>STY%~$t49`k-a%SHe`wT3YV4PE4ufv#n|#;Cl8Cs&rd!3 zGAV3=?{B_t)wyC$hr^M+%<6hL_tz?ncBYzl-0*k2x6YmW**#QY;W^j7v5m8_jjmw4 zDlg4Q4Wt&tCdj9Ro*v}i#*g5?x363L_D!Gfifrtog9dEhdOzGO?`~cx2ux}1x5F{*_41~AI z906MDU@)&bjO%t-G5&%GD+|tBm|+F`aZ9dXSniJ_BG(GrpdhT2f{rxOzHHj{fo)?q zZD=YEoWIk@%U~cC0r?dEL@c!R^tBC^v!Rp^R@eHEv9f3wo1f6c@H5tk<{U<2k078L zRI&kW8>AzhI4bbA11e!QQA;iYZTB`#UC8Es$h`u*CAPSlZ!3pIcVj!erIj%5{_q$T zU%$PZEH<;F3!o&%t{GVcDX|!0yM9z7_(A}Ot00^{Y)T+SrdW5;3+n>;U`~pR3v~u) zgc&3Mm^vOdB-nnoRy`^>OOONd*TKG5Y`*52C84s%tpyyRf6un?E-*132moBbTVyS3 zFgA8=Z1gq&fM~R%xMCKM@R>6+*$s{8F=@VGjQb>V0~~p%eF)S(!2b9yGv2pR=ptk` zZm11@V&iWg{M~J*!^93owh&omalmiKJA+|{3m!`nWSMz1C(Ix(Q03Pr)hpnf zx`_?6t{*?WU)?ULE4O-b+8lLJWcZ{ti=ZJeuDk~I&vXJnGT3oTLV=njJyeruW3x9^ z%?6bhUz(`mW{FTXa!fb}IfcZ|25_s_Q_Aqbrs$^>;UMx%_>x%wP~ zzt^j!61WI53b+h*-s%XgX@VWLBz(?Qu;)+^WtCJ36m@jLARo?8ISIjOVI<46SSrRsHHN4Y~o6iGBwS_g=Y*y2;gbI|}epQuGrYOoM26lv| z@ZXG9G+cEtbujZ_I#Eo7@4+=(6Q1ZakBWN$F^_7dbce6Xdbl<}1kWx7U%FfjStTC3 z^U`04F=B|gu~WI)l+b*kWKe9oEMEvxn9rJ?JM#y=O>_v?)t$(RCEG+|J>~1v|2}QA zBC_ZbpV}-mGc*A&YM@CX?`^6|I*2~B*(ib^Tryl=FO|@_TNP~9{(M}{x3)T$oU!K% zovYdJOLTDo>x82}@c_x|&BIo`PNyHDpNxL`toHvw`oB)NH$4yh+muR0-e~Ugc>nln zy4ijnt3Jvzb}k5=ga6xq-3R(I?=8w)4IHF_W2+E?IzTTk2lE-a34^;Bg0GK>JO;eK zy%7*EJUnv}MFV#+70~pR@Wyg%G#G27>Cuw^eQ~g(0@JN>Q;clup~x@fpvXbMQ-fxb zD@(K|3~bK~?@<|hswt5ZZGrt1$Yq9CmOu{beN0WnfW36e_Vnliz(&}7%9fd-s7CR- z!X!B2o-LqU7n@OBab>sbT3Xd4VEbwPXlxdr4VdK^G2#&i;}ib#j1K+(W&|kww`7BT zXD6+Rk9q}_{2WgwMQUlR6PQ`q_bS4!0F7UkaOs%1#P2s-I9h#Ve#rO2jLj|8_u}He zNBa=J#4(Zo@cWR_To+tN+oZU?0t{*HyYSx~D3XnP2{m=EhCpOv3hTF<<~WNpQc$J4*C0Z`t}V`OiP*16w_#=B}cVx z;_E_P&2SxG? zHZh?_;Nbt%q)k;l`rD{|_B&aNqodH@W~4!pEchz0Iah%+e`|9x#0vYZNH$kJax@R} zHoTuvwPo#*|v@;9R zfx^}M*ql+(cOk}Z8NpcB_p2|3Lh0!1@5(iXSNCzTx|uA-t*p+>1Hc@9Wix;xRLZ5H z!pxxp6G)pyef+|SRPLaMb)`a&)5bAMFH>+aG?d|3ei+Jt>UKSxa|4GB+$gwtZ)bOt z$Hn&O*w0zT0RW!zqSam(hr*PrW3RNNnw5^wz+8b9G>y!UF|fB)nLUfib~b*Pthe*+ zd>i9j;CV9+z^xS0ZrU?I8SSdy3XK$K7#ls9D*)Dm1DUGnToi1*@CKhU0( zKLjYnTsPk?C7J8S?$Lv4=jzDJ$*2s4Fe?SSEX;xmuneU>g{n;*uAGGy>M=O@>VnS5mRG@msYB+g^~c#(Do8PS(RMXT!1F-+Iq7=!FuyY@ z)aqWvf2y4uyx_dDLo2>Exe~@8Y`}(Uh8VK0Cv2e2S?^>T{Dt@~ShqaCmUL0C`;d3R z@{%GcDA11R5~xF-=+KS`#AT;B`&={WZSPY=BV`=k*E{AUG3%^Gtxr%2N>m#xK>1&# z7f-%kEoRG)+fSe%Qvq2WA7sfkH1JmaSR~NbDOSR}t-cJK6H`!Y*OlBXR@2W^lV@c! zhKDsS5P?MqCO2W!fAP;#|`hplsY2 zDB4MbDjkjUvhZwYS&Z$%|L4|jd4t-zV$Xv^6Z+_HliX*&bL}oV82xPqG*C^b%QBH_ z0zLCIYzN7Ka#4V3R5Jfzw_pwwXoupF-YS%<&&Id}#>pNQR7^cjst^`Kz}Xm=&|Hl@ zgu_+fSfYKjBd!j<9~r8HeY9))sK;C#eTdXRijzQ8hXcisQ5Wm(<>@ir1;%4SiVb+U z1`-$)o9$9nMXRhT<}lFWp>#Cd+n5^_3Cp8b7Kh|BF8yhf_XfmaYXZYC0pbXuGwpN6Hzux5ri{l zqr1}=tN}AF3>nrr=XZD-+be(odkNc-;Aw30JJ2^@L0TMp1P5dOi5Rd_ttHPJc{^A8 z{SH21!xI2xqk%eQ)#k9eDS7rkY;vo8)BqVauQ{&7Ttn=2B+-5YtA|fmJto?lwlb9F zt~UQZdIieMyYeATBF-QT7e;0<^GyexAH4!=1K|&!IHbEP@{k1SD<3fCiJd1K&JtWA zy_IzF=NlzA!6380=<3E-e=c%9zAehpH-+7duccu_FUQ)+kR?Ud)sLaBip~?3{;+eO zEuvzpZX7}%CGH_}OWfMTSVc*ViQ@vwU=puG`~;;<3NDU`2wusgy_y>LZkI`QobBL) zrS2Ka$%_xDKJ&Xm?!WTeWg7+;Qhd0y5~K~mG1psTa3QkjY#ft;;qL8Dlq|wY-PJQ* zDNmW*67yBrodE5XJ<7HvsyFzWG!8cC4Ifp(Om9rTLCwi7yQx)7@&VN?Er@ufcJV<$hC- z-mGrF9=!%#5$W5uL8Cw!Dy)_3{s1bhm8Y`kwVE5HHWkBtc@B$$z4*SadHEenUufol zcWXEmz!Vc}P~xjRzsZ$)2fhIQD;wg@Tj~PdTUVeAlNCMVlIW+uybpA7-NL{Dg|{TT zRYU2$0hPC3k%pp(%THcTXwl{c;bPU%$L8Z*7jxmH7o@0|U;;j(viX%tB`~6wWqu zT2^&H-xb_s#|9M4gphFwZh@7gu!fR2|GJ*3UR7%dyOH12dNNk~@k`i7wlU1ajQN;f zgO8d$f`gR_1InAA0?6=*>nJ1DxKtPNfU-iuK-k(WLLRTNprDTLLPJ4VMbLgfehIEU z;Sokem~xIZ-jdIS1=>wNa(o$$&{fF`_y|jJaIF z5b<>pYxdc@)7Z%iNE7dG`z~UbvPGw30H8lbu544cGxJB|0>kEvmxRvZ=q41h&o#QI z053+h;cyPHXhm(EG_*lh=(a;(Tmn^6MD`R3fOl4ixJ1qop>Hq$&%0nS0j5IczZka7qL<( zBV;_ON8-aE2NKzS7Ge?>S(9n`^c+@1YUh<4zXbEF)x?rxtXCpoc$Bs-*cp~sUl?J( zl@FLr8!}6Fky!%#p@!L@X^?R_^cWBVd7cTW0bMty%Vedg6Ck@>$n~l0BLB5Qvbobk zmf^z4EeDtjtJ3YlPasl*nlj0y7~)a(*Q)2(E% z*~=ujicN5Ft5x2SiTwf~<-I2PPr_DbDZ;@i^2vC5unV_DB+Bx(n0O?mw<2D8j4cCe zMmGs11QtR?N21j2uAB= zMtOVU+xEbvMk1FZ@|$A0jbdWtolyzf*H{$uSWqExG_r9e&&l#mp5^QuVH57_*y>y* z6W(DwL+nn%EdowTEK)v3{|9?E6NX?#vS&OP)%lvL$j{)xTYh};WwQ9wH$zD|qx9$O!3*RV zVbRVpszq4^Ev1eU%9Ii`RQ5WyHz-l7XL$_qmzuK-g?k3ZZ;eYqS?nTqa{@lDmN`F& z$bs_@oc~SOWF|JxGm71W*)a0cd^U{EfLi`-G-!oaR?O1Zem(QcwJ8%d-Bdju5T}9n z1VjGDj3w@qpJ!W@dda|{Z1Ccm01tg;?|SxSw%*Lf+ttHrvHCclsspY!sxY<42_5Q6 z)hR<7bffEESMwsV)fST;L|rK@7W9AnrC1;OQa1;0{2>a#*L;9-o9OD}dt@eZkFV zRsyDz>}lqvBu;NF&`LSSZnS&li%Qnm!CeI#1mE^u#c{P7GluhOO>12ZKhsSY@TDuv zNC}|f1WYl_4iHou&*qIONBkvv7i)~17>~e_gGDx2<9<7P*nS${&DR^ytC-K>DdLTt z*4lt4*7A^sIde)tFBIlA(3V5VC)MmY1@{s$1YWcQVoqSkhP6~tzgMQ>U9cMQYPO!L zp5O$Mr9@%Dzw$DZ5f(|ZTty9V0R&o0hSO?&J6n%c>x#irGHS9BV1uRfr*EzVOUWx4 zd_Ui8uL`KqLx-=u+fay1?+R?2Zs)5dyPvr`abmK7hceFl*wrh6GlR*GnpjIJo{N(m zEN07(+fRqz3xxY8z-e0pFR9z+)5BKz3mC7{p|gkeN_F+Y|1l>+hEB!|H@VDhBB`}z zdMqXVZ6|U!pDon%GK~c~6#Z?6HTbMrXBLashgNX>*C&2<(->v&_-wWc?jWa;smMZ* zA*|nLtmJFx%W>0!fxjDP3XXLs1tZld;Mg69L62j1tfj~)upssfj@BF13cR{9Rt>7v zpjrX{DV@!w;zVP^%cJ8{z@tI668oibuT*OBmy?p z!p4ahigOMM1Y`g#4VEQW(Fm!Rzt_iJ65x=vbbvX_HKc-3r5k6qcgVwH27_6{-Z3%R z_+XsOL^_3XwR;uFKRQ0+Zft>Ce$dY<60j$v*XEnp`7URuAZNg#!bMkfeGT*aFqgJd zhsn>Y#p0LM=JEdRo8qSml9-pOD>RbyWm06Ve1G$GtBj6s)u%fDdSt%!Wu0+=m|%$* zAgPgAyD-f=ZumRiTTjsa>>i6?Vevi#z}Z+65XP%cf>=z{f-CfVu&f=yIp zPJ&`Hh05Sf9n@-=-6B<{$`_4IV~MKea-Wgwm&7=T$OFCGM z+>1x_TS6wDJhw6{pD2+6p%;BmO3Rr6fFmJOPs4p(iDl)!jScQ9>L`@-Rdy#au*5Qr z_p7ODVW=8bKh6{pCv|wQJ4wNJW9$ML66!$2Nh%9!v$c~1slyh@p&P0v`RihJQ?B*g zcemh}WXH>`Mf*sjYQuv~ zRRZ;vSx=JW&{PM_@|vYiAsz`d6QFd>uPpWYJVfN}QEzLaER4L)qb2Wx;kR5HNReWx zXZL6oBPVk22O%neXjPbE(xv~3(R01+$r?Fp&cU3353}u+(m8LCS1^y+!LbY{9RN@{ zX5b4A4k%naUH)&kSMyuyL6F?4DiqkTM_z}S1c`VB^ovB?kus06wNeSix}Mz&S?er5 z2P2)4z9369&!Y;HIZjST0BVTYi1J`?uExk&a}El_zs#qqyWeW{`SD@A zde}nJ2tDS2a(>UrdH@M>ok+y6Gd~cf9Aqk_sCDS4OOUBZgR*(cIIeHEy8-!jn5kh7 zrQRV=&J~V;(?EN+}0OAfkHsJcM zVSVYXW_Y;UVmHJi=CO&BKnegP&BotD%>Uk*nwz*u5hzU=OdKZ{n4XNiB!2D$78PxM#tLI=V)et>Tubv|w z?N&YaFy6bsNFYvaw>rFc;BA=;$mAGbV;oN~7cM{t5*~@eFlH`9e8%1u zdKVa!v&@CmYTX(-eTPFwp`YHu(ZJs5Yx|yrpe+QqS)XSPZTL?i66FIPhup8}Ef+ER9A%GeW0~x6T zb0!lnbuUcRG*eNXBh-4-r#zKZ!CSu03>;ZwP}fU_D5e7cV>$oVqaw-B{yLQt6%fG} zyBmyPp}{rJWn_hk8!#hJ#qyRBNt!6N3>O;vz=hLd7=lo6>w3|P?m}rXmzJl*dKVa3 zWNEcR<*dm5Ybb~v=6(iQ2-`&+=4S9{XkZvX_^N|mB4WwTN_?tivt zDq%6HHMqb!>EE09Wx@UJX!8tFfs>FHROrE96a$)8U==3F0m5+tRVpPUJF#TtJ)?+ zq`h{<2WbG*4w+boXwwjNLeH~NOP^@~5UFJvsO5i$#v_rau@Iz3-BXqZ49ST|mG8!@ zasb$%Kgh^fLGk{puHVz-8{&9}XfM&Lti zbP#f_2)Qm|qdO9EL-=r@fmUdsi|}Ep&#yGQjTIFModjJTvbxu2OKymb4g}r`fp-xb z-I0(3zmQDYgC#dua>+?>uK z9fbCqRBqL&bDQn$>Tz2h38vj3HazbST12r_ERRixMd$Hw1Uf_fJzmf5j*ef$i0n)u z<9g8(J)}7N-NRaeQbAdLeSCIDM}U)~Miky4gidc*&$pkxovSnM6=o?H9(*EiE<;~R z!m7M5H&}{^Edo`N$eerbfw2c8Rkx|lotdGz4>52gUrL_iju=79knEa01e0kXVPm{q zJsUFSI?FM*KAw_{QHR#VO>*y=Q*ml{n!Tgqblud7|3f&LaFC~Oo{|s;d8*Z3;0Knn zT(RDfJdG&Uke4(l)Z5j5KI_}}W^7+d|1rvTBA}Hd{PCyZK%7*)pBBIpr8~!wzet!Gr z`ffg3D7S&HHeD&C1VG?_{XCl#-Uaem1F8J!qJOI|=SL&Uu2~c_a_(aOhj?SV!e&Iw)W`XXw&_wWVSAyZy8}PfqCtMK5 z08Xy&2FwL60~e6X|NUMKBHzG=gBvyPqay14YN~wWs-1pOBXCx~@U8?1zW2NJ>VETh zuh7!|iZKo=A%jwbFA27r_MM^YdVJWf)q?!EJeo2HvU1YWF5;c~EmA42!m<=kgCy|r zQCyFgpy^#;GAopl{TdVkd0|5!f7gNhhIR{!hwb)!PsAfd5cjrj!J7XmI$vtIKo6P| z150EKHAW69cbw(a7&)k>OCtvz^Zz?puP0xxKF&ZTVy=$^_W*!M@>SZF4uPEC&X;$q z-~o_^@$PeYE|Dlwl%r-^A&Q=A#T8~@k`QEBl`9m@XmLSZihD+TR_{n;0`Q{JU!lxU zrRBgUF38pP+)!;O#*=jhv8uS16W6p3)cHeY$TTJ(UqdtCJ75R#|^S+tJ%8P{mhQUWVzM!>o zJ>>QEQVo5#QqwDcJ}&251uwkwUFku+-Y9>cnhmH6`qK-CU%ewi5PvcoHkr^M2oE!5 zylbRi569!-A$ep%vt<#Kp)>P;`*ZzJIaep5GYZX?o5{hu(pHXX?Rv99V@dE~i}#n> z-L@z|1N6-XS$n=f-O)AT@yOzRTi@nVO0D{y|?i-{!jcHrLHM^q?brg)OiEHmAMQwT)gsyozgu zCS3gmZzqk$YCXPWsRLbFH&L{NmFtaDc(ye%BxwU|*OxLa8iGQKL}X=7CqGGk6G z@=)N{UT_d_4Z9H=3`G7FevLt zzc$sIk`kn74^F^%qdS=fM;Wvn2%WeY1OlC)<^XELc)EIAZfEOaKiI&0bNS}rknDDn zT_h`YuKNlJUth(cRI-kW^%W|CDgse4zn`u(A2WQ*eB1vR={t)&AAHoZwAB|?{tzyN zaCZ?W?ZVDYo$d^lGRy=B<94Uq?jeS|y99zQrBc?ki8@EZx;v{ z|3+9vL=|v$7Bex(hL7t{zig}va&ump5#O5)`m z4P&xO$u1;m1#c<|0VRw}s3Ffk5dyB7kw{`e<%Lls;q?MT+^r`8=|JWHe9~Q0j3xgR zT5QKHXlPNnv8Iz}K!@Baf^Im1{mtyl4~4_y!(u*FYQ+c{li}Kp>9E7%jznG!6pt#W zFLriIhzrZ7uetp%QiB!|EUNXC>zDI za8b>h-i@VZ(8@^^dCKX>5Myk5z{2}f3>X^=F_bRB@%Bx_5ptf7>l*9aBeosn_i}=@kenLHF)Llj% z8k}+g<6qc8tDo-xL2K9=Sr&m^ONy-#JFLdgHl4IQ<;095LI+Qc73Z9C%u* zHAW8DaKzl8){A|YJz(mcgymnWEMJle~elSG58hNmHGdN zaJq|QYmsNfZW+N?;BRqSa8hUS{O)eD*vyn&0J%nN1i^EocY=k&ErwosZ&g=X67vQ zqf}!o$VFWW#)MTm^U8#X}h&>c5^E)k6d?wkJP)X$M0OAJeNO(x7 z@fp^wLvaK(lNwA0P}z(p5}q9UOdB6*uqptauqZ)8P`*c~23Hz_4{~oaT?}qV$qcEG zrSz9;B83FiHvPRZ+;XQU>%9c0kRPP^3xaxZ3L}IU$(v1`)B>Kulk!4#yVr?t^Kn+x z(qLKLFxvOCO;L)2UzENbuRhY!Q|L|smCNx6dMa1wG43KHk77mi`&>a87CN&V?}!u7i*?pR+a!yi&_LPcLdAz zgrYqo6tRD!7b~@1m~z=rr2f%U#UfecR;PBXwQg$L@o zf~e|5q%euc+q@MzSm`NYH`=|5X#MUg*dQ^DUG=`zb=Tjm)+)`%oPRpB*)R~;fwEB| zz&=$=sYVDZb$}^#xN7?j9Euh_1@$ZHuv3rxgDi92rxVt%K zDjn#vV30+Y7-0tPY=yOPLJw`PW4>1aMBZO~aauPMN$(&};)HcABl|7J}#7wwU`y=*r1FL$~fntTp15Hez*J;qd|!3%wfv@7@5}YTwxC@`*(A7 z20QT>n97In6GJP#ymWnKneFy`3(U2&hy#N8Fm-(F3^#N{NbQJ#nBmBDqv2MmVnhu4 zrAmQfBRE~+mlYf^E_mfFenO}w;-jap=*~G03PiUK#RSF7y8L%wgyOp@w7Nt!H&y{q zBRUUNgc4)!$TAzMb7o9v7+@^%Ss}jlktH zq@WW(yXLwEbcr8qEKV$ZWPFm{j^;i|!*Son?sl!BG^o7$R}nQ<9~2%%p2e*`@%99R z+U!+)<^YpGnXdy`(UB7>^A%>nUW`*Bn^8g?E$_f)?8Ujenk3Xj@13*XWh!CtAGJmB z!T36V9l+Q!v=lE_;=paKqvbrKtfdBRcn(87W=^=ayOFcDhe4wZDzfES&L;SeB+iCo z21Cq$nNPP@N<+B8Lzfuapm9_$AaMZY6tFj)#g`SEf1GY_XVV2W7g>P062RVRh<-w2 z^TS&@EOJc5PvCgS_MSR2P2|5T;1~(I@*nUhrWA?dUuDqT>Pos(;6dy?Oa=$@Rf{pO z=s~Cq!|?#Tvb58ZJH~7Vw*MxJIroC?FIKD1j}Ol-t!QBT-wbS@cz{y;Ii5Z4T`<9N zGg_SImNJHMeXHQbd<5o=@TT6&E=WIM5-%#dTse#$e-@q8?zD=7VoPuXBgB`od9Xih zyg-l4HX&e|7ZMsuN=_jH%yew?e8@ESGz2!AD2RF>-n$c(gH<Q=WNmn#S)_h2EIeYi#KaqV!%GTA`j|l@E1L@ZV0wk`vp_ zQwR2L+dB&JNXX~Z*j^HWCKLR! zD^G2$t*I+m&v^z0y&cxKsrRgRR4j*w5iwA*VIPs&%3uUfrggR}q^w)M+sq+SH;N6# zI9B4Lq1Ty#BZ~`qZRF{O-X`CTSKWSM!Wdz_0Beg87i+2pmA#|px9vBM(qnd4n1ZA$ z!<<7`XLaQy3kT7?@yG%Tk#HF298Bqb-d&@>^H)QzQE?v*{|eM_-Lt&hMDzs#V5Gp) z266YyVfZjvPws~z^^x|^Xs&ntt5TqqB)MLy@4J=C5Ga2>F6UcS)Q@+PQxbF&A6*{2 zd*2S8xK7TpB@==YWHvMEfeOv_>Cb-)^DegzUAqiD+r}w8Fa_?p4$v464tf}k$1woo z!PB@_CP_hGcdLqD=rTl-3watLxr7@qhv{j={)>aBG1=3IjRLsUVItvGYV}a}{_lT} zKQ2}`lf`(t*?un61ZEakYLnG4Mb}l}WhS~dT{=p!W3$#}u=b76aVcVvXBjhDBd0nt za8QZ)X`>lz1-m1{gw3;{2g}BK@^+CiN)Ht)pNL z)Jqu`5p$E3722bTa*yTF3E~lp`V$^eDwoWnDlanC5ND__%9F(=yn<~rAjtg}C<=BryHn%Ca*q}?x* z!sYV)&DX6eQv6nZo++EreCx{<)!=#ybTYVk#|?i+xilWZ_UQ^|V_J92n;SSYl?yo~^fXlFqU z5sA)_6s)U3d>s83z6DjQg}b(e;b5g9K5n?a-moD)u9`6Pv82Xx-MH~_?*`5ygK_R5 zzK!9?>#ojjIu{OHImr0-IhY&F`N#wWu_Xk5SSNlfK}=B@95c)!_LI8`;cci$m~)w^ zF{vtP5HlJYKAy=#o30n~Z-lMi1>+@re1+aOZGcq1f*T)|W(C?z$-0%S&6b9 z%1)-iQAWS3j}Z#vF{LPu(W!Gtd$H>MF1)QqyN$RILY5vadyqrB*`QHOpb}A?@bFVm?)A z2Ju)b=*2PVPs6h#Odi!+9gU^Ro0shGqmfeTMp9`vh(|2##u8mhDmFgP7w{zJH=A|L zDI^-qN6w?*l*(yf%82A|sf zW@G&5k#xBz`i~47edhVgd*XE{c7I&hZ9@XX;BxSJ3{POlWg=+K>$>~!!*`F%>2|(a zvdl^SRBi&Lzl2D3!r*e~*yV7+o!=nL4$|dtfx)}M<&f%fK+JIBc_wLAHsul`dV|Yh za5)spOxOnVJPh6iqw-d}9Kgzz@pN!GfLsL=|Ch;Jh2uyYBpVjRhREb5?FK-@h582$ z+U0;zMCz1a#(U}PU=e%^#lhtO0>{E5;&3#yjWCY696rvrklIG2I<>34WY|6yI)mcE z;TO}RDRDBYy)N1VVj>`QtqtymPO|z?`t&Esp>Q{x%x?#0!{BW2IZVXa@VDEm`7QMk zn4>bH4uaJeRh97eiaW%~pjRv>F`GX}B2L`mva(2ApxR z0032RvO~?Jv|TVH#4H#r+PAh&C)40B#CO5EWxL(v^V>)kd!DPv-8&e-l94AmI2Z;8 z1298F%+!5A4X+}n0r04PPD6Lu0a+slO?6NV$J^By*M0c*I{3{jGXkx^*QR;gOR(3C z9BmxeNA&Wu;0d2~E!0Fln}{Fqq8&&xcNI)HP-!OfY<|0zdpRX|iQuR)a_Vt1HHPkP z((O3nb<5`sbuxK39SgpBcO zeT%2V>C)^w-F%7-D@@k<;NeeL|d z>FQRcs6vkgT{MJj*T-{qIIHmA7|zH4F!(amA$fI3=-=|Pd)_OrkKYf?yUoT0q$D8I z4NV_Q67}=l%@^Fq=S63@Qe^9o1|MC=KDtY;vI4_8$RwakP4hDC3v)A)JD;6MnZDnQl(vva;;IppWM-^Xc|Vxp<0sAHJQ&rDa_Q56x4_RlLX{ zkCnp+L0R&sL7P2xG4;<-L>iP?4Wn*-iqYTil^KQkOVl7!2HkeH*&dx-qM<`GoXoz# zqxui$*_+kvSM@9;cMm3_DwQ)ZJ4_Q%+g@1`mUewJNVK{ytn-KUO4W`bvR>cKXAAY0 zpEKt1mY-*n+kdM1Gc|;YW|Y6}5W%4;%}hEJ{cVP|JB!n<>^#R!)hux3vS=JUWE2O| z;Pw3QS^v{Bn0Zc4_v=Z{>FJxTKeEmS+%fYv+Ylf8??Bl>CzS7_N<15sd9v_@0B1P! zb)*l0wOHP~J?ZL;X}?tJof!XbM`!dDU+;oh3~0iko&lGjSjmEPh_3^!L)dUP3H;E! zRzPtq63}kga2S)@E<*ox7Zp+Zu>2VIhgPK$;_HY0Tw8FTr5!CGX|%w|+Jcg*jkU=bACEVWc)_tC%u z&Lne-!N3~~ywn8hOTeR+4LqN?5|3hx)FuQCN0|-;i6=7!8Z2O%md6urepmk4ckl`C zG~&MQ#DytVaG)+@#?RNIO%HWp%Ib9yT|(4y_nqxq3iiQR;JW#7wSD(5g+=^Pj*KQYYR=|&0lHu}kGu~-6ZhEL(BoEl0(X~VFvio z8Q}YLq#PF)b}yQ3w`TmZYBNU}43*WlYGgEYp*n2AlL2E| zGSU%hnsqwzgqw)(AQRjQ9}$nIOz&r_>3p~Ul<|DI`S^n_DzQp3r-x9w$mKF|vB1a= zbSac*fKJ+o^93<=eR(7=!Pepp+~{Tm$#%dz639p`llmu_M}j!=xfyNmiX?&D0Z&Ls zQ{1?O1_;6nD6Sp1+9cg3cYojPNhdC!?$fIFXtI6s&fPe$z_1K)mfQ2Ip=_94 ze#&8vDwGO*0WEHm7N*nbzs0}lhgB0BJ~+&QWWWnP%3kowa&A_ZKwh#&j{J7MAOfAR zhB8K4gEa)$TJN6jcTcjGZYdJ@ewo~V5g8iaynZ8oj78*J{nURLiK@Ho%8|ZRy;Ij* zi7Z#yPE}eiZ(w+x{A()A$Jv+RM}=v=JS<-g?}r%`t{$c_Y!?NpQ_j10MSgke)o}yE zNJ8Np6R$rCA1Nz0e+dv}qa&r|<}aaTT`V_0mvZx}oJfU;yY(olr#RstsbxP*ll5of z;XFc-wiA55Yt@}vvIg^qtXR!G)zO)imG-7;qdgngbueBPW&3`TkiGLRy$se@QqBSe zrySF4KvbsoRT{auJ%B}h#8E(007w6KzEFA&%*B{c-7rn2sQTyQds=RM5C>=3Y6mLW z@LIpEH_+bGv={Go|9*D$3ib^1yx|ko6MF%uL8nGjKh05>Vob2p-{!9sqVlM*j7UyC z0Imy}gbak_6D5DuUf>d&QEf?{wEXRT`Znpl%XAfjma=H{$}k&J{0EWMX0x2^mbtxA|R2bou`o8XBq5)K7ENjF2J1 z`8J;_j+y_zXiS5U3vW;n&5}CONXUYLV-pWIxbkkboQr-+MhgbY0QLVM&{o5<5Jbo} z)&h67G9d?!3y9%K)&L}4AW1evPyr#!^)Tt>A>GQ7%U3Q|Q8<+|y)v2aj_hYP#?x%3 zXZsXT(#UUduG+yUJ&xK1X|cbN{)68)2{S2(FWJ+>ZO8=xvYAmn0t~2NP`4tj;bvng z9XdjYfW!5xq>=}2D~?&PuYpTa*0RFUlbtV!R4boQzQMefRz;OHM5Kcd_?sbk>9~RwXDI5B=@Das|9f)dmc%-Y?!8b0GE=x7>+-8~Eq1vZq(;K(> zWdY9>Awv5KSPVlaIZ=z=m12S66SpR3hR6VHt_6aFDV8b6V?HuLgcg&5^au}X@B*!t z`-oDX&&%C%pBW|5xjJ~~DmvHPPsjSadxuq($WTgLegZ_HZco8$m#aJBBfJ-NBAx(%pm=M@N8&EDXD$RmC-ySV;z&xfDY z_v)vu0;=z|RzP~Jjz|+HsiKr=1YQp8_CQ8qEUJeegvE+kR9qWmVAh6cuq!#m#5xHN z5(y(aVj2CD$`-pvaK6yY1c=29LzIz> zmv>O&XzNW|4nq*bvFue#R#~l~9MfFHmQ>&xBUhTUH#z9da{oD*-(@GA&DM+QE&;`t zWF#EJ2 zb673H+=!v=?K7hakzPSY4!H+MGt?#fr2<_|CB*J^Q;obx!NLXrq;=nHUZ3zGiz4VyjEP@oXPht88>-Qam$V&m3O)KsWKXD}LU8GK zcNh3!Z0sUY+jH{ym+Q_EpJG3oylB1ShK+T#M?VI5@`V?;#MI=xJ8TKjc=z_`9e_;%%yxn`HPbmszpA=dK%F&SiXj# z>Fek1ho-Ne$IvvXO74G-35aVq>OozHl}i2%0 zANC)IU-ANmj}}$kHbQhl6eBb0s?gflAO>TnI3Q{JYXeEL4kPNsn~nOE{T7shfCC`` zg{xn9y{*3j&8ii}r?hGO;6qD_U9O+Q>J!(Jf=pSl%k^`}g^lsp!N+4Urdc^zO2uQF z70Eb7EK-*Ifg%Nj2_DVp9?WhAwf2HW;}(-L}%4%XC{2tr@;zOQSRh zry6#x8NG5?ZxPdWQ}voRzUcD&v_OlG7}oq^Qi z+b~FmL_|zCyzPKN*Grg64RTh@y|IzmB`E^yDtEq|5Gl%KwGy(i5aar}%UaNeG<;-s z6sTP)%~y)Igl&75i3Yc6@4|HQc#V^BZ95Ec()JP=Q5YDo>5}nF!}t{V&4xk@X`~bE zp4eUoE>jg(kCAj>ZxUI)kQJ^_>L9%u_=sZ84IJgHN*%7BgD;aoVgcdh9_B-rI+R&b zj&I-jLP<8{1*WN$vlD_w?ex8>WL&`bJVy#7tw7ZZ9%MW%e+_E1PNzPhQ^dwi zF~xRKuB7Orr@(j=*@RN553!$=00%fFswO#@}pzWk3Qx_ppF z(Z?c#66HGUXA(z(mE~6u>dM(2A!PmLfn2nP>#e(yUADp|lHr`aUaV;PiUX$|qdqi4 zO!eV1V(i zzJ#Na)SIgkRe~rA#^Z9H&8w(~@GIn^=d_ZD%j$E;nikk-_8Rl1h|f7vl4<6jWRW^R zrjEiZ~h*)Gn)=NGNHBXfWumFbK%Tn(_~nF!#zrJ#buj^N!Mp zb9U@R3Cf(6NT@1n&k%^k8tZdZQtD}}uIWhlfa!RX8Y@1I>^I)h;LTy9nHR5g5Xw}V zne?2Gm_j)C6hbqn%l+N4yXoo89?!Ac(4{P^57F1t-p?fW&|R@U=EbS*DJBcs3i}>n z+uA7yIvVnbVYPI*zJv^Fl4Ii2Z|$^nAx1k%6e6PK)LOboWW<&(v89W|X{oud)Xfs- zREZtvn4!)Wgl@fIF3{k4H%1I|9XiYf_HlimKT)sIoT6^R-1mTvON4AIVJ^T?{V>-! z&}u{KlAqh(g_dfxVJ4D73v1U{sS}Ml>w3YuM**7IN*?g*~Ck-dRs*~}_C@Gg1W?7s}ZNCJF4njwNM zQ(ww7k1TgbiGfn&TqrScIZUBy=SY9uaWK<-cA#clNuXEaFC{v+5=A(sm%p6tNt%$z zP=m3UYC_f0+-`J*4$D3QG1hw)Wit|>TBUxWO44SrMGZbuE!_kkV82ylkC_cdHO$B! z*ZYglS+P0f7HGYzb~W2s@kf7d4nxE^qE(&0vJcHum^v zn6PG62P0CC;B)d(0LT&oGJsw@;A8S=G)s}^`B5jP9D9UM3iebY*m<`$!D;6W$be1p z6A3hTI7-A@@l71Zzw?E%TVO8UNC0T>{7hpc;LwqPhCUifM9|gxV37d8QG-Z;mj@n} zaFK-s(x_LOaC7!L4rD5lakLZ(#G7iMlWoz*TtccI*YzGK?$$U!5RLxjZZDL|T(jn- zK_E>8(lI=(K^CAl|FKRQVAk(zpKg+^IOQT!O=2FPQ0z2;{hlt@LQ9;i#m|RS{2~AK zX}#Qw^+I;{_8xK@vd_s5bi<=%is9pp7_l)SQMn~1=;r>J-#C0WC=XBsH6@75liI9Y zz^b0kGRJB>;7#tW#_OXz_imBZ?@k`l?JP4|N`6CVKyKlqox7=KHWV%a#pyf^(Pp`@+QU-*De_Jyd$E0cDeb8?#yxkO;^a^ zl@+Rh;{;UKlmh^?%E-Ahi&&d5@KFhRV{K1ZiCOf@uo@UC*B-30bsBYutyUHIqiI@2^ zOjz{-lY}CN%+>a&!B^G7EhSzll@V9bGJJ=@3@{lhx2Csn<=CCb)u6Ji8Pp?j)t&f|$+@52)l|(7J#Oc%p1pYQ8B*L{$FjTnZ1bV&Ne;-=(GoYPjFJ z(8#b%i;y6nVGXzef71_u;tUv0bT!C`DmpKM?L!?zJe;c`JHQ?1XE!Mhw6;UjA_1ZW zwyfgaYmOi! z?_$4wN}gR`yM7rQ$k>Ux8xOLEPN>+=!qY+A`1J?O|7^bT^3d?>D_@&yW7}=-TEITE znSl*6IkxA>T5$qqLU$&0$sRRVkb%J5Vytmp(gL%G49ymy!jp{0)!5Q=;*LR6srkzFO5=RHlp&`sSyokLc2w>;{mxRkugO}1^6A1)JOl;9!Y zWQ4Gd*W46bH{>2?+sn9V*L8HKrJOS7u9NlW*s0F|&@KiD>NjgDq?nM*TF6}q_V$Ik z5=yAnsZ?;jP&0!4^OY`K$I=BW`r@4RcfhkcV9(uvG>t95eBP~=^DH$F6Wirh6Fzd- zCPs*(`lSo5Ylb!Q+M=$q_Vo@iIbQ@(*BD-lmpidA2;7zjq%IHxy`(P0(SfL|l2oij zx0ObA72*PpfCVfqiPnJ^3*mt)_31#w*0Wsj2JT8M%VaX$4EG(A6dj_a1>zAN1Rnx3 zodlSW>DzBXR8_1NYOs!NCk$q@iy4C=ebCA7AUY!RXOfyDH#ueuiZ&j!<|k$h3La)! zs1aA2O$)VT4RPg=JIN(e*=Z$LIe(^BTg(`gJ3VTn)J*M(C-dP)A;|7BHf+`qx@?{1U2!(oIylJ%o&V_6-eZuGsAOIl&EdlC%gT#Yi!O=dXXs4q3?84aqoS) z_#*Cw;E*gk72kr{ZEBAwzhe28rBk1eF@_pKa4K@Mh7ApsY!1MvB8Qa*?qFNPu0^wo zmIrdrk7a~Kq$AUpq`ux+Banu;a`1mzNh z9NBowN~_jcB{W*d`01eht|tY8M&B7!hfoIt8v)h(LF7jLSl?}N;3U(KhMl;7R;i}A;$)G@^3z2Y4E@>ljTaO5jsE6FA&w4|z0sVy4wM^p1NK~`E}>+D zUPex1!4#C38fhjCYtp5TZKk{d502+hf}2n^v@(;y7w2Jj%` zX*rYO%@7z?CA{IxTd#QQrZ&I0^63VmZe%Nn`Qfd6Qa0XAK&hL(xPewem91=$ga^D> zZEfAIeY*A<7nH0aSzXSA^VJYFcqw0IA!Drzzxq6VT4%8?w`2w9UO*^;ds)_4!!huj zFTyN7cmtl_M2qjx79Zf+&Dx=k*ORPlSrSTM3|Zn0QcmD*EN zuw4nKEVOtxS;R4n=uR_4z@*mhXRtpRdz~2Ges6s9t|7ian_+3$?KT3P&w&7e2f>FJ z_qa_%ss_d-)ogzcp+r2)nR-5OC8?_+k|L7)0g|;SDK>0mb;_hzC&<+=QZ(3AO;=#$ zHR9d;9WcjZq@dhFn3!-T%DQ|_%vB1?t-GE$S|z6G4pN|G7#Ohqvw^!3_&9h_?v@fc zM?#;6&%oz+MI(yF1;^0zV;JM67sPnk@j=4io&Yvct9I%@>epTyzx!tm>~eJuDJ)Pj&5G5R#K z4*qxhttix+eomL7#^nW!L++Q?0Zx6yN@_Rx^b6ESsSLRgW&7uPqr%n;Bk)LoP?3gt2Az)5z8KmR;5d)^|q|nQtRjEDC`q+B^v{@y3mQ~SJZ8ht|D@*8K=vl(nlZ`8;foija(d#i2U?sTW zFM_m0?RNG>t>}xO=w{u?5__C%TN&e{lLWCx?r(A1F;Axs z>-$a<`3rc{u4V{|M6Esl`RP&bbf>ffn6EX(f6TUeim(-3ts>E#$V1A&C`4SwhV(RH zKgb9S9GO`4`{Z%6`em=*_az7!axac*$jDq6qMj?2^89PO}dl>b$0|M{?e(x}nNnJ=grn7U!*}6oe7{nBz7>y)zYvt9zX8CF5xyc3z z+4_%%dq`fTmlPJ5E)3pD5ZqA14&uiJT_mP=BGj0!h5T5~FxC>`t_p?yK^D5J2ZuZA z38oYwF0=r#W@9x_dew+TR2tbJy8I=G$S#vxfsca+u`Z-u`|Gf!fYKlhCt;=ZHrDng zkVhU-zFYYwN`4;|to0tx-R(}n0tKi{yhR0TBQ(rT!}>!~uu!QsB~Foo1r}%-0uAnQ z$_iGH3o|NMfBDmp;Evub6Yf#Ll5#65Si)@}-41mXEGB3W)ho<_MLIKNL#{$rvIDwP zufS7MCfuTW71gUOr904ng&`hO-V~`eaB+D-6Gya6hPeWspWq z=~6`r!iX_#r+2se_p|F)pfSJAr@#vp>Md5I6)u?V-7E1#x7i`9E&1Kf3PZ-k&{sHS zx4s_X9s>bUdI3cY9Q17dnodo86_mTigrik<>no6Cj2cfUwI%oUF{o^K-tudreAG4O zEC?|OU{M2oEMB>64&jR%Y)7|&k1OL7FKnN8mz}iq3ZqYEMV5TQQRzub7@RLSX(^V< z52I9G7NJ)ydRK>O9Qr=ru^qE5WPr$k@Z_V&#a8(sim219Rq|xBO+{)wlH}xJnXJVB z+~cCm(nhmp|JP);_@C`^FGdnhj{J8$Qu&DU*=BaXTrKxsstl|MSI%D+rxO-7l5Z`K zs8Kr48Osd_-XdPP$vyF{Ykdb zdH3F&RCJo>6E)2qp~ee1DwM@TG#9$$;C=7fd?5+%Uxxao(7mkexZb@{Asv`q$KyxgV^!;O!yZp@wllMbnN(i z;m8}7xeY=)2?Q=^+2g8gAIE4i_BDQv&bj=J+hAW{(Hpy3#ibhAmpXD0k9AdPu+Rx~ zbg&i-Kss6oUa=nT9e)a_pI1qffm=s$1aeL0eR04gv{*2=gCq(|zyc?wPB#57?R24aJ<^jCD;eW}U zLW~bPWI1!EaADfd_&E?9SsQ8?Lh6vTKTT({O#2x>hf)!84~_<}#80)0z09+7DmQr> zgN?$Fe5ImMl?Ef1lvxpW6|*!L(X=6R$1}GWi)ce4bP=ogDvihG4&F+R*gz==5Av$! zcsOpms`+J;R7is|?R6ofBljfVD)Lt?T=A^POK<* zEr?Xfd2}r(jzE*${4I~7YeCd=mZ-48S%G57_K;Aq1lm7*gc>nkvm!(a=8waxbM?HE zY2){A_}-46+_|_GEWqYUt3BbE#JXtYR!EZL6)Qu%l3{DNoJ?>ryE|XX8FbiTBS@ye;t3y^e zcm4dB>_iR32!EJPA3<4ozBbmB;UkU_9`YTOT`w3#)iE>mP6Cdg$}2;}>R^GPyMwq= z1%#K!ad^Jqa~B)nwrSU9EkJ&XgW`k^$y&Pf&q$CaF)PUIKXGmRs zB!CIri9a#tm|n3aF`~)8Ztx|46q0qI_8a&(kl=-{X1|Bm`XEwcgkzQ~2szs%_g7uE zj^Bf1&g6;EukVC8a*mEluWdI5}gsl^@7ey>lpjn$5#)o%#?arJ?WJFHa^PvYNSzqc}qi5lMz;)L4m2 z(?O!$$N|Y!_TqN1l?@yY?VM_$6Vw}-8deKcuc zII4ywWV)$?`_z#7N5wv*f+~7PeSEWAfAhL>@~@Y-8Z60(D z{fG)8*=@x+I-_ekqd~8(!Tlxgra(FH8FFL96!Fd#IHRvX@RcFZAR&*)$bm^9sU>P4 z#lnLOyxUJBgWGt=>Ixcs)Z#Qdb`-WEI-_l8bid&a048uJqQnL!BQPaKG*$z>!}2jz z@?8U|pvK67Rw9nuRq4Wi>vR5-_JD?3hQNd2 zLzn}{tDM_oEI?o8GK5IG22k zhH#mRV<;W@CPgLt?Y+>OG~_VaZ**C5o5#IdEwi)12cGdEohyMEA6JktUWPzPs9Cm( zt@cX4Ob$$<2kY$#R4?E`2Hx$bk-=>x8+c#TvE>P~XM%RX@9}hOgEDdwK;9hau?$q{ z;X#GwgKaBR(z_4)9_3HV7n6{@<|lzK2M@|sHIeTtq6P37xI13HoE$q4-1KGW<6zMWw#eqYM4iQBv1eTEXIWu$4I`t4C^p&Xi28YpUc?a9=$VyNDPCcPYKC zHFQTNu5&pjOhKu{dA&@7V96zGn@2_IOsAjIQ(o75~sPeQ|dCG14+q`#j^h;W1_8S=@Nwt+m!KM68vC%?~T zf#;%YTmwfWYeNBA))VfkzUQxuSy$T2(@{2XNy#db^`#mcC>z(llskhUY=j09M4((k zLN^09`SZx^HMs}ck0|hl@*b5`g5D?_;_$Qi88(TDx`x@N{ph;J(K$GcXu13i`CvIK zvpFXdUyl(j;SZA{5(OD9Ztb)&%tzxD&6qa!+6_v*pDl%H0kN!6jA)7QoT-*U40tR% zBLqgYt_PGh21=vyz7h2=mM~{li&la`#nbCC|p! zzJOB&PQdgLX+6Bz#O4B$Qc6PO5FIW#S!J^Rrh2w~K(Cq4_4v0ev)!#zRkuNOl+^Ai zHU~Gd+xjqD?XrCZM#RjrU|$0rb&-z@ReFtqk_nm#g1n9GR6L(KObACSK%K2ue2s@< zD0;>F2h2}!(aAcK9bov$7-EskEd|RQNZhXaQ4;y%JsI+8xeiHQN!uYMti*A$IA17! z2rtf-O0A!b$3X_Kp)}^xM%9;0-o`7u14XNvRY@Mr^zP@U)#{g2EJw1<%Czx0gR9ek z71DbSvao-)r^ZpZ;VEID-@F9v3N>v7q6SII@KQX_Y3@CZq+G4>BNlD;la|paFkPnQ z!ZVi(L?ToraClH|&_K3r%()ewZ7$HFxFRfC!4?IG<<^x*LuIhNBm^Klzap0y?&@NT ze+)(;xtAz{D{T`)Ncgw(Tv@h|Aod1tIk5-EZqckyYw2U(#qgQn67RT?>PG31z4jIhv7PB?U#6&8E^l5|*lPMhh>CV9o4 z!{M@{BcNL{^5fXrcQu{w_MdWD9LAx#I&arYle_{&v=yg}()QPO!0Zb|i>*8{fu}(@ zt8bbnl-#&1KGdWx6CPeTF~Diy*7dgXuH0b#z;CS)6;DZcJ%UCNJi+zQ5f1FDDWi(2lsGF~BqFGkf31Znkq4`6_{eNMlE5is zHA`{?bHNWXfDBKxfYP}C4KM(54~}+NxpzXUlq*%q@^Y$_FLwf^*pOAJO5*&mkRl3C znJQK3J;=yWm5Sb|{l(|m>Mwtax>O=s9S15KJ=CR+E~I{bdem73u89^Zw{2d?#9#W^ z-=jOe++1>XcHqJmR z7FQ6h9f^L9K7GR$v!6z;wq5R6TPwdMR~o6!jd{N6hH{Gf1z-r4BCPRhEw;xeK z6cby8i7Cc4Q;hqqvIC}w5>uqI4=BYLobQTF6UvAxd&c5)a2Mp6{$Ib*4znj{k1G2* ztCdDunvwx7jaOwaN{b#`z60pznh3B(e^l4Mc^%bt2;sc_W1Wdjt+LmzrOtJ>sIsTP znq-oJpF^Fn%6<#NGpg)anDeNodk;v7K}-=Pra)!Cg{0{+1R5k{`@K)d%MR|*sIpf* zakkegJK~q1J*w<=Rrc=}cgw|eo<447+XRY}Fv$#6ji|H>Lx?H!QVv3C32m4AkI>jb ztev<4K9RrXoAb_$_SG+92Z&g3WF(BT4Z5>sx|f%XvH;dxW-enlI=>c z-b{k-(nsMxn>?i3*~*GpV*T(ByZvnbNf563g})YdO*Zc|B{mLV zn(Q`vk&Gjc_`ricKF`EACbPYJYm`#Eli88U$f2A;a6QzN@v4!0ym8E{F(J`!-AbN9}iYt7S0Ci8Ti?U8fjiHSjS?1?iETD~p^l*dlgUP55tbP3(DxP?C2{))YV zZB#q0v6|7qNGwZ65F<@MCBGrLqp1E#P<%S4K;1qY--4N{H8TjX;cY){wy6M=yxSw! zf0Ey;pGw}VpXz({9&*qTK4PRMrF6XP2;*E0;gv^FLx@zCx}#fyfBvG@dGs)cU8NYy0x|$-&SxfAXGL(1E~y0ZzO8SqnjmTU)BfKL}&xQ zeo#F)$1j07lBpF#%U%_7tK1>Yc`=Ht+(G10BL#&efcH%%Jx zODtqebo?R+VFsC2TT8~3qS$&FGXW>iY?U2g_5v7QS4)Y$qB?~7+j=96$Yf!j)jWOy zDNJ2W7PhMvIO!xor|N2wuwAt#sT=K%(eu~U41+2|SQ^i(_Qo%O2D_SKP$nEZ#ee85 zzwA|3_V^`i3E{Iisv#q7W!__joJ3ak z7TYGiU4s*{ zP5}MDi_C;>2Dip`2Ec)r_#7MtM%r)|Lz?ZA=^Q&;$#kojJG{>_Fk|lUU;cEc++hhs zGhZ+uDknPd8Ko|Mv2`teK9!F7@mxMBvGe*7|~AMY~8DU%r>SWi=n&HM$C zevHV7>BsxE$V|LW(Z-+bmP@B`|hIT{jlkexJbY#Jp58bV)5SzczM` zP~6XD#<0!(Tt3L%1h*+!^3ZIuwrc}w6bq~c93kY#~nVB2gTAF(R0V0$BV2`>TIo8&NMq-8+!``J=dZ5hSInj+BAWL!j!a5hQ7s07w` zQ|ba<-An31936E`r-zytOS z9|D%0RE8pVFD!b(vbY8fT}XJgZWg>{#So$;)JhdyQG_qlL`V){ZfS0{xb{#qTK*}# z99iKG-HC_UY9}nnS3yOoj1j7gmRB`OU~HT490EX^U~t75^s|c$1oH}bk zQ&@XZh8v201+Nx6VM)FT+Du_68j%r0(J>S)A=8#azQhYq!~G7Oux_4%v0UbXH`I9B zkt1WTxppsK9DisYS;ZX%W=nwE$hIID10v88`iJiKaq}DuQ}Pl-<@VY_->>(wCD6_D z8s?B^9Z#Z%hC~VDU*~~CR!{o24G85}5hIFr;_}tYTWN6@TkD^bn+evegJj|X}`)4=LG0z;YAcKwIR>snG z=0F~cV;KA30aE@o-fn^M+Y(TLkAvOIJ^jM@AgojP z4CDe{LBcPi5eyJ%R4N}VM+kDD5&SKm7My)h=>!Bqcg902H9NORryBTeX?W`XG&C*b ztn03rBe<)rSrocQ6S|k6t$B!63u}YfpRmwvMe)S21!4OH%&2&l3EfNd@KzDJ1`RQx zlYR?bL|G%nRf^v2?^&8w8My>X!LG%5r8#>gD#7CMQdG|QJXYA7fo7C8Ts{k2mnIAr`3K$cDI_ zjgbp$wCot8trHPWRn>eJ9s^%uLGP7UZRg*Dd0J6aP$DBn1!GiDBBzG^7pN`Q7`c-D z=Pe}AMXFL(sdEHL^1$HA(?EyTYy<)xu4X|6vssfPi*>YUf}<82xS@SIGcZJ+mo;^} zGraILj5VYL-oZ#GU4v%IVv%Y`YYZL9Q65%}MQ@er*(fgG z6=oJy$6?O$93Q!H!xsFYy)?t3I=ky8>7-(VU>gsK@GtZy(ug6xw&-JR=lP8)-re`2@bVW z;;ul7@U(Y6=|DfGb8%elXE;9Y<|-&D&R}b{-A`~5;D}cGQ%TO!L>wlqv+l0y*8Tk# z9?h%X)#VwXkjwMZ3M&er?0={Ez=s&7FMQHctcYRbmLO0tb~`yo-jGIT8Ro!!h_ zLUXq;t@R4?FU^aHsusYlg8ifvQe9iUrBXdn6`4ZAv`a=gB)ksULAh)XU_pfck^kUBwDZ+f+x**T z@UWBTILK$ z0wI2*1QIpDT^8%LB7vOLVwLd{PrnhBGRho^t%eTQYN$z;@N6TBV;K(**0tgvyocN* zD67h@;>N<~dY*pwHuE&g=2f*P$2|7%J%etp!3D2{7ML1?%nrWB^^V!WsEvaaSC}1) z$cWj&D&iqy2`W+o3)rp8HMik~>LnVI&}aruK(*%X#PdrZh%Q8UEN{vLCcglWAyzT* zh}KS!xCZ@+pV}b(d=M42`wbHx_%*CG*+#nXf1A&Re9bzh;JO_nLE^e@3K&h0xQ6a` zoQZWG;6@ZjN_F?hAeT{vF(B9>*`m8FXb%n(u1Wuta2r z*|CN<L*AVO0Za`?4g%|6pC5$^=O{_#Md2==WoGwLC`%TbsiFGH)=o_IEyi^mcs|~*y zs}X$4uvrx2+d2|SYc4NNPJ8YQ_`o~50zMOW48@f~#Vx!!VSz-PgtaKY0Y*mpf~kr4 z&IQr9#2Ju>;&e)I2`d|lOIzr{TPqbCA2&D)KzA0#-Cx#*03e$ z3@YOiS2h%vwrYG1%vIYGxI4@W7bv>+f-z7VoV?G|+2*s&Jf^e7bT+OQ??CBn0)n&q zV!j7L(ef zi~m?jZOWDaHL)O4_#&tf@$iATUz6EF2GU;Ct+~s__@C`^pQwS{@NY=u1D7b&TItG) z4&Bg~pzU`R0b@)FHL((_)!5LNAXOxR;16QpgvG#Kq8-S|CSMH9nFD!j2o`X4L$P2R zVrSvB(xxkJ5VOP;OxJU;^$k=at7DeT3~{+R13lwV5Y{k{wbdSYqV6-+RyLlaxGj=lePHykY?^T{p-_u zxfe~Cke$4}7Y!^%@HxR#RVT-4;DnDi`T-gf5_#$3ZOoauzdL@Z1G(1#h1EQ3?k&#b z=@EnR?BC4>Fffv!&_I9OW#FEtizNH`kZ$3`#ggC9JBAeSlei%jtz1k!X8Y{sC4}3~ zz^V4qA8-S~V@Myn#f>m7570a*pM^ALYu7@dl4rjK(W^%Aw8HLQh>Yk|i%vC(oMSoF zipv3TR)Hkrx}r$tzv8xSy=mo>-ilwYziTJeU{fp&r_>LWJIkxYSqKhs<90ziJugx)kI-0D;+M= zpsB`QX=re|cQWZpR)+WB#{VKNcO`0h z!A>TPjwa)sJ(g?PJGWdPNH|44B{Frr-gq!TC#<6EwGF9^V4eeiwJ`5lWlZK$%A(k z^Bm?;PLAS7^dk~Pb}sGb`9$ewbqGZ+v9A7-;j9bjaK3!w>D_PZWxD>yY`dJTf$(F1X7B+TN5n;GXdEao zH7TMKDHO$_a609uWEC<~WN!9~QRWuKp;v;03FZ3g7H&`B4m#T4NsGg8Ge`29^l81= zMcXjihBU$05u9gW8yW-=g&{FgYnn1tb^(Kq!Vs+fnhT0DDB6aZ0SIf+z&4a_K*Xq- z{*J#tyLkl;XK5sU-0X$1L;x-Sve~9028e%|KyX&5e4Fj^QKVl^gy;2}*V^BIiC$g# z^+;Ydc~~Z^tiVIvbpK_ah+MBX;>V?UQT6keiTs1QQI}N1YRK|x@w*yOWH@CaUVJ0x zEuN6gb%Zmgk*f*5q5K=(@K^ajb9FF9ZD!;-j?`sjMc=3L;ri`iIqCn@Oxu5T{dhSk z^A+&-={9)~n+y?ZvJ~>&$MWU%jB)qmQpz8HRagC0zW1#csX%6&p`^pPH z)gCG@f~n`n?DiVyc+O?FKu08Mx$D_LHqkw5z5Vz2@y`ZU*u8+Gs~O zohsVLuiCdvp^wi%c>8X-{xHYx%6&%q_?4SE?Bna0K5qdwfUip^SvOYZ>oR-aRt*~S z^)YtgYn88K>4iPje)9Ew^0-;eMDNHU!2Ko2@G!;>n`ogn1nx{%$B}YThPxcEcnmd% z`z>U?w+VL-W%d;AUp7Nf*vEKoPY}{;Gj@cMMItA|*spI}H%~8zM-ZA(*n%fk00?d z-9BpQq7~C#x*B`W($U5X!+Oo6fNR!2Ssm*bZ}e_fu%2!&Hm%`VYwZE z_JWpE7}VWkPJ+?`t`SR(**cG(#!$a{1^(|=q0TE>J!I>Phw10#ZrOYF=&R9ziXD z9Zd69!Ook|J!I@3(^Yb{#J+$%&hxsu^&Pgv@bsGQm#bUO4PPDRJa$@uc?=tQYmYLR zPqTA<`tEzVIzZUy3ImXh`FTfSqpO?tQ@nURtB3r2Dcp?Wc%Eqn_}*jv2DT+kcd4tW zBbXXvod-~JrhWs`qE+36SF?JIb#WNgY}J3TgI^;I>~$4&L{nq1^9X7V_Hp})IzQDO zgMBmG&K`RY_HhMuy)M`hO{WX?_M@eI1fa)Y7c8a!PzSRtjIY;>AYHXoZ^z@OIbXkO zKZ3+hwU1EWiNeyA@FEOAU%!G~=@#mUrN&&H$4?7TZ$C?&H!XSybW+0DJT2h7eY2F00rV5+S)pD3VP3%?h0U|j8U*vxjvqa5-PH)Ht@wD z4$kQtLCEGvqA~;L;nD*CZr{_wi<+MD?=NDB`;YPpZkF153INIC{FPyj>AntIY>e~E z_7iolc{90JINzk3dAePX#Lwkwx?83H`m^};;{UxFzx>bf*V*Uo%m1(M@V~5(zl)n{ zp*VocOMF>3R))U^Q!T(B$hu8)^qRMvdj)^!z5Ad4sk$7A;_2jnU&#MmtkeC)L)Oi| z|M&;pI!MXEUA15wgnv$MRL4ILsuu9ye*PE0qSx(UzUn`+K`jnKHUOXwu(=y60~-eE zK&^QR9;i0?IzQ81vGt4o;~JFdPH~OMu?XU1aE*--TE#UlT6zU+&{ytEb*1-6AA@%! z4C%0g7Ggtm69kiWV|A$W0BWwAT*1JtVjUDdN8I!g>se7+}s5TGNH z8Uvk2PYXbYf!hT7HOR>!0DYT$PPV&bD*9lgtMtQCWbICO>2}|L+=mB`a5pXZ2H~C~ zn92;^1E$8fZ~u%8>~%8)*y~25B6WAa+>7&s`wsXDXJxQ0ivsZ8&QTd25H*K;wu5al z`QbEZ6b{De?&-e&nJ=KRhAHhFYYxG9h9XN8ROaX&;vg8hg7{RdvN{;P z&mA$quN?$+%zCsi8vylo@&kAT)ra~Noelde06v|IILIEmn^fcM1t#ZM9I5w49Lbpd ztbP=uT;|FrFsl8~Cz}{y>vw`RKRqVKiGi|aT1`{^bf!Dfx5{SVd2=h$9bEktXHTIb zN>ji2>ObPYlUxC@IoE&w<8jsH>q}13?qI7zDaq%C&&p#&X+`C0aN=Z1cK@j9bUsgIaP zpNedW=~nQ@c}TaSQeW5+##-)hq5_E7sjj0RT-MV8={wOmf{kU1aj&Q5-Kk`8{2(6-FkG;KirNLu8ph!{F+SSt%Lhkdw(;4E>Ob z>E*vZt(SYPBEP-Q<_@2e9h|H@nQg>Z^3$12hlkJOX2F7>_osT48w zQ+==ALJleq-yW$c{gi$dhlp=~7blOuon>d9!>1%R{Gq4lMqo#H=eLW6xSud4U{Y}Y zHO7eW5;E=xo(&&(==>a>51;Ub7~JA_)9Liz;@|YcD!mtCq2>|yzF7n=gKPfCW#Hd} zdD?gBR&GlVFwRx=B=w=oOPR+0Mn4_<($y@PK-Jz_00Q^*$BKx?m^5D?o>x z-epTQ&9b@fkafT%Bjcv*S8=O(2tD0+2_z|v3Lo*9V9NoVNa0!V_*i`1^(?bArQ)!tR|N2c|PAEHyMJW|1=w_@#;GW4kQzJAMC-$Pe=MTkm3 z4L+MGy@*yEBlkO+=DFhjcqM)!T21qRzx^_w?dKn#;i>6IH{!soV=SwB)MM#d2GP;g zF)T5AG5Kv`>?_|2<7$IrN}lIy@(el;xmV~M%k3fZG_A@68h<#Z*PNfE+vF}gQ6O1= z21kuKJCmbz=2>S zkQGic1WfU5C9cdvOF>u^xitlhAo|E^`N(oga21=o~7>wEl!yUg|gbClO|Z3k%~bMhHiGtjw6H`MW!v24Lm@Q2R*Ft5Z{49ly+_YJ#t z2sv1WtThu0)XtD%IPxBdvus+Rh|zgl^}OgR6kS7 zQa`n-uf7tB3QqmR!;`r_MfEG9e7c+U)*v63dlTO8T>#6Z1GWGcx0tG@*}9}o$%-=5 z6oqq6n2bx;wO+xA_j4+a1v*&m`E|X9?eh;y4ZXxjK~KMy{yOO2*=!|c6E0gIeO>?FRGiKceWn-Gi3^3EgH6I zO@{&oRwuTmNpkN;1RpPH4Cy1P_|4`Rb-@KjYXd=#@H%2}so;SN%(uu~!+iKYZ{t zvi9VIh-=qm?WoC}0c(%!Pz2?O@oENFT`vGNi`4LzMG5>fDByvZR{2W^?WujGqr^>7EXn&Fl28gjX>K0ci7_{?f4O;%7O5k5XttbW zv{?j^byDHI=vO4N-&7H-`q{T)SYNc+8zZ)~eNXC$WlH$2qHIdWn=xZS@qp$HVNNic z&o873q~8h1U|xgL=I8{3MY1>=n+T6u~x{)ell)D-*Y zK-3oGKFGN`$T!IwGvFg!Q<9`#GJiHhv+kek*s1xPq!nfc_e7~@<{-gDm&lrHC3NIs z*0A8T@7U#T-Ia{D-gLm7d}O%-BPInAl5B zOs-6F<_jJ9gx8bXlt0X;G;0qnCear#X4NSm2L1aO^v6b7I?w}}*ujGS^x@K)i#>dV z4JI|>*2SsM^J;X-QBL1z#I$I{3`C;xJ$%3)#XF6JuPpjpv-4QhnR?A!gw){$>mGfs z;7yM{SKUTS_|5S|R)13kHmJM)bW#DYWafObTs24H+Pidb%HlhkM)QE6mvYq9sURr<8Q zn=KaNpt||$X@T9Veci)D3CkVJ(kA;nKoEM`$;4b?@U^_fd(=hZ8 z*f~Fe{sBnJrzz)nCk0!<--f4n1M7aW75kQ4s^O$l%=3lOq$*ujr+3eqLk{ zC(kI)WJc@OeG7fs^*nvth<=09O}d#rh+mxePVG1H!m*7v!ZGX}XIACAbSuuV`)$2+ zPI@(fR^zZ&99JHZO8z4!F-encc4j>C`jB_VE4{q?zvc^J{ViaZru#3OWasQvVVJ9< zSX`&P8>m9l2q4PMwy4Ov4USQh(K77l#2IZ>5Gw9!GCrgike_--n3YhsLdv3)Tsc1? z6sm{uvI=yxRT;|5M{%e!b^xzegB|>uu2%2U-IF+=Me4v#Wavk6n($rr{&$!R2Nz5# zo<-s)E|x@%Ep_(Rr$ z$n8KRzb1T_!dy;GdI)hh3YfV#l*{Dj4ia!SGnOYtNuSb6P1fT6?m+i03Wlshjh%6iE9b11m3`>Bih$88 zWGE&XQSLmwFHJ5?VPO8Q7+Foh@Xc#tgOmR#lCEixqwVCnP*CK6BWR3roO1->-O4C3 z#T3|6W9Sy81yF%KWtgqD+atue(4IreK7to%%1=?4hjzycVe8G)?fPz= z3J>*axdw+68C5S9Z|W74eoAXJUp{_ZuQkfQ1Q`TfaV1tJZn|YpHOo*5h5Y!jjG&y(G5nyy!0GE0N-oBlIH+F{7_1x_mAs& zsxW`{%n+-ND9coQ>{OZIP1)ILYIj{`NRiP;W(ab0R7)px6>S}>oQbrBMe-n(LRQY? z8t17fdp8_-`@`Sri*&$1=PjW3MkRbvU+@FQqe>W{65hg6Sr9oYFD%VVOgH&7{ODMu4CVOdN(ruCVy49etTj|1JJa50w6Gvb+!%wBMo#_Lx+0 zrGc43!xMGm3eAE6x10+hgc%s|pMGQ|28FfEu(A2O3L)#CZjfA<-oT%bUd@}$-_-yr zcoCD7UXLUl=$yXeCMgX9C(19>K@owUQw9gx`pGHL8msO}i=2|C6I9F|jGR)Wno?CE za!RKBpe9a^!6|QoGqZZhDe?7c7?_b$MoyVoly^f*e2+RM*x-;)L0twV@w{MevvgBFF+~n)t6r3qo8TBK-4pt{X6I}X6;`ucZy<(T zKEg4JL!QZgx=+*9BH7J{AM>QSDiDuq9P@0@loF%kT{O|+60~{|MlB{Hj3Am}dF6_e zT3Q8;BGNKNA>t>8$Q!I+B{o!P z1sR=^gRlAOfxCVQW0!scZj-XiVXd;F68)&tvHbs;Kj3DdqSWY1*AkdUCJA`VC}i{w5qar?RWKvCQTw&eQwSszY`uLRc%ul*tLU z&t3v+)|&{*2Qq`7%q~SgWxbeoZZjtJz54k^Ts9Vzq<*UJ6=}&q<-|s6PQVwX7L(@@ z5@l3R(W`m6YNTm|`pe+*ibJeKWzsi0$+9G`N3%M!ezRrk;MKVJw_u(|lWaI#H1q^I z&Xeg~c7P4^KJ~S^$fDC$^DMgcN}#2WB4fjCqhUURy6p>lcHv`jMFP^f6kt06Mo~eu zCZw$6v9>EZnFd9SX>Sx#+Dl}orh=^|ZH^(R-JZ4VZJyr$p3L{yC3pAq@UZ+NlSE`T z6cLU^BL*gyOK*@~E@Ns>lJ(x6iS7dia~VF(P@gX- z49BDyV0?`6dcGu6EJ$I)3hMJU8AU;=svD5SR^kj%kiI@9Q;338q+$w%04)j#;)Lk` zlJ=_veWD;88An}=P(#K~v$_*BGt|$tx)}v2WvM5pdln1Qvfdw2kO~9av7&{QOm`iZ z$*W>VK^g_=H?L8Sxz3$vk@BdcI6Of%0Q6GC0!7d~2}zKM*WP&4f2!8qbTpm zj!IsLvI^Fa;mn)>;|3Uw_FxUyxPfky(YTSSD(J>V1xBw>0MSd7YR`L5acHxBrNc$c z>L27faEW=2ZY(eo9pFBY0<*pvH+7m#v|EHxk`e`?3KXM%CwIwe`M6xq_Q`bpv|7zS zCi73AED@eDIP-z}wvJx61M(-lSf0^;G8XzgeH3O>GTo#bD6P%3qiCj|t(opIdk-6F zb!>I(&2AMBpY|D; z$enuG`PUW9l3ZS=-B5?_l{$$UNJ=KH;_uMV`|^2ki>DSb)OGTUm)V?mUF#({aLaN> zyL2aYnDOBXo)N|eC_U%WoZ%bCU|#`Z-ey8}dYa5(TCV4-C(!}_7(9>XrjGFFn(!;7 zrXd)qdc#4cJTSCKHg_MAb+XMOLgGKkdNF)027=^ToTAV>CU3#M<(BNL@W*~dg3)3{dtff|zeL}b2K z*^g}CnBurYRX-4TCnSzxD!is|`|z+5nE3F{tB#nuB8k>TOgYn|v3w9J_p{6AE;8^I zZ5P z?F{^Ya5~m^6`4b14x6mwyo6EX?7pa@83lNUW)3B$8fIQbQ%y8f9$Wx8>Igm15kfqn zbwa(xbbkpZ45=eg?0$ga|GeX|(N%kTu3FwnL-ck#92>+aYN+@ni~?=rGf@!En^B-p za2*{q<-Mng8!@<=l^AJfK%L{aOE|?@MSaekQ^XXn{_7MatLG?VY-LJ{HCMuiA6dJsqi`Q6j~ z?g;DJfpLyo4Gxau1G%u9srIe1(zw0E6t1e7R)K{!T{&^ftYNgusl~ zO?Syg{20FQpq}PSumlEbM~2qM^jHY!5LJlF$<2jnE2tb;YJs!$Dsg9Sr~&64C@J<( zA49EX`+FnG3Q=a@L4V5%d|lBj)7UN{Fu=xu;6|y5naL0;X-?`uj0(fv1X*+VIOdih ztq00i*|}O8gst`HTJGdwx$Z&Ll9!R4vo#9(MGY=CWNY*c6cJ!zJh!v+na|c5y#iyh zfi-&N#tg%)(d*zmfy~vkGaxJ9}%ioz52x+VXXn{}>v^i`FQnU`jq| z>5L)Tc(g{FAJiHhL&ArBLZk>&39KIQ`gwT$j3^}d&c)W~6~=8izjmoLiaHQuTKcU~ zd>nI2h&6g0Y>l?gS%RkLH|%RLzV?$($E}6~#+5j8$oh@3had+^_w?m7X>dD zVEmb9K433ar>WzQ5Z=!|AV4#L3tT6I6SRim%<~~g$P%WlR=;GCyg6xYAOOd(u|;+Z z`7%9F!*utVm~MEeVAp#f&*)H(5y5BkMLy56E&d^kMt}4Ajrbf3ic>%3*AxHw$x4i* z5ZhMIxTnv^;q;h179t8~GeW4z<6cMDyp{+geGapLrg))B(ER2#ZmT>9k^DzuKpmZe zJS_xrrQ8qXqUHew+z*2Hl1g&cMZl?t@lpw<08|;u%aw4bQc6I%PcKu;ShRqLn7uo5SyP=P452bx$oJW9ZPtSA%Y(IAl$yp#ai z$AD!5ykC?FQ6`kx3D@9Y87?K@`$bBKlu$|u;C~x1C14d%)Q+Nd^tAi<&1>cTHl1^5 zCtTA;&x5lQ@cp7ph%y0Xg8Xcl3QRnvvSWldgEUVC5~tP8XZz`D_?0Qt`*8gB@xa$Q z=^lRu`Jmm=Om`!DsS!osbe~Qq`0LpbN;@n3Q zV7o87y^O7z$r*)N{(WrjYYDALCnuLQ$CHT&XxlWb%8xeNWV21@$!@1lA0(VWy8!qQ zwHOvJcg~M=zED(}vlj1UvqYY<=sQ0jaVMERZje!ppAU=qNaX5({2lccp+)&OFi-Ln znoChuls|@~IdTtevrG!=9mruS&w)F%g5YONja;NuBg(rvZlw-Mw?8vgftye!0CGRK5-K1 zLo&bHCfRYM4@uPe5IKiyG8eTztc3u^meaZDz#`xepC~YPnjDaY+guwdjj?5s0C13d z7z94y3qo0>#id)cwcmabUW)n0XO}l_$<{HHKF>iAJDi=$=L0y)Bn*`#1OC^&ZA-IMn1HhB>H*J{Fiph{v+FOb7CUT3_ME-8-e z+4o-TzT#nIjTPIAHs^JjuPImHPMNqTFUv@)LwilnbIf$x|* zpw;L5*-~hqZ*%Brd5G$canT4Lgr8=gP~&V0eK$;A{}1ftNg`qaU$XQ)^@}-(99kBH|M5+>hysD)PWa@>!r@;Sn{qWCq!{$f=C4iGOwZekMVBDXo}IzE8r-br3F71a z`BthR%HP!lVO?lhp&gbB^~ywnd7UmzK$bO5ASVgb3ov)uQ{5#%58Xd*+?pSgD}9~` zHQKZZI&#|_T7vsZ*KA%O!^02hcDAxy2&g9fu-nh(p9F(bzf9O`fvKuXimyfC@&odI z{K1ZiJ3+6TrwdU?F?rmHkIqEcWU?00@gdFRf&TSrz1)jZ4rIoL^TL&mcgY_pd+|?( z2%79ft(KTMrYgFh4jAwL8|p`7#l}QL&$1U+Jq~|=T7SywNq(8A-{hZKyYQ>8#1i1+ z)R^uZ1@(Zcfv-EX>zwMX>D_#{|1{ly*(9WX%iT`%8+X~>nTzHx0n!2=l^(<&QIW}i z^3olAx`d4T_>OnYd7x!<;Jii>E%{HJwR<oXXVm<-i#HxOO zv=>1hqm|W+Rc3FO@y(Iuvn9WwR6wki3Scx!LUaVYvIRxVtNapO`IMaZ(k68M^hY_N z^QNrOSWj1oD?@;h<<*=ia5vyL8xM-^gZmb16nm@UL;C$W|%+`?hYmRKPWO~|O;1mYw&XaBAHd-^2 zaNt2YpakmjMKI#3*)}D?<>?0L+{S@1NC){xzzd?`Tg&jxX~EUCdKjd0EVXFDVm#DY zm}tUs=7sIZaLl&zGGV>;vt<~0EOyK4@?FKmEjut>jnfLPv{%`x(6DWDu87tc)es!x z?R-4*`0B1^ZswETqV;RqYt-zO)@4|Ab%!QvXn$Qf-ztW-*Qnu;RB1gx+2M5C#v@l- z=^_(;o^IE7^Yn3(rP)HR?SsfuecGNuJq2XaY1~U(t(bWkaz1=DKxMv zPDDfZs?q12kOx-IG4*CNhC0!kL(R@S*uzlZVH5JSg=*&o<_gF9X|@G%=5elS@lR7HTHBysQ~p{#uW-Pl*UaCkr^Ccp=`f zxMtK5f*mC+fX&!d^(T!hBPkq{7%|xE*1a{iVS7u9A(2eWZ{??5QMV;XHWKd;i5IFE zhDp3*UW_=*{^SH{tHe9oZf9R2@wyNnfaE>88r+@a?O~$;l&9gu>$UAH^->&3_g+-W zLP>!cH9JOnPp5h%@N2EAU$!Hfnk#{!hm;OMjtJps)RqvVNN-GNvtpzNf7kfifuAIT z?{~0F})seNm zqmzN3V0(M;6KoqGKvr2tdEyJRgVa5aJb_ddtEEh&L-4{Z(d8!dY<#hYlYsL zk%OP~e2b|#2Ryeb(n-h>=nYn>0-b;r78nikVm8Ex#|If%!Fk@1KPwn*A~!)c?KM$8ewd!Bd9bM)6d_$#>KZ+6Ti*ld)yiuTM~$XlkYe# zSTM9n%RZ*_kIDQ~p&uw#yY8_i`8id)o+-=7{cXq_>gFkXr2tWMPOlSMh>$At!M&h9 zS7s^d=z`wB8|s2oGc!BK;&GBVscsws(BJ#-o4uL%Ix$;EQsY~67!W;m5D1fpWwOdT z>~M-q#-Ee*=erN9^nNBXUBs{00|+kuHJL5`M|8W+8VH)d>mk+9rN+K7C_9hx{9g0< z=I?4QDrrtMEzRD&ShIuqo|Xeq?{HUsk%$&(gNejY6D>6r`dld_yt&AhMMW@#SAslg zQUU_B>82rp6^#?f=a1@ZQ2XuMSlMsN2-{SORWh}zWO9Fwn;LPa(irhiMnsUSs6m;9 zpN_%Uu>JM{V!ccKw%#6&V7ZVySH<1{g~!?THn-7reVV-iUSO{W z)A^|sOtfU>XY377r}%-1^q_W3l<7Jn$KC*Nf+lvhFmmK79jX<31DtYdb`!%81Cp*~ zSgSVxnuF2msb%%#1~_NQxYo-FH1hVCK-$e^dIY0us*!6-nS!1r|1ox1XkZV8_6d;J zIp%omNEbWO!7)tP0d794bbwtJyk#QtF!E%(O}Enxhyl}ZDon`eWAPIWRbGNp<@zpZ zs!3_@EO7caiXZo|MtJfl^7;ZoA_1rADDxbK{M&pk>PP-5$TRyeo%YLzhuLbEJiB}W z+eQpdntKRi$WiGC`|VRU0GLVS9vlr_110{2RuAp!FnQ?O+oPH+T^(E98=P-Ea`Z0< zvMOe~=pjI^#T+1DxDMJswA=D801g`S&jx03hrI`T$del^BszncW;B)j3l@uJI4sMR z;e_ZHP@2HpqW!gYcZs94hGFrmaP? zO7@YsK43w^;RF{JAi2@1vviec1vZ4Z>R9c?4vUv)Fq(cn#eoK>vF36$6-QJK-=hgm z9-X{=350h{i#)2VuF4wVCM8}tKGECnTXX3@PjG(erb89Gm+*S8d&yJOY314l7j@Qq z8Wm*S)`A)$E6;bv)O%EVjMQebMLAKzOA^n<$kdm9TW`~q(Emp27xhzrq4+h-*<~E?3GW2-xCMY4lqMTN865?qk z&;Wo35xuIVfpN_8_sEOL`S&QxW^_p0`67(w!D-6xbfkGJlro=*FUWR@6oBlzLq}04 z&4YTbNnQVPnlz97gL*>qC*r3de5#CbRPtMTy^C4PhwW^=Tg~>#9sCA|q!pS&XDoz4 zLDNDQV58|!|KcY|aq3Z~zjfy5mwINNcuJUhRK3=0jFEHA+R~z? z>UC4%Rb|=Paii=7s+7c8nROV-`tmgv1KTexjKk$>ww|rN{5!c@t{!rqtXwlqn>?(= zc-0xZJ&B_v@4aj_cbi`wrvSBv7z(s1aAdC-3QyjProU|Y2<|N{b@VS?il5?uX_P-u zP|J8|J%Pyb&|uCRk&B&={Hdg6cgNi-jqK%aAjMG0-fY*=z4>mn%%XlqG+K-CsA4Y` zS}dD!XQx=dqnbt^uL2czR5jNkvbd}R>IXZPXY+O2Tghv}^K`quTc^|IdbM0brYLbD zXd2>~me6YPawmN2eBs(7yb>)Xem?su^U)|LPpg~rfr*3CS73a!9OmtDoRrC9P7yWLQM#!Lc~j%Wfa=Dp=+G-T^@!g7%HDo+`JAqD;EBAOKTIE<*7N-`U7N?YcsX~lR$R-D9?$VF$I=uK&uA-tNdgTJ=;CvpoWlD2rm1HmRxp3s-ffC ztUQqS4Bbx^nrvkn5-!@%gqv=Yhi8|sz${`-MRSnA^*?pqAEN{@O5oB%k{oZHxT4x3InmCNb@W1n$kj>`%FB|Y`y#q8qWhw&&MDD( z4Dk!n_p=MXua~{$jdIHy(4TR}NUWWTwNv6OG6<1DMqD6~_K?3d=wF6$Ew|r$2SW-= zS*1GVHWsEjjo@#dY)#>mgZ#;tU_X)=oEb|WYDu6k71j1t#zUcng`upP2@1Z+WG%WE zKcwOh`L9pw4ew!zyx|wBn$sgE=9{*(23ULQWW6Lo7bijD`-!m}5a3posu#-w? zH&#;@dfJi2&RS+SE%W#KowZ7q{Kt3>bUX(-Yrc&xdg`d3;2G?qttiyY&C~3-Vur7X z)DngpxJ;wwbNVm{-w&XVy=opMB)t%7I##erD@r>q4WaGdXSndNB3b^eJJ>kQ$0$;Cd z|7k030EZ!TRJYq{!YwuEkAGkeIIEe1SI?5PjF-+eYBcTw?e4%Ba?VbW*)b`w9wsvr zB|e{%snF@Cn@{^p{u0Jml9?|CC6{BNgz?&oKLl@o3!lo}P&|?OZvUyBS%kI;;a7~l z7K}A{Oh0GGx&NH*;Y4@&6WGBpovpc`xGMJc(TU`!E1gAseBnfF z^6^asug=R%kl65B+rPq$VQLhB-23bkZa2I!ayaV(I$VK+!tmQf_k^hs2D>Nno#b2& zj$FTcYuXjgp92+NR!2vbmiK{(eHM0(G5+?re56tglF@JRc+93iZxouTC^=PR-I>WDO)&MM12o0F(Z>IpFYac$UGjLpN~RkS zIi@Xk3>CA}U?ZsNxu*o`XI$lt;=iFBMoe(;7Z+VfOWUbLYwdjmTIuMYK&p~ID}KCGr+^Zdd_73^_w@h#S3080L$k2 zx+otVrQ}C8)zA@rmfKp2$e`0@^Nr;f$qTwR5P)OkQ^4ROnFu;PzDu_slKpS%PPw_pCLQ=hx;#kG37Vnb1W!M{ghuzKRhlF z#j#>x>7%$^KjWT0BZt#t@`y$wK+Y9wI2-Eq9-)_k@WMPau$qY%atA7a<~Of#zt*#C z&T{kQeWr%XGE8YK;dPZ&-T;R@Xcn*Rbr0rYa>mR2}2(v{k@3!Q9bGiG=nP5be~Qq z_-iDY3pm84qNk}r`7S0}xD~;MxqCj&tE#5QF-YdiZZGY*L3?DQXi*soic14U!bk5! zi@{8C>lz(E0lu?$PxregaCL}dq=n<*mr3S05jA$oW1@cQKa3`LxL(=UOjiMI-G2?)qkIg#TO<*`x&NkWU@64B-z*}%6?7u z;v^aB%EIsDqfz%j$-8{OBfqYm!+wyv$llYNM3&C8t4r6Cz*{_dTzD|q)pO{LNOHK; zn)q_&@>xR6qtYz#am+0haDwKxq+NW+GsB5Kt!HTaqtyykL-jBM6LKeasY(mx3ehcPl)F*Cg zEgBgOBS;Ph@ub(8F=`nf^(&Q1HNzNZ;Uj2#voLarj|~n*D#&(%j1c1iC!o6Zb}|hL z8R^mRJyCQ%pYHYxQB0FDesXIG?TWdR%j-1MtaH#)VB(3F;WVDKYn2r=`1dB{g*Q2N zf4jXJ$IB3-n8t5`}YcJ*0g;y+`zyUw%sr@ zrW6V1o)un-FpiE^@;H6cN&1b-A|xFL1`ruBFtF!ql+Xh1w83=_0B@z7*5*4=swhBB zXPWDgn}U`vxXb);0}_FwG;;`sszg@SJl(GEP}SHSCBSeZ#+2UJ5DTzUfT7aJEuESB zV^-Lzf^;S@250$EaauQQQMfEljinLKQ6f!xxwgyw$2Uun`X<&~xV_;MWt`nc*tUI& zPtNtq$q=*#df0QR%y5cU53)TM! z&xVRA8kSB{$&YPyV2IWKLf&A09VGo!sQ&k9*3HR-r$mNsi?g+_Ku07VY{k|~v$c_* z6K+bLPlW*Fl;h)=TNUb=(zHmP2rZS9+t1)>3&bBpg6M z-Pg!OwwQqQtP1)DPS1nuX_mK<{IP_TE`n=Q5!H*Qe_<3)9|6E49%oB9Ua@6xKAc7 z>7MY&2YEW22{#P8|FGN7=AQ)PQorbv7*ELW(mRl20)?*2FV0ot@y)ertb&;o0T_WAPXS(a8~Ac5*y3 z1?fqMXN94ZvR90D{;6?d;2&^hq4-lR1Z?G!y{~?GcF5P^^Y& z^LJ>}o{DW~5JVJ)Q5f#AIS(I`bxMnsjf&QcH%qITb`gc4W%{9Gww7%uSv@LjCa1&S zMWQMsQ%JFBHVu$>%D~2<_*w(Vv2yK%EgaOl45C{L(^{?O+dW~#XcuC53k)6G)g->-+5wH$}gK&b`PLwvSP)MG8ZBY1kn&PB?}{LjfB$;H?6i{5i~ zJozgK<+yhiSc6awFtTo}jB*cjnxp(VWWKkFa_r>RYn|od>#vj5L;vxOXQBt;8+Oe? zX$ZU#Kh}+v;mu>FF}g2b!N6^zdjN`yK=(X-=s&zKU%3TT2t;j6YMT$Pv``uVZwVml z#>(*KLDK@fp~|F9cwb_Maj&`e`)_}|_>ioV?QEYcdT)MSzH&<&eJID>v(Op@bGKY+ zr0mrSjujp}E#SO$B?aA*{Z_@l`s(7_^=$QJx7=O)n(Urd``v#I0PX;yAe{v_u@2k; zPu7iF`O@R$hcQk?KxPAdPZkxE4tt|R& zihkK7U(Y`#^G_KM&mu)xBh3LAr7u7?2*Nw=qlMcLz$2ur8!K~i4~$wseCt|_$DrsH zD}Voge|h`$Z(3K` zF;A}FK+ld1z4-etZ+{+)x_%R`uHyz;hz)@{V#&I(vUTdgPYX~7k9eEvIxT^W`8W0m?N!Hc3uQ|qFnSb+mX^!kG z5MUA6q(RZe5Re^SuU{A0h@GaINoUAz-S-D8+8@P)l&ollRu_Z8+tB|Vy?@H3_k;OPdAPIy?AG0N zyiU|FV2h?z{f9O{CJ5SfmFUY>{B=BFnnU|}>;5`^ru~BUW;p<8gP89W+6bPeT$=|> z3($t#mp0WApyYnLKKfto{v0wNf^f}o-Vkl%IUuJgu6eYyfa}(cL$qA_4eW_1173<| zCLb;ymaF7H`|sa-4mp`&IOmESgMbcDY7BIEyaS&WkPfk&$iaz+kK^E9eI;{}`LN7a z;$PnCAM}Fbhh{3Y(X7qKxkzMXcS&b_90qO^-F)b?U#j;$?Y&X?Jlv?{02;)&Ig+T1 zY#uL-`Sz>U6%6fnSDc;-eWxnpgPh+INY~5Li0jbx&2X^O9Odo%4f%*`ulV`=%Y2otlhyLR?*M;U!GQ=HX8|bee;_b@w4?_l`c*D~#{<3!$M5frp3t z5Nn#6*j(A7ipuEbG1C~_?KeIi!Wlfuh4{E!Ec)*_tkCVa?h!ogG&}Y{r#Z+$Pis?6 z0kfmu99+1?vqQZ5u7g+TZOqw<0XAkkosD_KG>0}QJ#9i8+g1z$+y8m-?Zs~Pko@lp z(ZGH&yO?~vT7F8Pk-dlqZF>)WunmJWbG8>r2=+ORzOwG(fmCzww;m80^EO1URP*GI zd9v9r)AjCRkzTCR{l$E?-F~@PrSn<#^~G}ikZvF0_ui90c%%ro)>6YmNT8KSkq20f zN#Hg{itfMc6VW@$tULaCoqqi}+1{tS*{_8m4z6)aXbt6THBa1pJGkQ;yu{p|H zccc&Duo87j?8PG4Bx~VOpD&YL-#PkixKF+A;SIN~_|!cJYL4~Rl^PhZ-wOTLKlL8h z;hGAka0DnS?HtpB%zdm8w2lEG|SibhAz8$!?c!FaFmed04J{pQ?BR zT4|8NQa4o*0o7Dt@qnl~+}l@JfC1Va$$N$N{x@gCQxn2)o$Z$e&j2Ql9rCUFD(m50 zG4a+Tp&liwSFldkt1lNK-cLAr-_?~HfKV9Hxe|#GsB`yKhdPYnXt#|y__%$AWdI!f zZQpS{uArIJs<}ZZ0k#0BnSgMG%!aHxHM_xQ;FDnNa2c>VU4p!+G7(CF_;S zJ&^3ftMC#x*ffjthCrLUuQJEx5!0MwM>n^deDdc%_gxRUst|*(=On@7A}ngCjOvcu z+}QBHP5qiDzg{uy_uu|F`SJTVy@xjFu0bj`$9O{+HmAc>hBl9w#ta)1wAzHW$H?s! zwtxBiFO$Fj`ttzbU7)fDZ!P;btXVSsmJ(a#I=EJYebr zd#d@A&WW35*%i+~rIJt=v!u!X-)=wL-KXhlx=kMLKtG+pH6y$M*QW4jnD^=K>G7M_ zlaHJIRweDw1NQX)FxvpfB)o_GXC#Lj$&uwLJ0m>{a_r8p>1rjO9{yr@gS^e0^#9M^ zyX{JjTv>v8uCE|w>NP^N3ytRhIXNJykQ61}nc0o%mqwy0!8=uuvYVH$pJntoM{cAU zNk5)BXcS7Q?u;PA-N|KZuYIv?X5YT{5>oxo{T?j+Tikx>?aP1po80{QfB)3|KmV6$ zz~mP{rc7_~;$;x>pMU!I|Ngi7m!H4??)Ue9e%o*N4e|iZmH&=^b9)^9`zb`h6a0+f zxL;0Udd(XzH^%qv^U4SxK{x#Me|}r4@;|Xu<)7F5b zmFenR-dOoO(Qgyb|9oFHyln9QCFh5Z@7Lz|KY9iJGYUEAeb2=wnT@SiZ%eTL`F^8` z|6=y<|IZ6l-iPWxt8l-+;EjvVGJ9Q)^7oH}_0xa+hc2`Y>%af<|H@~(57>j#%pq96 z?mpbR|MSzK-&;xGJ=$MOKQ{;^mEZD&-{PfKTe*}jyhXOR=eqs>Z`W{C-1qvzq+RoI4bHvZ(V6vj?dR`ZLOwg1hi)hBN?%P<1piipE znTy#PMd~$lY7(!n+ED~mkz17bQQn4j8%EADmBQgc-ucWT;ncI@yWhY4!aw1U_ju70 zyl`K^D~j!qB}-h+-GqBvte6HX_Q@45pTvv%3HAXncUTchKFb?>th^t8NTS658h+8C z74gicte)D^o?Fqg5=e=qW;89_uEEopUK&q%NRss}z#fdu`hsZ@y1a;kag1q#q3EV^ z7~MYSRhZItkHQ`VLO!`f66WO-V}ffyaQW-YXM%@*6otMB2#frN4|Xb|vaeDl5dMkKmrvx?{^Y!J z9RT!2h|e#8eh)z81%Lh;f64z<%YqmdP!-Xbj!V+cFIsGIIJbC4xxvN9z&&iL)mp>=rA9N^D?kB}kS~lxlVvk)1XI974FlKhr zFH4!Y9-CXgp;qZSPE`eSl(^a8<&%?^tew5gV~$nZK|!J-EbS?#^<3IrJr2=0B$Gpj z#;LP5&eGHjCRJLH4aI;3t{?u=c$G(+QuGx+OALYf{{Fq?J{cHwC6!mmq8wNL%mS98B4+D#f&JP`4CauR}i&= zs%w0kC#6>}F)X9XYd36~xDF!Qv%F%5D1m$1`M;YbzrN5x>Wz=GlbwZQ0sI8{~n@LK2j9n8Oi`O~TWx@FzOZIxC&Zj1^ww5Uce? z@5~uLwb(%mg1ddiQ^(F4f0&9epRyoIyvmy6VGq}exGH|s{Y1F?$#FqK#sJEWfZ~O` zes&>?d+|Gp7bcQdBy}PHQcu=2M^@MmrJW}{^0PVYI(J;ga!$fvs_7hrWk{BU&q?j2 zjTtA5=OIqGuW%~li*QPs$7L+iDRSDp8B*VEX*-5uA0%k_M5J4M`SfZF#0ahqpch8@ zXIpEbfBZ3zzy9!Ju8k4gC9jL6Z@AaGp@`kQYY;_o=K3N?`_$oiKqfM|fvA=`7M&Rt z0%bXZMc!9oVNJ;)u9`^09x?ah6^Bh2bVRK5^r4J{cL2s2%L?~X6jb=YS|%a=np z;D)^w)(B*#ESlRgPIA#M!JHDGl5F-IxAMCpNry@F4y*AeF>)TUAhn_)b-k{gc`~-i zSX4-H_(5nvaPJT;+*i=@gOtpRZ%=rsO;kxWj*_uYTp*@!$S1~`PJupAmwpxi?=h1K z7N?1Yy*Rep(B{$Ol6#k*&c&(PRqP@ zgoppZ089ZY!P6XqvcArpWRDT?NZ>kIEy8A^2n}~^(jXwCEL zYj218T-G!fN!pL&=!J6>xka7TZZwY`N!%z+yfH6YLZiI!7;{5Dc40A>Rze&mmSIVp zF;B^OWM_D$LxgZ&r499((8^d-FX6&NVht2~M$UdZ2^x~LO$v)yJJ42Z-G~(V^uirb z=JgYM(MDAcMMTfE7I>)HG)qJji+<+t{IIMDF?~K*bJ*JU(%5}(&122Fva~^e7b$TC zKi7+CcBZme3KsVhwdm-Iv9+4|z|qOn{t&!XoA2lqAR1%-R8O;&1!KEZ*$yd#SjqI) zK~(Pw8(8l6Mco$!pxW&3(85}pG+hjxt|2}lv)ypCS=$9sUD(1&8{hNYsts(B&6OV( zMNAxfaR-FyRhs8jRY%0hon>)YR{Or4w^QW#tct9;_lL%HTMv$A8-Aj+@zzetyV_<8!}doTRcm;9^u-S5}G;p)HqwI1ZZTK{`j?x4Yhp|e6#kFBVpirIF( z3}wt{kd8s!V#GyTMN5(6%kDJd?f172vf2qA=>0+RKYfXR_!@5?1^;gQC`&f5q#YMB zhJ%}i1xu=+930W6PCDmxTh_#PyYSLuU>FrV{M`E=U-3}|4$U)^VPDvT7kK^P2Mukz zws!L%jmvZzf~1K{-dYIVO2-WnI89%iigFe*MVRvuM>E&_lFCRS{yl~4t$$s1yf_iQPzi3Qny2G z(}woUtN0w}B+Kd$UcaGP(arIt=UH9X9pS$G@aL~ddWHIN;SN>^!?lvd8Hm^Uw6N(A z+g(&~WHYH%=arvWHg|J8IQ6^e(i0(rRuKAQ?!^HjV}*Md(Je%*Z~A)aEU16TT3W?% z6?Z8gnQKq^G($H=!;`<6pQ@0C{P&m&JF zmd$Z`O$Lx)D39Uc(LIG!dC-MWJbW|ln#Zl3q3jZj!24Qs!L)+R{pFTshR zYx^!q{H3_^K1GxES>lqgv0Tq@L^O3?*0LcSYnQ>_FTvO>(ZH83T*^`m!(un~Xf0D9 zM2Rr1&C^INju~}#>jA!kb(;~_m}TWKT5{%>MKvdmos@-7t3)&tv^XOpk9)vfm2miJ z5;z6QPNJ`Kw--JmuGKBP$x!vn=X@Ydou_?m^{s8qjU70VGodzgfvz%7Q31}q?EKM1 zY?Y%Mm3&HTZ)~zc)Oj+dWwW?Vn|7U(xA{~CLAPY}fXFVsaKwP>tO!HOLur{2tHeBoev`6`7onaoY^r_;g|}qDyur^@~lg( zEY2osuhlAdNl`N_dMENa<~LDTM=ZVefLoXGFG>D~-4kjUkva_3sTAo|8o8xF;5UyE zcl)5JN!I!tC5CF5d`I*FLi{lqXx2rnZWl4e4s4XiZs4NOdId!MyfW~A`4a!@)~jh^ zmf5qLxS0BaI&oeHGx5b-6(NtqC{HrhB4bP^&c?c$-@;GKjLt<#}I4Cm~tmG<8lCvsX%4>9kRwWrKnxOVSEEVm0aK{>Xg&dw1A-qb$ zA<{tr4psgzMzupD$90NyNo=<)8^rUzQ_K~rO*Ue!fMknxRY|t~FHaCn?7W{Qpc&lG z_BqK$$LbMQXMM&HZKfzlwm{p?`%DBp%Y|Fd%V?n7F}w0D9!W2MQMz=Lj%`R zT2@)dhzNQrrztNmM$SSgNEhCZfB0&7x##bmT_b~S;95f*FD!{&<}Ef$EUO?aXuG`P zZqrS8DW?@7B*dX%7Qp?Wg2!4irDZg3=^mCoj!4| z1-NwU9{=0dFF(W{Vo^nWG^9b{+3ir|>4zY4YC5^gOsd3AT_+t|XW>OBsN1<=ckw~C zJ}>{V*hfJc9tebJQ65`#_TnT8?5gnw6n)(cDp7Rd%7AZn@%YZ$||P>hXB8KJS;1<`l$)3fwqC*lgC4`_$5u8rYfz7(|Vaxm~qz4eN5WfY5Srs zN+0v8cQ9Drh1i-7Uym1(FH7Jt*FVTB#d-xVeS<-Ejo|cA+3|AwgZwwyFZdANUjBFe zmK!XNgIc(}{)P%*ssH|O>*wvyHn4sFtNhdtKTY4`GO&_=l2`ISc0ZNh!&Gm7ke4*0 zfQxr%vrn>od%Ge(5Zv9Ps2k$BzYPVx|Lz|9L~-wSW&NSA-~ZuD`1QYi$*r%k1>ajF zc#z`j`d~%${SWz%!C!D8S}(j&s!sip_fhYTZkt(4oQ76S1xb|YX;APR;2Yp2RBtdF zaF@@&0w|wR;>|EvD((#af_}DdtN}gczkCN80K>Vx|5e`mhC9CJ8Qf*3g&B7$g8(h= z{F8KB3hqC6fB9nhUho$y|AHAd1@_33HLVTm&KTL~l$kr*8_Ft3=`dNHcb|$q1Nkbs zlJjqn<6Tt5UFrxk;Btyi2RPt*25{K~W&)gjvSrAi1sqdXFy706=U@J0d0+D#=0z6J zz;e6|gG> z>=>04eAT;<=)g^SI}}b(c*xoia-rGl9A41$4Bj=2RNw{R8+eI;XdA`D#ASTXGhDtw zXo8nWT4=e<;ayMQ{bPd6e4`fuEt9OV5?|zDOZ0&;c_g4_jW1TO~m zd?S|p#9zNBpMG3tZ`Kd!VzZYQY3qlvnUm&INQTIrsw`eo6hj&$)yrbdZB=s2=vz@# zI_bocAhvAZuIC^x>j3X}XXKlR>_TLKS5bK+;X4@oxk zb~~3D3Lh>y`Owe6RVT6Gk9@L|$a2RYw)4Gc4I(d}<@LAMyu=^8jLIJ*AG?VD9@e{v z9`GS*$FphpgL|@pM9ZJs=eEB0D6-6TktOqTYmN%n?J`2!Tt}d7_9VmEGptuzzjwq^ znK!nIIeUBUB+_FIP<6Hnq&|!_yeaGvU#!QPt^udUR@cwTGM{3rM_B8evKX@=vBRzu z^A(s^V>_X=G+rfHk}cpW%p8h{)@Gw58eB2HS%n$YxJqdmw4f#Z0A~5zdr7>(jN!Qq zbMfb|DSk#xQNybC+;J`!yX7Tc@+t~_Sz5buK8ExZKYd1Q;HVjc7XF*B2S z<^6Q$d-SuFl?w_Eh*a<)e=7EgbDxJt_0((W-_t8H_MI8i25r%zU8*JA)*|t6d zxGx#N1Ea8dL)jyy8HJl1!T*>HK*m6d0T`k%tpml3LNxZ{Ff_Hxn|z4U&R#e{i8%?k+ocX7?cGmr@>rg z-6HtF`e1H)va=O;-~4Gq^Ky1b{* z@JI}_p05cDN(aU^*yYtK?4>l zlO7N>RGYyhy<5ROdo3z!m?W#zYs#rG=}Jj1!hBL3=Y8j+p}q@d*1DZv*zngE9LG7# z2u?QUia1EMSI!9*YZ|LYrqbLF<_7{NkoM)1B%#J&SC|cDW_<2-5A%`1YMn>x!8-^y zOr|2n%CuhTMl?rC5Q;N(lscni1~QFQ;igi4*GB`Zcl!7!t7T0}3%7(+8nZ4~ zf;HIN1ne+Q)e2;Q9=LW282Uh$xz7Ei5u?xScnE5%nyie^B+0C{6seB&8XV zl!b<6w|kJ2L+#Yhi}nkahr2DS5b@=8Nbg&q-VJ#}AaMPjKVkE1@yvN4Y+b9o z8C$`{nMX~)ZsEvKc-IE*I3BcI3c&}dJ53)+WP|O1`!>Wx(GGB z+c;pT{FzqmIRrARB%+SX=x3Q`y}h&=%fa|`EZVt-$6`5If77xEK+AAPt5vv%sdl`! z!lEPBpeq)+LDOK}P#7cw+DLV3v^7>d%^A=ziH%c$hHZcby*@BKhHq9woyUMil~eD7 zS1=73^vd?frh0W1P;VOI$(suYi=eOIEc1z*l^T0TcFtE_8U=FX7%LsXu+SIp%r;c7cZN!ZX9cm z(U4l@e&l%f3mD#`iLC>W&hbQ;*5SOLmXSPEN8#0eIWh=CxCSVm83gEyQI~G_OoJe4 z^xPoSp*32TE#@MfuCQI2;2F*`GEkHU7gU%9rzliQpRFyC;x}}<^Q4;vXYq?BuAAMg z%#t5~Sr^ZQ_2PI<%1eq(Ty!vRVq=C^y$_Bhq8DOgm$28*FW}M}tnc}iBg<3L!ztyd zlh|0SF7Ys}a_cHC7Gzez}k*y0ZK<#ZoFP2n}0?g|=jw?4hgMylh&PW@@}F z8F6C7yKq?!oXRf31p0N&Zp{!bV~{^=#lRQoJHA*$ueBO2?`f=f{hXAc%T}z-?8Sg= z-dzS-VmB66(5@~=hXfh#2QcZ}hg}FK*6+C&Pw4Oe&&J!$XVd#GH!pzE(SPAeMi=Hcr>w2*9-a4?Kzgtm>T{*NU};v~B2nK{x=T%5G;D`f^9k zTbsQmkktYHOzgWxC5G5{^^CHRqy=0u9|NTTewYmEsGfB+>r0>>+x;whN~Solf|+9X zvtb@c7p2&?Ljt#LecwiFz~DGq)6~Kp$3Y=eyQm0B8D_X7!S4&1>{ImR4_|-8bUF+| zR|V&5ujOI5*0f%LZ8Pk;jOy8?gH^P%UD>{f(X-pQnewt~Xtpl&c{Db|JE&HT{QC7E z@P~crIJQ&J97De@9+EZlZyf^n7~gdmB6>GYO0W0UdH=mp;~wfoTLpbM$@K&_0O@j3 zPK9Inq#GN1DGDcuIkutjtYR2@mllndW?MVI;Kg7Q8#~{Q0Cx^!W4y3#1aMi6(H*?8yGWgzil_pwvdFF$@|(#phroAfW=ap%F+MkVP?FD7Jf;GjC|s zDG@`v;gt?gV4>1kw_wPTP49STEy}Xut0OcFMJ4oODARS7l?G)XiH2r8jRxhL6n!Y( zwa^Y;g~>ukxf$mqVL5Y3!0z5DXApl1{YSH~#4rpl-XCV_zMtEmWxNC(Yq62!feT0bZ2`ik8iIVrtd)=8QO z5iUW~3N;8Y7%%+68PYpfBoka!Au$hveyWEkxO@_HL7>Iu^$z2A z{ah;{QVf--{%VU0l^bFo%P4vj;=br9OR?AANV=w(v3(e}QTHd7`(l{gIUbiD`ygR& z7|Rz6>m#mgn6TyDm16%+P-PgQ5|20nO)zT$p)L0tVor>}hs}JQ9XGcxKmufFu@}_! zb2~vb5KphAO$0B+Bjoc%LeeCvye^)TDkRwBW)e%@+J1pEHm=zHm6G62X64+0ia$l8 zG#nbPYYpENh~nRW7=Mr*7=s%c!@rG-)-;ZdTnP_PRx_7H+xPR;RNFE1rQH-suQ#e7lD37$!ukus;Rv zYbjPg)_EPb^X#@*ZGFF;t8tDNV}9nxq**HQnJ#K@_&B9ui$$WHE3tbd6qS_OyjJ}k|dejnsaZ#Mv_UKUXqauxUU9s&ueXm0=8z5 z*Y&(M#1Bv6e#@pbGP2?>i-#Is19v21TV5aGuB*-#4GWl)dj)Tq-!43<=Du^iw(@z8 z(>01JSVX&7m}`<2Jm(qp@wh7WF>I=kzhLX?b%PiE0DxyY;y)=YH(zNQ0D~XB83m|v znfThOLen`jwoP-=9$Om)dE9q%7)8C+V+UiAH#MtWZF9)BKnrN^gE=WO-~Mi1Uc_yN z;91y_AEyftXNp->Vxwl7j}sa?B$-TkvXst~d;+hnl9A&ZRW@?+ zf?9#R{sxeu!=|tTGVx&@2CZ(L@ESUvh841+uF%mCR%j)dkhUUqjH50B+v1kjVSDF# z4z*$2mVNl$d-lO9Htf4EU4L6^^zv#GXFZJL4cvG3$+GmGeW!&OE>Dg-W?zzIP8~%B z>w*^B)-l@kBdxH{d*<>g#lD3?L|K?bIMC^cUU=V!>7^^|2^SODFqt<56ndG$DA=}2 zuNnG#9q+HhY`J2h|kU~nh^;Tedr_XoV89%3{pNsZhmW`G#?wh$jq`RUWM z&`p;SJ2jFVi{qWfFGYpDpZ&O;Xfpd@R;5N;VZD0%agqz(KvB>~BPtumYAr3Z+|1)RDlBrbANqt1a|RN+ z);N?n1<1mIllQxrzb6x0RmBk8tLnA-Hs^8yrxd8|0#2ZUZXpc5;?u8|XV5+j*bZHL z7F5na`NYD}X)|t9?mA0d^8i~QORBt6Zd260VCpMn?}PUp)3_svhRbj7t!gPAUPm;T zn`-C}l(Z$wRy;LC1C!v!@iZEu-9Cp0Qiyg%O9LO;+|8|6u$l4948Xn!4Z6)U!^2yNCglOZYiWXb=Bg$*cUo|A-ZP4(V5uUyz?D5 z>)jaVtfV6j5DAX$ClNbplpL_2)Wis~UKjy1(LK`@6mZmEWg59n|PnZnf)@T)IMRH4R4DlJ((itkT zCoZVO#x(QM%%1`4UF7qSLz3Nn2fSW$X03kQa!v*7twS;`HVTJc9;%cTIdcmy4n@T0 zD}=^#o`Ed=rwZ~Od?d0}=GT>EPax>-n-g{rjOD(MsPrOBd`@!NO9~!HCpL3 z4|(tMu?=zB%N6=Tp%nUE;-9czIgay`!o5uc(94YJ2q3DY0jKq1A*XDQ`M1QC<#^zn zJ0>KEgzYb3->D&!N-CNAo673v4< z7m)FFGb@e#u29Qm8G5KO3>RI9w#;q^1V=9Qf$xVnU1f-PS>t>qHa6qX;Rqm!)_a}A z;I^+cO~6U0u;ZImf^$c%jK}3SV^;Gn+|Yr(^V`2B=<90Cec*KCJ}gV|+eei-T`JHn zt)E9li31TkBm_}ujk)M9Icsp?aka$SPzQ6F25*hN?`_fKcuQrM%OO3izg5XXB7(hJ zdd6DO7sVK3iBVy-yucCUx!;gE^J(3*k#E~hwltyW`^LxGCv4eJlmx3mR$+YSpQ@`E zfQNNeo^`I$4ku$CZXIvX)m^^-fUd&nq9_a%(HJ}KVb=xxsK(fDpL>LNT8sC4985ad z)tK}+ce%ih-r5Z#w@D~ScLw)PTk#$$+AqX8Qda4(H4XZZbyZm(1N)|sdaw~CNFMKv z9AG1M$1gu+tDhNtFm*XLZwf89PjQ|l7J}f+IFVbJJnz~u^07N? zf$#W-Q&Ei?-+#ukRTJ)0Kn`P6X~qOSoaQm`4BxEs?15qj=GbnAtKr$Y_r5IDX?P~* z=GM7og=dFZxYnvYyNE0=wXpAL$#T&;II1#jvtS_s@A_nRAgaZnAa8Q#AF*^9pQvL% z*$csJvive_52|4dcEbAD7+}i@e6fBEP|SvS>c;@)zO{8jBCc(l$z3;&M{G!oBWA8( z3S-?Q-daJ9tSs3`IWVlw~ZDN7lpqF5nF;&Ljgks-$M1xk_qA-q3sxw6L`4wtbosXq{F-G5LsJK1{24Q`69EESp%KlaHjC=9E`L(4_F)w470v z!co@IxLA`t+Hqo0tFl~Jc9#1tE&EcPPF3SCnSbAKFzz-e;}}Icb&ZOG9Zl>3T^-Bt z;~$nz;En%-1!!0%Z;v@CiCy9=} zRb&i$&|w51lOA+U-|7+3bwOLOOoSKM8hI6gGg^NrQqx5a=$$->93Z_qIf0ZtBPX}^ zR6WYbDI@^<+KpaL4ymo3UY)I>9qS4kwFG|Zgky+xp4tNKwpJ!nwKDV41mL6o&+3jSP%HEAZ7&oEI8KIErz8I`^gJ=N@6nN!AfS;;J7 zp(UsoL+abs#92!^obt{3s3_DJov`89d_6g1hv+=E_|~J{R4B0fESJ)U#zk16?#K6HgZPvf`ZUjpw~LB zhD6mePdHAz!;Tk#oClL%F}H4cIMw|n*e@`C7YR$Asj%ZD4z$vm+dB=xFuq5H{V~Ra z)nJVtb|e`PY>lL@!?i0<9!jc~_aRma79t9ZI`eH3qyCOPwi&sG`W2zUtEp_~7%LTa z_QKHVDX+f?tvZtq_13X{2m7OJmjbSb9YcbrF~Kf}>k8YoKFlu$?CyfuV%|61T+rCg zqM9Y8vnujwXe>cGDR)T1DEcDfX%%sTGA1`~w;|10hxjgb$RCn+xt+@a;`#*SUG_hW z_z*T>=9WoXVe)v0<_0s!1DgkbxzwU_0dcrI74e6q#f12dmmtUtDgl1<~ngJh&3kE;p`cg;?YZzsJ|>}?KG z;~&4t$a`&5&Nb}2>yGT&KwzGue`My27*XAogx-)5&j8LRiS4lox@-FRB zC$0Eq75m4WnQ_?@jQKVvp*M73f6>sj&a6)ZigVI1i9DB+@UyPA1B$h>UEhWFG)Fzo z>A;SBy+39O=?%d%=yNs#yC`mdpf`JcIc$)^ul(90ujw#;kGunHHPJ~ouOJ=R#&pj|S%D+GavXJa7cShWItNDVN~ zLb~5dfn-FgS0FU7N$zOf9fl8PsVLBPDq5Sl*B zkR4BR1~eXO12i;h1GFIYC}n|l>&T8DSLG&uD3Sx1Ab|;J38o3?UHyIs^p`(by6#io zo!n|ga*t%ROXcZ`Ww)(Tb8&)p6uEz`EB!z`OGnKhp8OiJ;38|XDsN)M|) z7%QEW0)S_rNBs=xxkgF>FG2^sRj>DUi+hi(S~n}#$9dspeomaI1i8J?b7FN}kU>Tv zh-Y{MtPS44Nu@dtBRJe=9@e768<(HM+py~FhN(1BE!EMJje82!4fSO`$w@M_-qK;Z zzs;{jVi!R*JaDG?NN1lqR7zS&p?V=BZehMR-S-rGTLzH$U1CTyIv2nS)NhKU&RIKEc z*WZ9pRTVdMghR}s+W2Et9?2c8jRSnKuEx};D(<21j?q4}aV^uPnMbW%i45CDvR-gZ zMk~;MJUrNiW4bdIP|bLJGHxfMIo|yG^z$*qhDi7Z@isg-3dgi?_hChR5tT{YhnCl} z7)>T#R@Rm~)KxXRk>%zyw@IWDT%UctBC5Q%Z%OJ?`{u7N&g*A!N9%X{hG^Z2xQczB zwg+EWfJLh|?}|FNmUwafDWO<2(Z-8sJKh`>P1cP=h;w=BLWquqv>Pwg#65r?~GoWld7xOz4)*X8}g^qRa{ne^)bv5JLz+<_wmqU&i0#z z863H;q7_%zi>;ZXAgh~lF1?gw)(~}*=$jy;OgNTvtzYs2-2u?XrC~5s=;7LN1c15u z!4O03Di061w3W7oBLHwuR@F^(#~y%Nmu7%oOLS|*63-sz2XR)T4S2~}Gd4~hA=1d) zE9+e4UViuT3jUnSE4Irea-FNRmF)%~@FBPJeH48?-2em&XYtF25mZNr?=MA#+1Iok zCf*HoXU(BQ`*{(I9R~lef9>b zZDHI^%$!5N(^uWxm^hpnHuyAvqw4Ep(4%sVFV-YVEA)n@(c7@Vt`z7EMy{QnVXf`R z4&9EVk%$-O_RDBZSnU$UQ{N4s*2c3{2K7X+@cY8<1TCK=XPb?E#-Cf#LTuR4!CX#hs1SfTg`N-_Z&lfCMxStDJ6hi zhEuVW|O~gHWA2}RGYr6Ph@Uaxu!dW)GJxcfU zkCfMt5r7N#E=$VAX-0sS8u9x2S@J>Wvf!e@e#K>7?Sg)FrQIBQuBi`rpVPi9a;Nq^ zdmd4&Uc9_vxwk|(_h7*84BQJcIBTyQRu&!fZ=tI1X!8T@ixPUz_BW7#jA@Xt(#M4BJw5~GhIzlZfm~cO*{?4E2 zWYs;KH-LfZsK1ldN2xCxY4xz9q>Fn->ceCd>L0o*Y z2DYe-!H47rN7G;*xe?PiS&(K`fK+mylx+bA6lZ{AOjNzcTdHz@vQ<8~t)o0N_ZhgM z_V@-P<)zcq!i~>umF5_3U3JY#gLH7LIK-Z$u+%-yyPyI<-)F!0>n8iv1r;{ooGJ^Y z-h&D$wEGF~(+M#yD@w=F1ha!WX$vZ5C$rt47R^%Q&maaNc_fOpXiBdS=ACfPv#fsugqW{zN{@>-tO#I-oOSJmQDV_#A4zYObG?!*CgD z?Yaf(pnaVPw`9c<4A_O;xl}O(xz6BT$L+pisH@33htm-FivOSCMxY>_>s~c)%m#%N zxVQP2zVWbcT5PB3gaPVheiNpRaM_eLG-Fv?7Gdu&RMMhRD#4cLI}ODXC>nZ6L>>Nu zjk2}t^5pu&wbY^{W+zQUj%-~zR?rl4Eiq0}$9jG@W^IvmR$|RqY3uSCeM5GevMv|FAL3Rhw&ZO5>Ap4{OOKaC>&$J_p%R z*kveO=_E6<-7t0HnBXAfIGN&>q$67LrmU?K*Ttb5GgdiM&&__VG+-S*#kw&f8~k5h zDK?`4%z81!>+0}_q&rll1UAZfy@s(sp!N+!&2yK^Ak1Il&TGRyS6;+sYq5 z_ilLnd+!D^o@WTJ^aVwpm%EtlK0yGnq|Se&>WIY=`2EA_v2^Hg_8rIR19YJAu!J+A zEaqNXwOEg?8vC!{;_y5ho?*Z2Bc1_u9nZkm5A#e~xM{5t?jf0%qqcW(>*sc9=Qe=S z`L-d<<6y+3W5n$cZE<1dY5Rk=rhKK*c<06-iz>uupTz7JE^D2bY1fEnWOPN!&<=M) zA6(ZWj7ja?LViOKp%u_L2rVv(ILqt2AEz2s+YsH{6vsWdwr$PS3kx2wZpQELjClt? zusxH~ofp&9aAgnZ{lNrk4Od#XX2bF)2LO>8zE~rrH;S7EKt_05KXxRg+F~NZHn1hy z!$gpI1Wm};xFQi_kF_+|mfy`*_wyb&oAP12{~7q}7u|<&x|2=>dr?*!sNmx(bl^V| z!U14_2=3^?cTG+9>%i>v#lU`@3lm{I4!Kpgy^PU!^xkG-a-QzqSK*yEzDl#B9dlkd zCWYkSY`YKFGr#5JjpD0LyARomjA##?6j}Oe>Z;C5Md~8?>X_Yml~)POiP;1j7i1yk zYrGm=9)mY~B8%s)^>{h>PRwkzw5K|;3l_fmeT8H@Cx+7LM^4_@S$K{Z=HfDg8`!X7 z4asm1IfPWEy%SqMCv!L@SlT)<)8rUQb5YWYWZtmAu!jn#-Nm%1{jkKU?nyG}oOXEV zd)lq@^xrO+cjkHJOnNar@6eC+$8IEt8#DlSorr#n|w+Uvv>6y3Qg-ZC0%5#B<4P~=b)mP&SbRgqZRhmBrvT;j{6=XxX z1@}PnZJ@N3QEiCoqT4dJT5k$*K3tSg-uOw>;>c#tr5!FwGQO*c32wNcGq`ub5uR3s z`&G`b;_90xSI{=)1sH^9(1xz)oz{xBVNwS7Fio~a`(aUu>zMml)AKR$os!w@g0DOW z8*hAuwTxhtg34aPF-dVyPLbYAlLO#uRIndV@Ed3lCX;^1p*QKrE(f?oZPwPpBfibq zVuz)h&Nv=ou$&>stV}sI*qq~6!h1E6`_ahGJXBU0G$Qc&8MT~tU_qZxsWrt|6Xto>%H=Yyy2W6|6$(&mmE(X))f4W#9@T3N z^8KWb1ajc9iQ}?@qUZR{e=3L{3W|PWj9PVUVdgPkteKHH_eVP!tHo9eXWvwOn}Es9 z-tHq|iSTJ4j5q{GZGV<~?Y!ysd$kZQ2~%M*7P+q$o=k<=Oj^@n{?%EjO=@aQ11sFz z=XpGMl^bD`#3Cg^^;l~B3oAS0)vCewwyaxm^m^DiZ-~C|U{BbxEChn$8fM7=*R&fS z?!7#?E<;;FNu$&uk&`qC72;){=}53M;*gP%jcz^8*?ei18!vE%ztkkfcS$8u4V&6H z!t}s<;a7CtYiHRHcrW0HA}vNxkN>UUud(d)b2}|Pv}{wdyCC6f<7)JSI!gS}3P$Wq zIrW6aIP@r7?D0F(?ZK9?PriEl8Q)s0|4>*pk{ihOPfrCsA+<<-sHHtA}uF?Kg}f5{6CT4?V}Z0BfRg?v^gpZT9g1 zKBUG21m$4+vz*`46B=GXhcI;G!wPvLG``qGXhWuTLmw7)jId$D-pr0!9Y$@^_#$=e z5J!(%byC}H=69shZX~r+OZ@h|TFP|eTrHU+myF6L!8e58)e@OJq91&Xl4PB#C4Z3= zJXA}VY3%Sw;7HA-sk1p41I*d}VHK67fXz5y1rlZ)jN@R4f}R@(sG(u`3$R8$CbelF^P@J4&uv4` zt<%z~DE4}o0-H5i8B89Tr?ReFoPB|xIoHw)-fE|%x&{I6;@}`TXx}EZ)!YhJrQiI4O##?^#TT#cI+9+dgzNs?_N0 z?ny-0=3Hpiok|~_J{q{MG zml8)Dl}>csS5}sg0-f7aX9X?iBEs_h)Dl@$#oV3Twy%km`@-Gyix_5jKjPdV9UJbu zzsjWJqeCEJ)>HpBS)-x;Is~VM6m8zS)LD_(ov5m%Y#qV-%3@x2iRm)9I$`?WzwH!; zy-;-Ln2#O=A$1CaZ~idTS2jzfFv6^lAZ`kSH&B><7%`jVCe4^XjI9^5KJt49%fp8; zEgf{%>MOK6M~C;ncXVt1sCqh0`=P_Q{qorX)8AC(-P55KhsiHcPw{kDcan0~osbS4 z9uMNU>ZfUnmIA$QeLr@{{Kp#mNC#!oxY(|2BBEbNGL1z@$!5<})w^K~&ZsBFN}U+k z8@TQAqe*TzAdsE_b3iU1puTjV&yQ)j!^(k-t zIv#s;79>d=IXsTapwEU&+aIn4lM{R2dA2>~u3WHiR~bkRq69jV3r(FxDD^C z`Ys$ye#C5ifevPT&66-=VD&J|NZQ2g+onef%tzsaZpO>@VzRj2oQFZY*ew^!rr2W$ z6t0kZ=ftGgs&ErbbfKhZq&05n@Wy2o@5ht`nChJvC=XhLNP1eskKvwzw}IBWNR6Q( zsbG0jmeJD0{;IR5Q8AcxMryo@N@{$wit;N#4>PX5IWJW@44mFc$72{n0uXB(Ic>!F zR^(JLKH{`iKj0Ug5HVR9(u7A#i<3(y7KeSwoTeHoJB13KXCwgN40`Zz1N|QRSLdjQ z0sme{@F&5?M9%^E@}804+k9l}Dz(9|&h@IX;aN}G-WtYmwZbzc#Xma3c^8Dq zDLPe2u~RQOnq)@ zGAmp*L&c^po%!Rr;nL z7*maUtc;eCu)=5qyjtOioM~uKARFtfMuoC&ow*4nY*steLom14Vg<9E)1{-)x?P%{~GWn#ZG1)85z_h>ve0=P zl3_Y1w!_B5MoB|yMKzg%k-45j6YyNUCptsRPUX}~Kz*2rgc5gGL2!&t9OHca#0#=(d+qGGV( zx>ak_Ti0E3tWZq`mUki_+`wutA%3N1gQwdd4)Il;7$X!qDr99sJGF@q=n#LB-Oy3| zqMqHi%`Elo#?%}}<2vH{AZ;0w&h~N0QlAn!c8QgDSf{F^`8Kr`c<*}|U8{Mjj^5W_ zzvZ@A+D|{rbMW3EewWd;vrX>mp~&dTVjCtgI95*{1!T-fX)g}tlvgAkee6Tm**K=t zCPl+$%Nv_!9$ijSDkkOKRTY)GcY8zQOfH^ZrP-ck$14}%jdUc#W!0=7C|a%EoQA?6 z?ty?Z$km)PHZp{H3=*&D0%U)7=5d~$FqsG`O43lD@GE7rFK`STrnwBZR zj$)Y4(yZq?Z8gMB3Haf|s)>U|-oerb)JdboiR(ZFVvHBR^}HdAmy|nQWqXu#7N)vl zyTutuf{sbic;n8 zLrG7L>z;n==XT9Qhq0nw@!E_J=iRr?ARC4o<= zNXQEl;37of4efWigFi(BptSb;sPO_9WmWsY0dO%Ubk9N>nH7c!Tf6zZBXIi~u}O_|ag zXj<8xVI!tUZhK;BJudDUs(W&}YOSDL7ehT8{idn0wQkD7sl`z4(v@BR5Uk_O@TC(} z!bxdTYH)Mb<8JUf&m@UA$neD`s11uY*3azt$xwp@4 zJ-(qhqQiWS!zx5yR@vL$+;Qx9!m;gUf}Rey9OJAL_Ed|or`4cltA!xrl;D9x*X+oLW1IWG>wy z)5l#q-picx=G=^Nu6zL%c4&~wdu&AZie+NBuID`lS10aJu4jP1btM`V2YRLl9-o6- zxdOZ?#$_H73P;vrDvO4F-Bm;2+GVdj8GMWRl@wM+=?dmDuuw54uk5gQ4uhe?9F%(w zbHmgEv<96A(nR;+-TOTS^P_1To#3#{fwGSa8lx+ofTm!lage1#6< zU1cagVun5*%wA%xm=kt`2kj_=kB7?4XcF%u$=s^LqW9Tw0*;`ba&s71w}O zX}cg98JIUT&cyPzim{Bo?r|74zeM*7!Gd%EisO=nu2rNdbFpojx%2iaxgSQ_s3a0l6SfE%vX8DW35xPLrdrz>m+U=)V&58p&^ZB>BMesm|*G`(Ck^{dyWDUGK@Z zK#lWS@9nn!{D6JWJ5}o(#&QLbHK=OSwtO>~k&ImI{!}t}Q%fk4nKtB*A ztYnqmUJo=g9omY|p|5Kjk{L*@3l?UcX7*$!lvWLNEx@i_DdnY0N2{&)iE)-T1_r%N&gVeEw|CF2#&%WUpUTI5?O(y~9PnAwcF2OUoMmEQn;APloH zk>0R1{0)6Pkb?eE3aD>7&+Cy8S}vUknz0pSp0%@^_|v$=HNmL_2@H6zIlCYNG&l|- zG;{T6xp*a?nbfOqTK*nHVD=YVM*tNih&X8n;as|*!`V@PaF?K|)6#87G7f1UExENc zepwDyK%yXTh8)W;u)@P}>L>_VRyL)xR9Qsayx~opczNkfE0x}LSdB5RG>J=Rtx!>x z{Z$>nDrkmXD(@mJb>8SI6(5*4TH87ei&1Z_oASB$LU6mc6@o)Yr!7bLkvt@8CUuoE zPk7_J!t@Ta!QEXkeGo~-TX#VA@q7iSb;s%xt}J(;^hUar(~1R&X2)Mqws~nQvBG}WZ0Drk;e`V3z#a_p?9(IvlRb{qOfK` zn~zBZzV%t3@XTsmYpJJcsM}>uy>J}6ZuT4MVr=i&xN&3B^PGgUt3KZs)pfGUUXG@L z4K&1M)!-jXBT#dJ@|YqjLp=s#b*t4VG`u9MHt*+n$%r3&Y<8w<@NMc-wS~$ctMXya za0VKx`;wc?tJaRV6?-eo>u(%X&D{_#Jc%=o$JPt$ilsSn)zl;o!v$T^#!{P=!OU2Z z&Al5gT|6mt{)YF~#ASWy-o#}nr&_osQYD1(m)Fk@68$%L-$T{+ekH}kkLP&#i}X2% zk*e$4V_z+OipD)lgg82=;XV2Z^K#~4+OMYf?nqSAz2H7}mMiT=Wz?%&>|xIT4TQoM ztA_i4|5*3akuw1r!(H#mht&9w+vm3N=gx2qyQ0=fR!&$?gf)VN2x8;}Hg$?BkFeGm z`$t?`Mknp=L@akkEYoTga)OOgk1&U=>7paB@;S^6F(`0_^O6R#kB)BRfnPopOKr+Y z2JBlfQjcY0D!Rf8mKY4DB=^I9%$YNJ+NRv?gn_ih6Ue2JDQOlisEA|U@nGGvpdBMPQJmNFRqhJe5(eoU_a)7*?zf+Ju zImqzaTkZ~6w0LI zJ{b7S2q1w4%El2mlJ@Ng9HnF00idik9X+Ai|4@(2AZa`ALtL8+(l}@O7JdW(CQ3&%StR&UT-tzRpl_eV?iG{l1P@GD8;);US-sn0+`wwB zh_=%}Er(9-;4dxCc$xVyBm1LOC;Cu7>%Nmwz*sdL`FOm9-L9?Cnsa+d(}DH=e63Usqj*?9 zIedVMH#T^dRYF{UW1CXFGUZ{<*z^rIe1i#Z!>sr1b6fp9#M{&*S9<>b5IRk zhAg331@J5#yV7c~S5mk-6?GhmfgP>QHA8@NiRBL{DbPc!hCU|6J=6v9qZ+TZe(te@ z2z52XhAe~cW{sGes`FD4`%Quh1Y^^}94nvMG)C8&Judh%X|GCNCOwEc0>pk5*w$Sd zk+z%35TW7Ut2*=82w-LZ!3Y>4v~_f(J}Gg7!Ixp=hMEwb<4pNM zUlsnH1UX#p;R>7AgAWt>O%%vbm$-2p1EW`E2pj|V}b3ZiNRnT3HmR~=|6q$L$C37tqw;JYhU%pU=LOOoz zn7O&R`Czys4vvnUs9k)d(1N!2<8Ae zhZ(}e!HkdJSfW1RFA4Q%@#B9 z`oxPC4`oWzGPU#I#Pe+Bc2sFian$qbg8A;f)s9!#p06M6b|e^>bRjiIp4n8d5Y5R(;9s%dt-&QHoF%%fg2Qr8ZKx(}Q(BSSnB&I<;;3b0i9 zh1Zo08VezGv8UZ!mTxCE3X{sxO}&m4)19eSM3GG$xA`*qf71RqF#wa;4?0ff z8|oih$9p}s>Xxm$Gyy#fLw#?~8jbpI%!S_uB6Xs9Da~}qqp%z7tgf@(jxupeJ@Wun#UCF$}ZYkI$IG`DWCMJcW=Gb4b`?3yx~X93a9uxSBVhT60R6qP>bv4+#+ zu*Q$eSm%J?;sVSQPu!R>?Dd*LQE`pg8Fr|sT;3!>lGcoaVy=o(OB-ZPw4oH2wl!br z?6V6(_w07C9HCAVDnig5;&<7tIBn%Q8pM>n55qMqOKzp~3cCv&2WO;CQRhSG*}GWbF3!rDc~TKR1_jPYN8OogvDv6O zE~d`#TyVSgbIbE4-=1obcN6NvhqdzIR93*gGwi_!S-)upxlt8exQ_$uA($NYKu1=U zd)Pm7t-PuLUl=0KaVS52Sqc}faNMknwo9QRX{TqEixvskzTTmgJzB%&CAKT%6UcO~ z5gl7gG0#Kii33}ifwX)7@IR|Ucdhu{O_2B&bQD}e9T7(kOR(^ zLD^s1GLS{W6?b5WMT;hDUW&2h_1Z4eytQSPSf#%>Ew+ZXqaBPx8#Sq^4|RjWi5gj1 zt8c~Rj}p|1m6SCl|0LS=OrZ@!kd7r9tu0v-SD6x z9P0S^9n_IufDx%sr$~`U9bp4#AfdkHI$!RU_<3TAx|oV~>@u@hiaBll!kNr=2`fKM z<~lZt{@UU>2L23rJf)00w{q)Z@$mM_?c01;WDHj&Q51r$d?$|}A%DboZvJP&ca?TL z`Ry{4ec%ss+-Nu+Cab1PxPviguN^-N#D@y zbxk!(0mDhE7F*Tuvxn)MvMRKqt@>0%ty9-Fs^l%r?R0k~z)vDUjt$74p9m(#NH$v= zpU7^?avz^ZwXzB~g$Mh!U;@Rb@G<;(UIRxM!CbHXZlk@vM7_CKu{-wmfDO9qNhPOK zsVliKqPpahxJW=pO^Z4b0G6Cj!kyoN7%>*^poAEKB@Z!t+14i=DI*&Rgyv+4` zXuEznB)*r1c~fL@P-`xo+}@d4Ys|W4j8QiVqjkj+wPrY@1F~-UP9$p2YqS|fR?VjK zp={$<8XoS9oLkNNG>=yE6ixzo$_%_$BLFwc&j2tcxW*g#dW~LN^396mKbIYB7T;s$^tYve6=h$=a7g?(d){aee$o{LdRtiDI$M0Z`gIgdYjJ2UOMKySr4F0sfx(v3=kVxMV+u z-8F}2=Kf<~f9~txVaJIwDvD$0&5q(nBFj#AyB;mEXM48wY8lfcnd&|yRo`jsmgg4$ z2U;7i1_sH1+Xl+S4|quBR$!r=Y z&8Q;jiUHtJ&qiT&7}6*}gJc6RAD?Imc3bFi=XZo2juS@Cj{=N?l#(3<7zZr!iUF`i z3{{B3qxG96Mkl!Jt2a-(rD0tu!Dp|g0fuEz@r={nyQ}<d zD{B*Ax6F6ICe!_4h=ZQZq@_&YhTFhg5QA~+{Kfq zc&*qv_Vk%>ACjnCnRs`jR#`4`PH3vITu}_X-WiqGN#kp*OK&7&YasqQ5^Qjs!xr~> z4_mch!$B&k%#=JGY6csXluqZ&Wjoux77kt=mD!M?ze40^Y+$mBbivryL?-Ely!5~( zGVKS>GEg1lfW=+>wiM`|tUU;a4CgS+Epf7Y7%wA^pdB483`7+fy%V&BkJAKtJ)6;d z+8yfIMNhUUE&@BV1|28%Y7^QkmP0tBB>)aRYwb$jJ-3OAFYu7YQ;LgeHnZWOaeUaB zB9XbX@f(TEwG5o*PK=GORm^J{$bQbXUOU~Ez`3iYM^~_^jfOhx3deUdca2R>V`13w znW+N^8De^P8DjGjJ8TYQR1ur)yjBi%yO`IZH8s`LV}(&M^>}^{zbwhc6 za3geH_nOG*?3zju&!-*$;tcN9`@JeFlgmNg@<>_ncATi!aitt`%Ols=IolT_NqC$fat)avSkK4*bmoftF2!0W!p zX_^)*IZXs%RT|QR?@2&!BQZ{;^L%icP@vw-EwHW1@tx2!V-eeRVb=UH4coYv06JO0 z$CJru)3%Zg+K%ZQRe%6iPC8b8XTjVv=12s79FB_W1bgbHBYhc zi!!oj!$Nn8#x$a$NIVTI&T%Gk?@}7$$|(iAuo#HR$XhODG4L@{}=!#ks|~IZvW( ze8Z}H&$gnj7Oh!%TG-l&Yui~cnMGqQsYV?wqdc;Tb}q(RLPCy(@&0FKPopS_>|SGW z1lU28rU--~Co=bLC2^>U0v?3n9$EaFKVXdh=bFU5t;F)nZboQ#daX60XzOGdroz6| zB=ky}j1)?Om`=Xg*f$P;cBB(=qy&!2kG%sjEEu+m;b%Z5>sD=6UB9;r7!(H1V{OJ( zW2Ir6h$>Iqa*6C(GnO=JGt+IZ&3R$Uua?cM?l6;HOBcv3P?6luv;oDFa?+glbaBU_ zeb&d;t()4i%z8gXKK9yR_N%x0lrK}GM_Mz^$zYgSUoJ~yIT693TX!umF%+ImHt-ype|;JTTjjZjkw8F7vc$$v z&n37+1LtBW->c_JxBayBbeoM(cg_Pektl{I`ug|Rsv*NL9M)&p&WCvM)IZ(>;W;Qq za(Qe0(4~yPF~u_9%hKHJ11$(Nw+uVcG;^O%RkC!t-Y-3d=EYlXSOhV;%_44L>w#V0 z^!Y6o!P;dO-3U>z;t!R=FER>-Ys#%3&+ox_Bo-acE;&lIg*zKw)_GCUqedyW?cT5| z7_;3J^&FZ0wfOJ?{G>6G1Gzurq@aZ{Vo>W%_#p#4{NOr0_uhzC_^HW>1phTJ-J_;) zm=oXbGj=D}cHTpN>Wtiol4W#le6n%R4>V}{pwD@7L#_S+=z_q!M6TpkoX>!3Px<@> zx&k{9EhEUXjoVSp;V*y->y*Le{EtC3Mszc_7b0%Kfs2o2>gLz|&r<;)#rLWEq_qU@ zt=MPC+QjsOI>rH)VWH(q(TlXMouR3XG@fP?i{vwb_T%&w=o1;A{z$oZ4ALTle|z_> zD)4+;wyB#T4?f%^q*y6D9JZkuIS(l`qsKpw&1_-dFbsj8_(uLUHq+N5 z6Yj-k=+1y&K3wz)tE6zlCXa)~^EDv?JD>Hb_G)*=aczDy>lON_M@g(FWDs7b>=N?<~p4gX0e-EbvqSNHrmM;oAuySUX!y0hP%)LAe;#>21yoR8}BWvfo&Nm zZl=$_Nh#016eIAy_fv)z*r*Pac{yOt$y1B~n_3l?We;BKWW$&=`kdFEKbFH_r^1e` zrcLHtpxoI-mm7)s3J*w=#Iq5gH<3bTuC0yp5C?5-VB(x0V{CsNe}LymB!WP^AS-h( z>-R{kB^x+etYHfd1hx$oZEJ_M7%OswjmYcC6lLC}X};vMoKsZ6f58CAlqL8nAeRlm zz=kRu+(LTBhJU!g!4JE52mYhON}e*q7{C*L{IUf9VFaS`hm-Jgq=U36tn48l^cZRG zuuO%E+A&uPdMpr6S>X=dwJao|cM_QFgTY;7Vj|1@`9UcPekgE(8;YGb$brzsOwcnD z2c(Q8JiP(#&I|><2QRXH10zMu=)AnLdL&Y{S)?qMc7wGw0=#=om4tuGZKFGFzAC*&%4nu9RuaK z+O%X*k`fty-cOF@uB;K_&1Eui8}xs5lKFe)`?bN0F<&YxSXawf^Tqde+eGf0?=bXT z1s#q1uFlF_IHLr!%5z;XUNBw8+rpP*Zcr!AdJ*c$)_Y4X32)q( z#~XnIJvWzg`&zk_Jl8ho#?~y2dFxhHkT2eXQv|F^s{&)1rp3)r%~zXYek~DqA+MS;aaMZLddW$T@&0`}EA!O$;wC-kzx^7_mC@5ehK{(-TB zLy~v8+B6KExbXXjZmKhJ2o7p6-E0(PX3t5Rrh(-rPLft**DkG@6~bS%(TKw$m|ED) z3kn+_MsIKu5bb=}?S*h+!6i+-MLmR)lviY^w?_PH1qVkx`X1g%GV!gZvJ)`F$-FH0 zT+57{5UpLOu#B(@?idMLx*B@pi`c0Ox2VLmR0O%86YR+^NHa0osV|$oX?;G>zYd@C z@=8o(){yY`zc%|d#HY>cJod^;k9FFxI}vA#J?qb~r@@G^yVO86dTcJ{wS79jabH}r zM@I);b+QM)!XXT9{u<%uLN1zRvBF;Ph>ROMAtNkP8;t!}Gi=#GINE%@7S9{)%x_MJYLg(XQnJ)Xt=#1I^7 zY3l2p89+mu3Vpfat`C;1w4)%+^4bl$*`NCYox$T!o?Xf!3dKINOubX4#F3E#T^`Rap$nXs_=<7e>?*VUQ)3Dcm<-Oyv8MV|98vX6^t@^^!uArqNDx zU}S3KgW=?*i@KZf(hSpq7QGVg(hSo;mnY}(L91&!#yyi5TPJ>6SY)^W<(P4^h?=~p zjjSKtuI*v?L?VR;PD547o$YU#W@ zTn=6PpGC{#Gw&2p<(uLM{R5H=EpY(}4LC10s-#7944OpCqdY4YyoY(+M@}{PdGA(v zh6BhE%P+H*#EtM!;pK<1T{LVl(l)NY;C^-v9~>!U;b5N7kbcxwQGhcaH6COm(VK2G-$#I-3%$G!ocwbLLUbF@*fEq zRi=QP$Vp_i>dNaf8{_w})kz%qeXx-8`|o9DzEoEQy`sbpX2GoaDioOwuU|CTt-9&1YKIb9FSa7_? zNXU@ilN7jA{2~uIGGLHxUQ!_G7%BNE z6zj!+qg`r8gfmWUN->fz9!j1mti2vi3>ajW$wWdc8?f>?iR5X6 z^T}><0LtNvJIgrV5XvW(yl=v_XI#bz<@0+GqC=dycvmZiVoA^!e>S^$oN?eCI{#R? zh3SMj?hOi3x*<-n5~Gtu7m{uTBf+Yqqv31I~pA*DGs)A&;tM&`W%Q{T+{}H zNLceeYK5NmHu2CiVegTTVxt1#q1lpyV;o1bDMxMf<3B5pDzMLKz6-(bmy^K*a#(|; zGS>HK4$LsRyqr9*DY3@jOEg!+xvZ*KC?s>5)F_+S(fxiv5N3{kafRCCZBE2{7kyDE zxe*l<4;)1T0EI*Q9r1p$7mu$(oBWarZMt>^ZFW}S_@ib`rQuAj?)h~pbh$cGkwBzn z|JR^^=9_X*z-$3E^oO8xOekRca*IDf0re$VR}1cC5V1K1zkC=C1htVDx|Z0w<~3N3 zqGhKdwmfu#F*{9nHFUd8%W0+?{V{Qs!vbi|8L?n&vSe~0CQg<>HIjFtOko1+32Ljoii;8vqf}^URz$$o7m<+=KB--%dx_h@_E8TGJjeWk?r3Qmj3vl&%eQt z)fKn6q^|d3eiX*w#tii^0BBB4etfU4p1z zM@)Pr#1+DOWn1blgrJ3b8G6(B!~qcWIS)O2BM1F%YI_TMYM_R}Rza`WmWp9P=XO|( z6sIYyo}F$*Vd%}m(7M&*|8QD?@UM~wTs%|6vJ0*03ad#E?}iimGmNaq!#RqK zA(R}ccdqOfoq331=1&MmFMw0BKY$QAR^ZW&SD$(uIOJlb{#hp=gD6Yk>*eaZHum-!qX}h*V zZFXMcG-9_Lr!i?9?9N4G@_JtVPZo!|&+N}~xVp6CI20EhCUZ=;Pp3L*3p5AmwX*An zNO*l}&7v>zs2T&WZRPpFCOEr5H+qQk;LH{W!I^??AcRb}8&lu4cNEp=Gh&Ksm^09G z@9FmZ-d0QxgEN+I=`a^d71WE<^_5lTb*5X1*NvfG`EeUqfo1fA&#ox$be4kJ6OV=$ zl?MSI$Wa7Vb`&ghP^lGewCGp=@o81R7=v>xo31$J_~;|dp$LC zvrSr@0_hKN+9UsE^o16oen87nOFvBMP%n|dK~54f8^wiBIb+^VaVZv;nTLWH@rwfx z-wCP!JXR$zBn-1~MI3*N~Bn_my9fE+sio_Kx_Z*W;x*~Rp@bj&ny zMJI&F*fOuJvXY`n%1X~CD=QPM8rCPa-ATJif0T7y41kqrm8Pcc}V*lYqWUWM$c3Uc$#$TV0$1i63?>K27sVvxB*T57jn2aE@Ug)wzHnLP zjwo7u+NrXm+D;<589sLZt7rK1b{yOoo_FA;{Hh9WFqZ(>yQAk&Y~fY}0IIECMHuvc z5PEImPgWmS$*A=m_LW#x-k07aA7QA19=M)C4-X(m0>m%bI~m5Oi=_y<*<00&_m~br zftTT#4ktu9%0ka{vOS#b^6D~cQ}09(+V(6sIn#6;+tXq!Kwe=yAU#8U^#ZSREv|o( ztsj*k;a51q@8Yj}s3YRVIP<7uO2wzNR(-h|@u~=4m1(r#6xdm_+e*`mbXv?BJJ7NQ zyLz#E*Y)N_(C`1@J-=wbixj|{*GPfDJ`EJyPy3-bfgZ2-o7yCU9NY06tWQwWS?Bf; zdXA2bWr1g+QI3sCm9F)s6?>VUX`WHLd8&I}VC2bYA5FHdEYCofWP=Ahvc20bDm&L? zIBV|kU>xz@R9cujAB?VQswN)I^EZy|7v|pPS=Po~)^+1jSnhOfe>f@PiD%md65PJh zhjY-W>vHb;o}oWwk~eh7D?PVLUbn%&!n5(hI;E}K)bZrhP8t~2Y*%@|%)&zM0o?+f zCD&``QWSTE>tujZDwn<_lO0WR!FH-|=A9cZaBPR?e>1L7vK^M7L?27-_?{FkA5C+) zWMXg4W{8HF-IQedsFzk$hgnj2(<%(x{MK)Bw|GJsb$aFv>gJ~l&f{|ybrt9F_wS$% zUgITs&ObA>Jm;U`hJ^Z|#;}oer$?RJyEyv1z|oIh=J|Hj_8OX&2F-j8dm`zH-s}Sl zxJ(&)90W-%O0JvWBl9L(=V#9r=UZHJ%xc1_S^X(yI$cK;au2hr>Cwj{6++e zt+^vHb4&4U`0xPCui*ocaT7mqDO}UEsNo`n;TzVUL<8||rJj^%xVkJRsGZAk93i@~ zFq=-Z@}^H>f%bq_)4NE4%d35&&$T+?gO}n^hPAwdGB}NA<2j9MBn|sG z?TIpUOF~%@r?K*h?1NjnZr)Ez;rfF&N4}$ni#`ON<`~5otblE$bPN2i&VA%^KNe1Z zQC4@Tx>vo4JUGmItS7p-#v_lpA}sOO$d3i)z(!$KMo`vC7>aJvA~O-L7zc0iA{O|a z55v3P_hGi@-*Ypsc{L@6wPPpKFwQYohVSN%zuOvT4~p z{i?c_ORgL?r@kD2-cwA>Re>#U%Sh0IH9Vtm4H70X>M;J~By*XZw)VRzFZH4@3u~|& zD<7AIU3=Nt@p({y2b6<;rs`7{rgb2oR|C(xL4hLhgc(1R4m{{|c?0?`@KmfKYH{J1 zeV1j{TrE9PH<95+9fgs*;B$*Ke)25*S_R-WcZd(`rEOvX{0BB*AN$9AL#xd^>67^y4{TQuf$tX-_k z6nZlH_7Hn>?uR(?KAr0^G!K=PW-ceQOE5<^a?V#DuhW2K9QlBtOuie+3Z$R;S%!SL z29D8l@1vyW_im9-k<-$P($|Ir`;?}6uuILf3tbm?LK+HjeaK z-{5@Lm8aX_bZ7AR=~sg%_xts4bz>9p`Dx2O>?cGb4z7+`3Xx6>-d)>r92XOKH4gUZ zy2u`C9I)baP3lx`m6_O@6wJ#n4xUZq>rn z=DhX7dKn6+)eG3F+5T@#oQqwBg?+a#i6$9!3>=3#hWrhM1+uiwh&6E_tJp@43F^}a zN+!Myqu6w*rRTL7%&`bfyX_aFG|H&&M0Kg$l^A*A%E-@3O9K?W$n<{bhf<2si!SBv?8I)}HI_a% zC1yxSAD0fIs#K2}%kT3pl&oBq)ZZv7)2ACpsU5j*1?NCr*|u-EZvc&ralCEg8VWLb z&aK}gqmejwQ&Iaq()1iB1-g;j*v>pj2rJh2DM2A|4G(edxj!9R)^_ot-_GjCb0b1z zI+C4uZo?s54w=^Fc5Ha2kKVCOQlzM;&dk(`j787W#w$ZF@dq?l^igA-n6Y-WapgC` zIajmhV4IYBW(eEiBn?_A)9XfXju4LmMAmgs=fL$EBWdgRb~U}1Tt|!MK|vY#&cLy0 zJL6tgY)+iTKJdm1$nmEK^sB>Y|Q=KPTT)9Vm^{+@L9$DrWNu*4$GwD_Frnq-w&rD5-jgwz2Y2EV2o)JAi_mWEXd^)``KToGOofRPAJJ&<&V};xa zhgug}Ka>Wl>m;7Z^Ov<2>m&@?+48=5qNy9GTBgsxB}@*36t?`1bACdJUu5WoYtNkz z487Lx(Z&?XTPWzo<|u5e^2|lK7%*p4f<(DlqD0)w%g(M#32RkEgkL{%&^w!^Jd7-$ zC{HKQU#2!P=!#h*eq6Tc8xaw}gSW+vdC+kUL&}4Wg(5tu1p1+X@|g;AL7q39Bgg3G z+MKRAWnlR_aU^??oiby@rj7*-OFBvS$BNg~H@OkXiceR?4LT0xc77MeRgBYM13=Q6}YR&Gd^Nfla4V3ZMtOB0!zVI~+t8neP zIe}pnt|3nW#!0L?sw(1w?ZQlA$1Z{(P0As|0m7b>3t{!Ct!2e*jjlCMY+H@ploC&J zHUK-b=4R{S69-|wgoCnK11B!>#^1{WkOP;CWnBOg=E=uJ>oxUK8rE^~PL~Hjm1EC^ z9p{;yM5b`GD4pj^I(i{CB0Z8OI^4_>5%|H(QlX>xSO)xtjsjG~0Weq<<4I$A#-n&) z8yg+!C_B|jw8<^(Olt~58ysK%v(oDr5E*T*^4`E~N1coUC>*`&vbG`TGvNOD7nu{xp zCa#?^nkI-6++20p1E(0%=PaX1XE?a=u`DO|#9MhhA6ScP7-JspXE)Q!a>7;gRXB6b z4$=xOv4q)J6^cX?)9TtuggW09XqTh{ShwS0%fMvk!+D8He!^j_qrjPg!1u!8hl7CQ zI2@krekwhqv^|8rcA8@cQ*H-A;1-=>p;v9_SfN*m3Y*m{-)ZxxYpwbC^gVik+d(qe zBI_u+P$4_IiZZ*I2EW3w?j0Qk%9NsFjU3nD;4rKV>(=k>R#c9HWR<3ti5+!AZC9le z&HupGtmJVwaMQ6PAM`nIg{Xg3Op|AgaQa9maK21W$_R7F)MKDzYM+a+Ks3%cX#0k1OlzpD`W181xFvM{Y z#O54S^E{%3PwO+cN*}_G$hXcB;s(;sGQ_fp3n6o3lnj6WYra8| za4r}uoZ%Y=1eC)y-eY4WYO7@1nGc(qrgB(cWHAoAaRzw17b8}<31Z6$x+E)-mmt?C zqx*PRa(S$rvcivW;O^!;c;c%ms}(@X{i(oJuFYdmcKd1+Zh}h}nrS^(xhtC3O-!fm z63beq%8jbZ>;w*F51q1?309P|$P8xEQ(yVr1E zg}h>2?@4()>F;IK>6hH^43s)Yrt8KlIzP3}Dk;UgzDg;H+Sk9o>b+Bt=Kgl#4BH<4l7UmxHl*dT8lU6^3C` z&S?}`X3-jbWAw8>hF#g@Ru$`XY=y25`ZQ0A8xPJ%jQPPt_S}@_Ix}{XB-z+^QfgQN zL7_StSGaC?A6ZIoEZhTgoI}CqOK}a$$#_aeN@;h}Dzg&jj-qikGK_G%_~LcFjy92% z6|{qc?HwVaMXb2GWi$Gd=U!{g1vhgx0z0|ApY*^ub9%dTX50w8u7#m<+e&f+rwFXM zP>dmC_0JK&HFgAGExTS0zsNCJQ)3sSIrrMoVbY&?R%N1fQe21F`DB>M(zZc3N$gh$ z@2_S?+?FSlmVSJaO|jAorQ-(SO)~(P7)E{&(3_T2=1Z5fe}0nwIe3U*A(xUF{9+LL z5RH;%!_|VQZig~A`lYN}Jr%RD;BX7e!wEt)J;NUoq!5x8UquXS2mGoZy$`*0OI^i@`o@v@N`1WXz;j%*XqxK4@ z@vCQ0<4-nFL#Vb`8_vPH4Mj5?*ch|-RG*d?Z-Y&$zl}651<-MA6{4OqO0=Dm_=-ba zt>JZ5hFLN&3>%F+FqpiO07DMi-I5q5=V~55)j|a=tp@Wver^qrczoNTV!2&Q#9$g? zE_%dY7oa{TF*zVfFDMX|No+WlAr2G+cEBNxk$H#2KpI1+LYj)qc%<>YEr}J1!7UW@ z8K!h&z?P)Rlw2Ky8T{*QSiI&1y*sO&be#pXx5j1uS2~X!0hTIQlhm_>QAZ zRE&QJih90$#V6cV3Zsux66kP@Up-85b-9+=zHa4n7vcE8I+zW-Yd%Kf(BrVT zTDI#=ds4Q!oYh@&T7iyhb=tDi3VXSA#YKCV%34e5k^g2?n!bz0dn6n|!z4O1dmf#6C zh{Bt*`xjbj;lXA~Wwu~*cAO7>=_g)l#EtOd6bk{RKG*Yv4Qt-06{B$4a=CI!#hM^~ zwPA`LR?JudY2~+O&Z% z>V0_U@w~>f?RjnMJdWHnTP9=_T9qW-(s*6$6qgo596anR$FA)NMF3>DsAa{Mzp?#c z*_t<^G^)h1V@F}r?(}1lT8^%yeN+}$Q(jE3&aqRkEQhizr+lpaGIOS8 z^75cAGdqt(+Xpx|6u19ljw92J2=atL+4+3OHW-mCZP?SsMd9}q&_y<)Mo1#dOB!zYo>bQp%Amyd`$8qe+cu+^a}}f#dPP& zg2CRmV9?|NFl^eByGHoz^d4W}LPaq6oe{vI0H~lE0ri?QvjOVJ+WA!~>;;MnFV=&g zuk~n7MkEx=z=@I|535estjV54+|<*hVYuMp0hS!GQ!Kn0hG!xcpMML398X~+xnaQH z%fpagUs?d6mFqUR2B^t#?z1fGHH!SCcuHXis<0~6tfaS!#c8l##jb2D*0wR$8iM%> z=uo{g>7i$rW=T8dJGOJw@K6!KVjyJ1Z$z*w;))0s*DO^~wSJFW!#jv;voT{~(l3`% z;2go$pBtky>$1uGd|VSFXTj{1m7MiPuhA@5hkU%l6}RB14m*1Xcq}#I4!q(VQ5_`; zFkcDyZ8XcPOzcmUBl>BYO}9z25UIncHH+dB)E5X2KFb7W5-w=*aa5y8AuY2QjE~<# zT2ThO3dfPCkW2do(uSc;WpDUVJvLL(4MRKWv71@f0y)_ENg(%{Tl${x>(0Y9q{_IT zK#oe(R_&Amxs&OFv--TlDSu@2UOn_(8{116)Lq>VT3Pm{GuG${)tZix`&s4Y&C=So z)s=lY&J9-0aHyhO_b!KV7pMm1k*UTRwTV+y)KgSN7y$-tQnIX2Xe6VL?Y4)_J{7XE3)1e= zof5V@{7*{rbVmTU4bl!5AGuj#k)@SDPX?_n3$I57qv(?~ai(!`(mEkpT)-a4p2f%O zja~yaS_#>JFL0K^!8y<;98DxR2ZrN7tp*TEZCxi;o+HmBrMOZO%k|A^KKUCwj3+hN&+jKb!#e$ z;S!bMKhCI3?nj#HF*;)CbbJNgotJUI%E>s!@g1~5No9Bfw~43WsSKK9_%EZ)<}|(5 z#=%2wkA1#v=lWh5wx(OgLXJ&2`Rm=xN?rX5Ey&(N1HTSdnF?L7aMvA6rEnx{bW*Cx^Z(AMU0wx!4y6 zhaYULRgh~p&B`(67k4rPO{`CYo{TB4*A%O*&ubu_M6Rt)vNgAzgQjj{gk{wNYb8
    p*WQ9+YMo)uQ@>^Ea}Z4nHQbK8CmojRO+&<+dUfN9lFGT^Jn{5;O9?Cm7Y zy~#IAqZ`pvFxAs=3~e{kBP~NMsE+2}g_Q>(x`owcp9D!Vp__qdQ*+3hR&$Ofxxcb> z;@6IMuyi%=@CA-{08Mryw{nJGMLtSPhl^o(M_VCqP2%0w(plFLPu+rwTU#$gFxXj< zVfVb}C*>H#n%+A}*wj%}rxvW|LM(d0f$uDyS(LPNEEK61CGnyh^1FHOEsG}v@oXyO zl}B3Od%KCio%{k$!L?1aS9h=+KD0$qimCn6>TY*aUapIljcR*!7cF4IfxU}O@6uec z&jIoZ38m-e#RBvxN(fgaJi{EU8haWz;NipWphsTE`T zXE7d7$vzHNB1?_1P%B3a`xat(JxIZKjLaKKLpyJwRXpDhh5+BAm`@sl!-+3DKk$M$ z!z|3kQLuTL7s=4s)=-DjFgKcTt5y@<%H3_`y6aB^Hk`#6``ItzTX_J`0arll@i{*N z3a#IbKn`FoM|DsRMcwmi1ncylH3EEZYXmCAQDoZuJP&8v$If#*EF*UB?QV_$a9(2! zuyr#>U{dFnZ9R}Fra{`)%F(vPwV+iBF-|h)*1EmyFmrB}X1uteZJA5#qEv4vI^7$r zZ1(&}&fUuM1B~Svk=D7?YZ`@yPbQK{4a^QwO=+Gf z4}xL3;%E>Qdiv%dz%}8A4xy?rrlone< zJ;{Zs1e9A6(nc{`m=PCfurGRl3Y;SBQ%f`FG)^^P_Ce($?R9BaH1*GWl~tn2*T2=N z7qr%QAMS?>zJ0^}lY2q+BD^;|^etUIFG8-(tjzVP_j)f5TC~^2aZUB)g$_K3UoUm+ z;l!4_4wuH#R0CUK!)T z*f_2|_eD#DyB{;vZe6*_TxU2e%X6mO>Y{>xI*XCW`#ve9tlis*s)bRK?c02OK8~vQ z6oBK-Zvg%^7vKR0sCYFNaQYq$O9K2zRQVOMF4L1CDBBUa$#%Q~HQx3NY7!+W<3+YA zLycx~DAQA~uj3Y+nBqQfKuysYZmfcuVq+F+d~fH!4!O-m5q1+exr=u2X@h19m;sMr z>{w&ji6kgTGssue&WKJy&>8%@$r!gNbR#W|^tZTtR`4^ziBkls$YE^wTjn8b1Jco^p}MuYWo9N&R_Bkk}663?dMIK^)~ z$5A`Eo%0^a2^*sYxf4!#Jmit#kD^(ooSk=V;ig)lGg6Zb%6P+GWKFSj-L*bQ*1sbL zImZBCIlSNWm&(%Lbu37tdA!YiPVG=r*c^iq=Qv|-YBMX;yi3%+QuIhVC}EA4kf9bP zH3P-RJko%aLmDF^tM_eY;|@Htk)Z^8RY>oudykmS0_~VOdRHf%g)%&QoZPkxW(Vy9 z(#hyjcg#%7Pf6)0^(2Cv>zW1g*w|r18NKq@8hmgKz!Pt z{y`;{$J;G*BN1)g6|TKt6TcXAJ#S-SR%VLBpoUqS6W`3+5GUpmXMVhw91mZRQ-VOE z9PI>=IcM9(VRWXD)17(WARS5w$ckCg;hM6}_W3=4EBUM=(y8jo2hB;Hv|6mWTs+rG z-OMi2Yu)Pc^Jk)pfss*nc7d7=ABGh)eE!Y4OQMRu|Fx;F=uy^&G4&_1e4vJFFa%ta zP(Kt^HeSHoSES`Udr?!wUfo2duZbze;lW<+)?s7~$huW=yg6+}m3p`^4v;l@97xP$ z_L~B=2p{x0KMo4^BNULwA-|r!1)hk!aAV~-5P$XwpY?mYtlM^}j-QWI)9BZt&g(&M zM(pI7v0c*g&}DqA4FSu2(HIrp1kuS{kYrC%8)kvd6j+j1FVF#4&c(f{HqRi8#~Ya- zj0KPP%`7L6eLF5~F^MKzJe(*LRc>oHVLXc@%epwnNFe)t0e>KKhClv9#vdQZbcgI5 zl&Pw^R^T@8u>H9!mPa32uS6ef8Bi-pb*`~}YA576OfMTeB-1ogUh-EF`~twcXqoGS zgR&RtxEop6VI!9aepgUvC0PZrVg(w4!eo~eQ$TiNBKop-g0b!GD#*%tsA8PKekpnR z2^?NyHF7;)|2{}w;?z5wP4Mb9f0CDY6Yva*FU4xd`(n*C3v{?9g!Xo$(F&Y+92vzh zcVlAYmQ~f6{FZWHb|ssD;gW3?W+2-t4mduV4B+oMBH`m-+bS?bEh0vSArh`V%f=a{ zT(4c-D#6AESrGPtS?jHt*s-r`;fVUBaSWVcafY(`{~$7hG`xemvXRw_Eg3hDJC-E2 z8hI2Cb5#vXK1U7O>4m{yF5T=- z3hkZ1c%?o4B2W8gr8y7(v(t#=qV5OAO&j}>71dca_}FW&;YfR|nAnpeFNUc67PxVS z`#vzkC$idDI*y?}=(>b2T#}m`(?H;k9YYW-9(Q1iQRQ*Rn8002xGVe?ACqaBWounp zPHRr)XMU}M7?b8Ri19eGo^u7ju6P_20OMIWh=Ke&$^sa1V->_|Wx@9_G!n!|%EG31 znsiLMwyaC5ND2RY(fElAP(TloVMcW$F2EVt z7|p}DU~>C-4$y&Bw@^qv$DuZ-la)l#ghQ_F#KdA-y?A3~^?iavK%CwaR@6+PgJtTM zV1E-gV4%);wu>96^iVcs!_am6U>AcTa)1EJJiD=?PP_Y5h32?+w2J{nVPLq#v-=V{ zW@vp=&T;8={#fW!=bF7H5*cijbzhj-(u`c1*)q$@`UCYl>XzCH>&D0Lp#HQ{#=4)V zV@w$V#kJ>Y4if4L0p&sleY6&%n7S!5JHPS`T^OZ5Y34Ksd9ynCt)%Yk98dOY=YZin z{Fa)kC{J{BM=YC%ui~L7Gd3lQxKwCJ5k~1^<)FaOWn5L zy(dOO`d|r{q>R~(|4sk|@SYhyhDYl)#U3PF1OG7I5|0k8ms`R`>)>h_#vz-1I}WsR zAocwNVwKrmhJ4S*IHuj(BJO7-xfKP>0t z@xtpLtz66BD3L5_<<%`Z!aWsD95+{Fb1|fa;ovOtwbF2(A8tk+x z>UPwtHZ0N@-M>xWY}#xNm#mxS)@Uyk4bUTtRD}$89R+9_Av&J5q@VR zo`^CA|3sT)>WlL)?Z9yjHjiu4NF2>|I49=TOIH@y_hk2-hz1-gWJUT?qhiA;EhlnD zcRlmq+^xa0oZ(OJAw{DUL2nGfcwn7{KIidQSOSLS9sIE)nHSs@e?=M^*HCl|4JK!^T(@nqGNYh5K7NPhl#^xSiWPCif)=jfd+*SEa4g*s^?*HRIAcc` zmx^LO;oPL~fyCqM1NBM^&e3tPip3>Vr~-bS>i;Twu&B`Hc*<2aW;6%F)GK!gov~}X zFm)Zv_u{nHZ7gJWg%MSla4sGCcPHpcibLwO!m~sy#vHoQH-1_hn**OV(Z&Up-9;NT zNstZR)=j9aa`K>BPu&^(jif28_L=--yc{intrTMn9v3iI!)r?hzgA_1E-4I3HtWE7E?h9+8yJFOko=IBLBD?A&==yJ|%H${U^ zN*YVjajLNilnh8*(}{9BRb4lhlP!!w=})P5l*(v7T>*kT@-$;rA%DI7Et{3mwK~B> zH4iOh2iisCf#~GLGgWQ?PkTo5)hWIj^^Bs@>F;1j@#B+ES9G6Ev?eLKW()K2+2fe(vPMN ziBYE?6-tkf-;@}rIZt$?CY4J6>}MlMjAAmuihQrf)=twQ1o8V>Ol{j(o=?*zsJ!n?vce$R)dtW!CI#((h8+oA$p^VHabS$7UcV#e&TX39$O#Yfv zAk;Gx&nOkHJvV1D!6>G~xCZIG;wf0AYGBM(JeYH?dpL)Z2I$-)TnPH{Oft^QN(q{F z9<7upY!`Qa2WND60H1lBF$TQtKF0vyT)~<3$PMONrSAIaP#bnNyNek#*|H?Lh%P~n z=Q?x3yLlFn+A`b3EDIlho3kvIxB*A(Hvvc55I5!lN1GXpvWnOwq!QrU@RLti8+|yX zwq*}CT7K$rmT~~^^bybT>VqjVIQ7BE(>SPTfj9XE^u-6$+8Sl3U%N)1Zq&BVOIf;X zEO4v=x-3T6jMA0Zk5VsgG#JyQ*?n(e$umg(wu_5KA4*XnudG)%i(>9gcxZRD>pDp> zrZ0B>zKUXmWvZD%x6~S{%3~@Lyx4J>Tc#FwdSBV;==W)PY2}~j^rW709eN$U*SRL!3ia>+3Wd7v=9>3L zfCk}U1n_G72p}T{ry`AjBHm_$q9yFBF?RblLEhSrdeau|7|&i2Ae~r+PB+Ab#CXMA z2MT`%J(w;#Hf{$u(#dM_Q__jk*&X?FAab#Q6S(7lj47kDG$!jc74$ouRY1?JTa|g{ zjLlGaiKx3J%aJ$_nyzcJvU~Qo)28Z1Qc&3dA_T)oG$HOKAt0FUC`RC9_XeK zPsyBjvr}x=r8f@k0=Fpcljj9Ha-sjKYk!bVeZR9DKH(M|yWn+x9j^jQtjpl>8MA&~ zBeeGHTCeRC^=PiiP7ZL?Wn_2>4ptJ4XCax?cq8F-mNLRao!RovgK?YFbYx4_^Du=q z?>yLzY$*Z*CdPHKu$@y3)5Ju{a`WvtbG5KueJ+ zwI;YYw{HxsxH1A)(knzAiSia+PS#V)#+L67`kbdc-k762{$8g1-65bk3O@u$no?-a zQXWHx?@2?joiVUVidrrd5RH8wc!uv6PN5rYpthScKnh-^lZdmKG;k~2U^Vu4tX0k1 z6>bm=oeN1wZp|7j0n#6frd=gQ=vHw*g=rb4%V3&K*iJ-&A8VFXlXUJR4Bd++2q81b z_(VG8z=$zm9D{)5uNkz0s7U+b8WJWnPJ0He-`k1i;Y2`HoaQ#h6dFNeWL=YY!`vba z(pvNv>Lz_CGA$e0K9{gp1f)c%vrzzqvK`Yq$fIDowh#xd3x1p%g`MlVVaGu8S(M4x z@p?@?3Me1O14(wQh)7v;&WE;h+@y4a!l>%pN8OY*wwkWl2||Wt!4!cA@CWY|*pup= z7{Gd;*NBw%j&GpH(Ne$IJcmJj#{iIp#!|HRE4P|Kec{9)Dh(p)vx$kgT>pbhB z8HT>8Y1$e`D8dj*PMpc``h7`;IS*(&4m=;8w}f^{qEK!Q6#@o>kMJxt>>5*CGz0(un}8OU3f!;Y4H^| zX51byeF`*#H|hXzqy@%F!dnq>sN~m~wK%T~Ev(bnWfL#kgcm*O8PI@3R$8t|9tE=p zMe+)u-yt>t!`cQyGzRDuOa-ym@9jdnqggT=-$jm}q-{82_f5ETZkgCiQ4fyU_5xe= zg%Os6xK!D)PQgy74r`D{06dYS4Q!~!iHHPOXWZbYYut}n>?F3!Mr zWb7aM(dI_QtmeHS<4Y@4(34rHpxr9KaBFeJ5XmFgoI}lDkVWKLIhDdyc&1?9)M*UwOsCSnN z=V5)gFMR%qqT%TfIbal}DEhe2$zCVehnr5!Aqe7l@aH_5OwCKGs%_Ar!VkyN&SEpt zr#PL5-k9qobQ}$xk;A&R+Qh7fv5}ik%4xd`;oB};X83kkjt`tT^Tw#l!CMX*TwK5e z%rH|t2qUbxh6For-njPMe0rI0+)S>Y7NbeASIBaUxzyppnn<&n$gX;{J;XjBb1L1T z$eX;l#6djk2?x2koaO4)gDGxg0vL;IWVS7%rt}7FBCIw$Yd-`of3_ z%WF7(-NS)wPL&oVI;Rv>toeFg$u3@I;04dVV<5Bo<38;FX6Fu&A8R&-;UKPIg$Mj( zG;jUhZUS+nd2C$Uxs{bRm8UgD9%@ly%%SV1a~OAX%CT-Yp@80>xf(Je$<@&ORQ5H7 z{i}ATfJQd{8@XZth@N|RHXe9ab>zp-pj7e7$b@3yO|re8)u$l zNB5mJGGbTMnXOk-V0D!f`0#R-J2?xB`F5?7lH0E?jbc_gs$x8v=~`4D$%R!2%PDH# zgjL30{!jj|{}TW2bzbqezoOm*UMuMS{Xg3Op|AhBl<7S7WVKZndJzQqTyz@BKx1q! z)how#jSgN-1Skxv!`5?cBR9LKZZ@2gec1&`-T5-63(u$ic@)1xJ2|TcPkzlbfJ+9- zS~aeHs?K*@uTiWeG3~Z5tE$#jYj6Sk;Elqr%`#81&CoLgD``zP?zF}lXRoTes11goNJk^bQ|#u9TZck%-I?=- zNyw3tjTe+7C+aAP-aGi59|lMlHw;kLH->>iR(LALd1HvM*?|bWABOcBS<3Zc;OFVn z0My4hHJ8Z>7au3{#i&`*3*@H?-{|W;#}fRXH3mRR9s|69JOr+7LTK?ZKLQFvfQh{^ z0#s3gUsR0%MJxOW;Cmq6^$}2IW~(e6V>@$-p~qJ82*nO{(t1XA1@;SJD|uXL&Onj6 z47>4o9G=*w2&HhJ_i$H)tt-1JL9W7mcb=|dwTF|QIvb{`XgchouyipQX5aNj(a+;l z22*bYsplBZlBDSs_}6uVr>j#I`%_3L;m;`#b(>czk7~Qco4iM<8b(mXdqpVwR330j z;{7(1Wq~*PS!?~ic3S-GUe$L|1Xw3U5k%zEnAJt?z9B6XzXlPQrh-;d5OLB2>620$ z$039@#mVfnn!wh_+VF~MX_r*^X+O$kh@02}YIx?f=%;1sfK0OK+Q#wOtXZJ`&Dv}G?Xv@)(eX=((z4$|Y_Z#tgX5BLYB0N5AFoc$HaCvz>gs{{AkYpc!ELAutsiYWDVnsI3#V^O zbW?NfIWL_!gudEcBt(__fl>Ep{v8fK*na{>|j!7gxonht`6G8 z4ruzEmtKO1X%~S3kfk>VQ!XxS0ek$9XX6zC`x(e#aFoRNc2!_Ui5RwGz(hmjW?0fG zx>kS-1kP~YIXGq(rsGUcGmlc%X3BIQ#{mGiCjKCf_d`=YS5RqfhT^DT&& z%IMt;gU8iL`uQ1SrDn|Y;_I`SwC!ADE6*5b zm1(3#?KPc0;Y^!K?~TzfXE793hevynUO9C8P2@%+C!0O;i$_yAh1KIWvf$oW7YDS+ z_P~cR)O%hd|Gp30a1HCY_~q+!2`;ec^f1n}xy)u8HO{`*#q|c41gL>qZ$T^t)*c&5?^){ zg2?h>*Av*%)=qgkpzpxqg>{|71E%!Lrx6EH;SBoKeqKY4o!b*8@tsQqw9MwOu zruuo8>TfCz35yExXYQFt{Btdlk`O<1b}ZNJCiL8>9hOa#bCoZHaY?=m?|$ygkZ*P^ zsSoRqGoz-bYp5zYv(uUon;LK(=O--9n|!Y^ydh{JzsdBpAI#Fsh9rNh(4!7a=FVfn(D2OY! zLFM4VJP7pUFvgwV!&tGd0F2=ph@5+8ypGMSEFy2tdD5VpS{7j*gsK{^hBlp}O`JHR z^e*zTV3l&{Z=r9j{z0Gfl!lUVa08Hc;HE-26{W>NK{8?{y!LfZ%vB^$V^A+~qC37n z$d&er@Sd=F3gY#w79>K(`Jo@AwANBhCa>~NXb*Ctp+BBoB^>Ko6-#P-55s;PWp;m8 z8KEDh*49Bdv}t;6`y6?b6QnWxazsGcHyLW2x&TVk=RBqHD;%Xk$ag3WCB|^6z*aN` zSrvS@X;+aUXY*HaG-!U_xsf{#QJA&`+7Ff9h(iL(RG7sjCAkoYfk)@Cf1;nY8g`(= z!Hxla2X@kD73_$`NJ?3b;~EHZo#R|gZy9OibzLsly^6ay3%~aoXW^gcah7G{#LW*n zO7@$}KVcS13$S+%FL)!&`lNF4<1ov~hqOZ~YCxAC9E_$fQ@a%I*e_w-&9h}|D5t^p z%w=W7Pe>q#TNvsyJKjh3@Zho?TjD|WhA$zeTzIQ+P5Ap?6Y5z&%M%LM5cQM(>7wl+;Cw!$D zgxS`qUyiq)eAcyPPM(d6hI(dn17&r?8g%o^WI+oVZHtFVpA!}&6r zaFfi5tZ3{^_Ed!JeBcWD^d8a#AF#>jpLBpwIP*8X$`SeyU843LeYhfA!!Ju-Md8eO zEf~rQB`skR+79-?yWyM#>EK|evN;Z<+^2J}Dz3 z_f6EXN12yiAPdgpM|srI>vzlf!~7h(?(NiCIvM>?wWe_L$j&P)Vx9vI0-w1$a`Y0% zxy3(6987vusefqKdm5w2KtGvG%D89w>JQ!uSt00con}Tb<}pg={N7laWSLmFx899X z#;@LOA4L66sOI1Q?eG8fpR~d4$zB^71su(Q_xWb?PY^|wTkBl(>qY4sfBWlku1Y`8 zRYheJ#pu)>VHIhv#x`1_G}8tnoFjzikrhhRlAYGCwkjpnwI_m9QJIPc+? z!2t`ujQ+Fr1%~cEW9lwmcQYGM(#34xK^wnbQ8%(t^Q3qI*Py|njU3E&`5X?@Vz$Rj z^`Qzd>M>iT#zro`t4dbJSw5A8Y`Pr79llq&$iXTyc)O#x@AEg5iyXxJ6%>b?Td=8iK%NE6NinpuRx{c9X>a!%a@-^O!6rJ6!SbPJQ4WE3x-kW`abe2A27N7uo3aKs(Sa#OGLJUVg~Ruj z|4kdzc>+BKr6*|d)EcEHMM#DY?y*7NmpIGYL(~QhEfhD>YO^^qshFL9%H~@Tz+o~# zfNVA|A2%5Q4CoCwWpiAH5IHEmeeU4t-lVwG>JQtwi{`Xn2U%f<7>mNu$6s7DKbWH6 z1TfZ2ih$6w2!2??c!rf2{wlw0GZ<+{XmI@67XeTbQ z!K&&-sLPQjVdDoy(gxiki;na`-wtTJ)YjAQ>;?SPJ=xwk_PXANA2^LuN|ANCB7$w4 z;#XXL&?G+VgATA=NfOdy$;~M^lYYtA^K6gm+-R86(|DCupPkB7B5tY4#;gq{F zjfcHQX@f?VOgkuzf{eM0Gjo$zW(e$OxJUk@w{77!# z0Nv!jECX{hx+zfEk^eTMyX>@j&Ha#J>|RddA?m%3>4^J?>XOWfs|WJ|w|56(u7G9EmoYj(L^RoJlr|pur6MO9ea4~j|h3zi7!wI}$uR)ew zbO&YLM|WeQxan>n{EY6XD!?zlneL6C*whpcVHua9nq=0eSt%w@hk_?5nXs3$O83<^SNI8R%3c;#|BjvBcjz2n(Yh5hHOy%+KzcsjzfqLkYrk!U-`q_*co(y zt9uZD$Ff<`kidojBn%L3ye5rp&>5rF;X1rW&uy*sCNSb)MI6VaCEo!7_}nFoVX@*q z0OW$%S@=;Bk~%BPIjrYmE-}O~oui2YbL%9x18pawZ&32WX{Qm1qgi*LH&fUFdqeb3 zwA*_`f1~Lsy}yPw-SkG`G33pn_la5Obc=&2uDvDmCs9N!!V-{j2)%iYJ3ack3GLSA z6S8ltd57*Ba$H*#uEnoUm<6LiN4Kgwfm}%7n;{o;M*%v%*vaRH?#3EA1|=>s@F3lt z&N0_glG?8)fd#O-5Jl1PW>yW6iZ4s-XMFC*B(1&F9;Nd22o9Kw`+PO#Qr|}|^23wS zm|yL<7tr0(f~HB<0^R#_=S;EqZP@kvrHOPnAw>m`!6Q-HE0u*1!&4xAK&2mW)U5Ea zag;qjo9KW6WvA5&^l;%QzUR&SW$d9d#~9a;lr{H#^HTd1RjdiltA$^8uX&4&2!Ypp;E?v=bxCzjP@8j(g zodt?t>gUdQ`%r59h>Z~FP!@e2l+oH(V)my+tX@8MWX$^M5n7wD>^=52fN%Q-Kp&gf zKB9u~$*y42HpswDF80PZzmM3){y$^ylj6BeEDo{UJ+aRMK9i_&zgpH=BGVg(6Z1tj zcFVXvRNHNC=^nN57Mt46VMs6ww&EKnY#SZ*oz35keNVzX8%572OJ5+}0+R!^U6Gw- zRXi(J9uO~%mfSB^00`}thu9h|tqs`S^8U!&C;v_Sm;-5ff^mCmu^=V27 zM}Rd#d(3`_%vC0`pjfiliOmPA3iozxRT*sf`tS2Bgkola*j8!cFsd9$h&!Gv-Q510 zDTvlXt5&Qhl^1C`=wQKeIvQf<&Y-=5MZ>6qTU9X+un&lBvyW?(=F_dwBF36=G0;1e zK6nAQ??%Pba4bHAhypb4O*wUyrTbms%P_B&xTP!(!l~eMwSm|^DzXo~KA`6hz9~Lq z%P4jd-?CsqOBad3>+exKm{oyqaFZC#i00{?b``4Gpdu|2-?Cs&sx~Pqxa+*e{H;z8 z9CabSPBt*RNAO3~4_L=$Yget861Qr-PT2aL${qnd(9TWsOK{dra|4Y#n(uMZP3Jg` zA`w+%T<}2qQ=qkv_3i6Aak1uxN_%o2uo8rLHC9p|Ti39s{mPVj@j&sI7iDUBppVXe zd)id=P1LMcUhAe0XgM(K^yJgTJ30CdW@-jxRV366Zz*~-F*vBEk?v$8*U_TIF5YJz z<3fYe_|pMMU7_b^&ToYLfePPS%1vx}3oko3Zo76Xt!SErEGge}oC`?B&|+&)*tM(h zlfGklmj{zz*`(KK3NtU7CyUZ2-jgv`?oj-IswQ4$!z!ifruTG0ah&CwaTQd(j95CO z_<73w5>c=yelwHkjQe?MrxNP%*rQB5q`)2^^#M*nA)C_A$Qzz}sZcX6&c@fiht8+1 znwK-ck2TUD-{j!n~Ok4x)jDSKkBQ! zJ%%5?ha8(&STXAZKeIavQvNv{jS+?51vkBsg2AJm(fd$rbIT%g#T3a9Xz8`(;D^zg z%uP54X^06K=|hDdkF?O3ySKR9r6F%8#ZT;DoacME92ydw<)%8|;*{#RhF`Y0{1(;S z_F2YO#uKTBEcZrTV?mLN$qz;uZur_L+p37azVOw9brjnB&(9d!X7;~)3kAFwW$?+- zjBV(9or}F&Mu#mzxZRu97}e1KjaP?5fZ4-}cIiwCo>#(e{h+F`p(C97xt?j|mm-jzjP>`NHp5Q3(Y7VG;U7Vjqc`97Ps_VpE(-A2k$4QG8P&c8Uaq zafvKW?mZNT#SXRTrubzRqXRN8r-8G!MD`>D=yGM7T2S{uzlWA* z*(p)c#Bcm$3WQ9s#YiinWkh6D_N*phJGFMO=8WSXP*UdhEt}LxwoP`1q~NVpcK)rs`VJWGau8LXF)+ z$b*}`hxK??`y60a+Q$Iz52gZMb~8UNyqWp0EoH>_Ho9Ve7?>DJ(`_o?+UXRzi(ue} z9zO-AGc3lbkTe>GR6j!I2dIpDvZ;J*Amh)*13Qi6oM}{0={@)k&4}A~<2y7ezTL^~ zrj83nrA6lxe0K+Q5}lW0kdw8vA~ zEbepue9Ih$5_R$TJ82Jq#}}W`-gw4N+QZU0xFa`_##t?wuGxes#-y93s);a`v-3&U zHF2p@q7eS6k|@qHDV#xwgV_w^yLO5VsMtQS!)yl8mH$DQ6WnKYHvljXY5Bx=vbtj1 zo^sP8jtDk|E+Z?e8erBHGQ;VX0>Q$0=_G8;7G@kAr&*9c>o^wXX zF$wn7@(xPBN8VvUgRk5?2YxgM=hSX+-S)V8E8=oyU`WB|rf({NS)F}QCVkix$&xG2 zjyas!6(H+9${YP`*%imCMJy)8*kSYfH^#FqX;@DWNqY$Zd~nvp#?S`B-ITxOyFe4- znJlEeMR}(}i~3SGq?cqc)Oqm?zmoYN%@E0JX*SGkOS5A!fszypmj#rx~aa{{p1}vvu%@^z*=yH|Ap}cy+_Y1CT8n|g5I2GKj!q-tf^&Vxh zrA|zI#KAr$PP!>$hv4GEWyFcaKKs~-%Mu*|5mcg*eH;aSjR|{6-LqURq)+8zuGd>c z74Ty#W=_S;xe-Q1IA{%lc{50Ev_7eT!8J%S&cRJs=|X`t1A3+r>d@9%&!Bso+N$341Oagd&3i>F#wnl zVoOy$S|Ov%x!u-T7?MHOj;+N&QVio+Qog2W3yf)851t+yr5LL@Z$vpow`|Zyr2Jrn zniKx2eQ+%=e9K5r0Km`KiV$a_<&%tTy7Y)@( zko&_Fjd~7eS+y#|Z0&>rVZFTe>c*o99_YS8gP7ktgM&gdPL3@KIDuQ{GM?CV1n^Sd zAqplw<@R$o_}LO{C!%0dl$I#G-joml=17RhONt0jcc0FWfcf@C*N|gkM>e79GgVlp zgV)`l^Ut9)x|8vXr_2sPaD$G?>+UFh5 zls9AMOavVw383M%dA8l#-B1GapVgDqwfquFFe%3^YHHbMcLIZavn6ka5_FUetQ*>n z)}ibp43n<>dR0qa1_fEmM)4pDG8;7Q*aFbz9z$)4?^=Y-G1?-GM)_N9o@Ysb14F$c zt-g`;grRWFgb8v*yMTk6FxYKBV9fKiW=j%B z98=8{l72Z!hHzY_Fh}Y&_Ii}0iBwi9nsshHy3KcnC0@^XkV~8K_%=5FydVo7a3L8~ zij8EiU_4Ov;?z5k4A(BbE)LzXT{|$|EiLT~y6AaLR2&BbiaKVQQr!;RpR#0(Nx(&f z@g;MFoyJxBP?z^3D%{xLmO^n=8ZbmruU%8yA3O0I@bY_(0Wywza~}ha$SwcdF&JaR zlK)%B;LJnugv}XBFngy$IXGuV)m{Yrv>#+M0Ip%y#4l`aXV}QV&Zlji!7PN}J!A&O zfFq}5#x*y$Usg<6WWEWrZhI*sW1M)gUYD}ZF|CvEq(kDh$JC74>zkF7l8%)u%ZJJk z-_-YQ0xP?NkOG^g%uSUVhI(+~Rj6!Hk?9or4!hvJT=Vzn=_NZ1c76 zV@YHhv&mivau!(x=iOubgM)D3dhq;>y|HIZ4{>@At$`ozn8XD?jIf^3dXFtPncfL5 zX`B+7)p=J&^Gawf_@e4O9ZQ46`w+yzmH+{-Z3zQ|cWqb78O2oE>*;bYrUWb|I6e3p z%apidyeesa_ongMj!ITuo#`SgqpI#F$?FokInqF`jns7#-!fd^WI&jvhn*zju~DpL z{3>Up#XPn{lHa4bHp|si20I{iyJobSgS$4%_0UNBD(7?2?K(zrvNjLpWbp9!#Mj)a zWgGzGYzj**Zta&kfy(z-Xdo2aMtt!pf!*S}ea>iI3o1_t>Cp(ArSeM0 zs$4sblH)pHG|bFBsCM(Nc+`7fHa-r`sAD?X}b15`KWz2aDifemzls z15*AO76G{QGT^{XX;UY2MrnKwP-Id17I(Xn6pO0jv4|!i(C3cfx#DU=7#YGgtJI~g5qwUBV1w@lqHo_ zG8apS&EEw1^*N$*Jw8VB`-rlA&$YW}ToW5nuqgoDcjrm<6GVBZ>bz`Ma}ynW4%~AS z9Y}-&v50=t@^!*^MFX9Zg}1Y0CF(r0x|z#SyZHQI_u|*D?OwwI_YGgnM1Nn1QWsDF_%X@#18TLw+FXe*fS6XlfFlyFX>zdbD0+VEkEF9E~F+OlxSzr zOwy(ieif;Hqs%=e^+aD3w-nYNNg-E2Z;fwAU_V! zO=4Voki<@lpc)3*ROKpBo1AtHO~B-FJ}zz07Q7figa=fW;l?|b*!D%&xamwNV!Y>h zLJ``|Wr+ci-%R(7eU0mIM)#BS*5yq0bbq8e1S>?=Inf>IU=Lsd0;#dd_8}AOZ80$1 zO-%wG1h#9TcY-m|9j%;`+)cb&%Jea+Pw`_W&9_sltQZ;>dJEIBcY~dD%VI>sVZgLs(u-LN#@$wzqhxX*E3&a`ldnPBV7NOjjdr%s<|$(*zLuvPso1BdTU!*jIKIQ^1Vg`BEt|a5 zc|WEpe|4*6e9o}dj;BsMbthJ3v>>fo*tluVyDAuhbC1FsV_H&+2NZ|Ev zNM(6WTmPSf!ro%CT&wAx-5v zQ7=oPMezcmkw#($CqUOjU|IzYm1!xr`@Fv=Uf$9F6Uy2(PoQV<}nVF;^AGvIbU1Ns7I|m>x+h z&C|MDGK$c%MHySjjgNJY2hH7DW}su?95u$b$+5Y45CZ2i10ic~=0PMr!ArO|hvvXF z@St0BY@ahhYKsSN8Q<=pT(?G)5AzVNv~spx9(WHDdd+)}1%$-2z)1IJb4*_&omb3< z>K+93DFK{@YXFaQ~*DdF}V42El1&w@j7fE+O4ovUx?kPee3t=5azhV>Hq zYj1X51uR@649>CS`J~nXY;lk5c#AF9j?e`*eBbEkKa0w}Q+r>!q|eB1ToEVP&2jDB zyPF|jw_uNjs=Os2;WFmIk|#u{QfG;$CD$RJ#k@BCOe4D(;N%n@Np={Rvf$N-jLrR2 z2n-L|M9G!zI6DS_+?(MQ94^laq&MzADgMVbINErEhvD_CSUONu(W!K)MDLYZH#&2t z2av@xTOfNh(jCqo<D1_;t>EFTg&*EFmLxa@9UdYy|A}XbPON#h4+afDk2K|Hfsur4C+o z(fgppUPo_ONp4I1LQ@>F15I%aq0K$LZ=$Ko!6sg@m@1Qs0vt zw{?%=2jB!F+XP1&z#9_t@8NHgk-SOo6W??D+yOTCo{?K&4TC2}()D@Xu6k{&c^c)Q zb(i;@XfZlJO8GGtz9#RB!t%fkE-)`m3RqqET+mQ7$>Mw`wYo6K&txZ|;0V1n~{Go1kn zXb8kNoYC1>%Wg1%MGOqJ=zPoNa<#Zb1PgM?AY&dWr?#%y8qS(Z+q*rn!$6FCB*$af zBzLuWtDEz!PDp+*!(PkUCvC7BexDRTfVD5UZ;|{aYdga(nDT@PyIHPCFqBOfhgc_< ztw9;=Y68B}{a}tkgIt?qAjK};;?p!;zhb(s8*_%p%2zPs0&T7dqctw<=HS}ntP}R!sECYtEL`ok+vzEb-o(3(mrbKxA_*MN{zz`^zV~R7H|2w4-hSc#qPL#0w90nX44#`KBdo1l-|y zKmu%1mya6>8`qw64sF+r^>s>W$dX0sTguC22#Y9P`Xchub!@6oKg86KqI*gU9AI;` zqxPC|fiW${=y1gj&Jr+?m-mqR(ro1hkIQ_GGg2R%juR)(Wg?U@wm^968j^HS3++h| zN>8pl?`0%}NQB>T^^h(62%jH}G`wKjXuiaVv(0&QlKuoxaH#Jg{b}@U1C*1(;L9~g zz{6bbqJLeqRWgp<;B~SXf^iMOdBH;S65t3%XH+4EU}A3g^dfB?LU-&7OSS+SH@ z{>t(&UA10UojS{L4!i|=-r=nrPqo`ops;e!o~8X_nc|5q`v_lYnJS$9ZQdG#F)HeH z&X!ir-7t+^kZB@3)?;fykj8U{2bx(^NSj$;=r2gNBQV>i&V=c3&0Q~IGA~*<> zf_VjNokpNBK#b*SHzzq|G*82!n~cJ}XCwhe?~(jqj+)rBEzSH1!#i)3+k-MLX~xU8 z_IRB%-uM+5&;d~4;_{WMQbp!M#K^8YC7VP z?h_qMe2?e{!_=AZVz(BV;#XWmKe$@2BRUcc0COjVg0Q9}#YuDnuV9;pA@q#TV^LLd zDQBfCR7N+f!_bE1p`5ny(XUa|u=RZf?LBh=A{(W*&$i%cp*D>9f4#=(eolMa6nN9W zk2%87ByHm&UV}xmu;XczW=o`lFw5e!$>&r~H-`m+`R}pV0ZLtgJ3(QP8-8_Kh2SL@ zivd*LEZ3karu~De4i>|;i?QxtvF&sB_gSAb=r4R}k-GDWyqbiwQga{l2RGd#@Buho z&ZWZOqw_wc6Eo=D1UCA}OhRsqh@O=+ko{IZ3Xdllt!YG&W7MrfsU zY^xkiQTY117J%_o?s4*_)vR0!?%Xy3m@mP}=80Yf74I}xL zbPrle==QKGx}`O5Y9P{%ewW8Z9jT&QTE88Vn6ZqAu@xmFntIDTw4+o!n#QcE(cQLgL&P}VMY@>0&xIpTY^gn)`KMgFcuMQa7*xZ%}BR{!!Do0QQwqcCrCzo zTtd{=Bb$-d?L3h_W3n+lc+ryg0>;{Xkv2kjVXD)SNeA8JQvUQw-Ox>s()-mZBmlkx zF*ll?ASSM%h!R+0(fd~5(@FcP^Mizo6cy)G>zUMZ+Ud!3xE^VK5NzD_wO|`vzmsf7 z$-R)sjedB-X?)}97s&>?p>P?${R+uu1|x2J=xtMbQy#`$)JNl#V8>nLlPbeT&}8V# zs93@!tkx3YA$ac07lmI$tFN+URb|x$*cCHiiN`i)A|Twnjmd zEmD|UL+SrR3fIr0V;UoG?CS)1lQ_mRFkw?}bEE{p}>*ckf?pAvMvOw@45{N>uI7Kie~ zHB07C>``ZWDO<&q_dG%WI7SsYn-s>BoIo(r+xL zuP@vP|C(9s@UQUUgZLx(i*6$vy(Rv4I=7yr!S$P(T@Ru+1G>GO8NDUqBAMBmOhq7O*&&U*^@aUG3tQ?9hTUPngU}l{#AZPHLUBWmVE`zf_#8eK#c=uH`1O2 zOmPkW!!KJnd&{>piKNqBm;%&T`5)vG*|e zWwDx@(#DX*T_3H8?Su>@T9kgoiV(u9C0l7awcRX+GFZ#nAF7^ff)&wX)%C`c_jvmO zdK-VoqW6KdeZg*Vw$W$~FAWrV54~YQKx9toZMeiq?+fr-^ggp7P9ld*nJwP9cG*yg zSxsYAO2TNrMvql%Kj5kv?PHVt0OG!Q!rokygZ^em*x6JZl8fjd%R7mVFLo0hgvH&Q z5#6|P7SV5__*o<-5ExBQLJ##59*=8JRc(TGmd>Jyy}|Z`ySp~n3}4^1*973;I$F^SsEtZLygXx`(d^lg*^gY+Xhr(tkJBl%JRoV1YL?A4rYp zDKO2!d`8J_hO>kDa1F_Yz0Yuy`P_Bva+oQrwd$}UZ)5?->?x(9yj-xF5Rsdz)1@;U zxVI)u{OH~h9E>>JzkLEvC}g*CRJQbk(%)P95uZkRwOjfx2Y{P&9)K$mcmla@(pN>L z0xgu_DVBhE=+{_8nzdPxBWK1bQ+y24++PZd^UsWEaPvL-AI&+Oh~F%#0kwTF4k<)}?DPu zchgQ(8*uS`G(Rb^0J35v>Mfe%i7c9*AnTb$g^J$^LyVeA7T+y*Xl~asUmcY!2txQb zrp}!xFhIcj$b8~6fXs*jpD8dC1%8msZUxqK9WN46Bh1#PFwAI zaAN!&o95uu9h~rI_-O)9@(o5GG8zC>@1wbiGPvOce(?>~J~SNOwA!2@h+!HAz0|Fo z>Nu;gpwP$e&A4vrx=`IX%q~r z7|e7K?}d6FcyZZ5GpBW(>cUS-xgcwU**AVA228G>k}L)3MJr2-)+dH2)H5_Bq4&mcDv~=7KgJyngJ<){)Pq z#BYMaS7T6*bQI{8pC`R$w%>a~bi*q5sD4ym4k&%X>5C56`n^9VL+^@PT{WyI3q59na{Av91`DfC9{OJ?<^_MTrSdpJ4&_`l8l6ZEz>{i|MugbNR;LIj{wH)BHpB%_0taqxXqvVcl3Gu?PD{3 z`trkn|7#LvB>#_pH&6KOMZDu8YR{fs0gVq{f?aD;(L9KTf~YczBAY(?_g{YL8+`eH zMstt($5-Nf`>T1`uovgoRY=k2Uw`{Sp@ejSFu$kL7KLH zL(%&k7e(9@W6&ot)sl36jS*zipGX@L1e6&4P6*VW%cv3UoxT zs_3JR$)Jq0xl=2vBM5S(jP9iUC z{U}Yb%uo-Yec$HuDppQn%(_hq9aeQ4k5N|oX-}w-S(1}T$s#6mvrJy6i-k^dse(BR z{CZX?`lK6?26@`ExbSs3rdV7sEL9eIyO_pX=)|AJo=%vFxW(=cbdrpUj%KX@Fo(uKi$t4?%`&ks=<|Z%qFzs(-ZKSza0@Ir=BGrVX zQ)Hr~6fBOjAS$pO)RS3A7}{B8Z;BXiI+hKcpN{i7XfXaNnqErUV_|6JU0>Eg8Fsui{76_L_$%l1nWlSQWxofK`+?DV&)i<+`|ap zXqk1qX?_=xps+SjwXhZ1{2E(-9Y0T`ssS0~;|i=KfBXH**!)_50@!{UJ|orq9Z)q6 z%76cngymm1Obt=Lp?8o9fIdQM{O=?R^p7WiYR5ZX@GYDxzw9&V2|B8@#VkiVU4F$w zH!&5S5Okk~6mVwTNP*R`S;E6HNn{$XE!bhGFjKA#;*g9*9VVj8Agn47lOOt;brib= zGeCbW5r0=>PU|9S0j^YGBArr|9GhsuaDACqHA?)v%pZCk(@fW8x6FlBXUPQ}wA$5qt_Ng)U>}d@Q|kLM4?C9hWk1gaW@DiJ zlXY{0F7mkD%an7k z8NFimg2)HYjaj!smlR!6va06TAuxADx2f+%O4S1^daTiEgHGq~*Q(q**dH9rGq*VP zo{tt0rV9^n{^PuWebyYaKd4_-Dv3~`jQ~I`WCbg1@rxyGMb4Xqw1CvIsLPVq7+pU4 z5bLDsr_W!;zxSV~PhZIR@gE=C&*aySJy7$4o#gYEpX%o4 zTM!B@c0f(pMvsFB+ufDY3hPeH}MmO9B#&v6BEdL#*-c71fy_#7?Q)7czM&2KA6H5`ga`YO4lF>q)*jaY4ih( zaTIxl3?tfC^AgIi%UW4PksSSDVv}kynJXvjDgxM^YBxpo>ToZ&{)olsqqA=J96r_s zY!RA0Z|6%npetR;F5^u#*IN2DN)l1UoUnZEGOy)nQiA0rh6&g81S?_m`~g(~sKZza zL}z#f?U1nsLvgT*HA2a!x2AN-$8`Wh&($vSdUY5NgFb&Y;RNeoq>tiGxT=SP846u_ zsSJRt35#OYXvSy+vz*t}iox6^DzUw)OBdBjvk?ZF|E`0P9FZ!-oVZFCqg;}p6qv6Z z=RrG`^^#&lV>vh(MpIn>$UI%*=8jSdGg>4s_8~T}%oxm1I<5AOQgvHWPHX?1vcF zLp_dTUUb}OE?LYIZ^BWiQRSJR4%H?C4;yTNh-=9C+&BODM1F3DuNZ?QNc|h>>W^J> z3uwT@E}$`Er*}Y;1aVCAIQR?PUl+!hT96~gBzX6q%D-+ci4O1A4)(o3%nBQA9ap#- zR~UT7)He}^@`EO(lz({M+{B&rg2Bs;C<-yE8Xc*aaxRE2))HJeCZrnLiI?SI{E;Hz zm{7Z9K1N%1**NiJURgUyfv$s@!rYG%)-{zTSbj7NSszVw@;f-SJf8hdkUCAH!pE+5 z>yh4sJZQPt3#tH*no!O`P~p(Wtb_wSto;msO2H*hgRT=jW7S+h*x)H*f2J3g@Xq3u z_;ZudvheDF4@AHy7!?~>D*Dt{kny6OgTBb?sGs5p^Uq;VmmQ3VVXoRKf-S+ifWakg z8TRBfW$Kl2N0}GG%4vc=CA8b+<*p9(VWeX68N6m86?&YvWet1e<-t<*4515@D)kr* zGP6RED4o`XEQ#3s^{H4}Zz40gXT4SQ#hfnf;FFHQ!Q!pxLO%JkZp*wWV=wPpokvq2 zC_TnBUn(V`n@E>qg`3J_c&O7TFhJUCRj5Ocj#0^mHH{)bI_>**=wN{5y$=hluee_4 zw21|ikxXm!vcNnQIR@wqOH0}%trCV|yzgfi=9_8*i(bA-#k&Mz3Ik7-FqtV-Zy5wA z)AFYwD6SsE+HJ#iD=-GoJUm&e& z{kXz*nGo{OXzNg~WZ)*SadlmYy~#Smys0Ob;Ahxmz}vDYD@dfC>z+{`>jnAT&Z#&0 zL8_9B#}ubGp*fN?v@L=NQlAu5O{fS(WW1VJBq;i}PdkLTI;sJEL9eqaiF}#JsOV+F z<{+g^Gmny@o`qdTa&`2t(7;mc?J83m>M{>WDka9w1~pCNW$MtiY=&nFU=2*izzke*>)7{xB*RLKClKX;?s+=| zO^q%XnYQgR4Y|VLGX&NqzAiKKvaSl-CPVbLg?r90xfankl=lrxqcf<=M2h*$F~U?p zd7jQiQzCxZ)n%$`szl1VvFn0{PhRX57>83M`eu64{P?ZHKzei%O!dh9v@fG~w>mL&CMJXNn3^xXK72XG(ck)}aUA>%4vr zysVz*M0WGpNjX6Zkai7HFWB_APfhjdI^!^W`nmaiAb;zB|8&dh0yn#?E|U|@hy=rxF?_MGWh%K<=HhuFpiv$~IhYMr81x9H^>|qg)G-h6{pBlm^96C_-BFd9MpZf%rkBqqd$};*&lf76jQDFRE1MOhghu ze`RaFO>~jH-s@S0x}fc=_lBp4`!XSGyRJQ^r36s`bk;J78is8%Ygf{#YpXI0gVk%j z6xANx=+7%QfsBinc9eIpI?=B%?%_1?tf2@lTJ`09chuj+)sqj9J=6;iw-$BGV=leW?M2^|%9VdL?Eh1MW?<3X0 z!N-kcEA^7lFVe#7x+YmT4T`k=LPe|R5e|KWj0m3;3Sf#do19b?q}7CsevNIcAF zS?Vr7#6oy$+KU(~2kXe8;ID5jxICDCDTtYbl_VlQv5bJOG?!Q6DfL z^K}p-V@sHj%<~X59YP31Dv`fKK~_SsMp;Pg7Vbw9wzTxK0z-x|m}m9JIa0QWUopKC z1;O0cC|G|UKP^9hn#eD|fBB^c41W3aaUer8{$`A~k4T=sK1Bf1+5o7p-A-rL)iiZF zO|)|aw5I~RAYiW3%Ck+FM^R7tW+sJYVk-%s)1}g?I*j3?)~ZoifpVrS@qHLyu3_ev-~ax1V^IBClixlMAAkEy ze);&z$K|7mtRn<{jxi8Omyu;wl-d}xFYY+Hfo8bK5o3^tyb;>6$6~#J415qIAN++; z#Bb9aNf+x39|auwbiEmFJ1J)5(&Z*Js2ix`H1)h0wG~C6n;`2c;r+tH9=<^fUqFgE zTI#qH0rE>}lh_{W)sa2!$#F9X-h#wuB-qVu9~i3{%RC8{M0% zo$HadWUh)Vjycq37GWJ4h8MNSq_-G)Hg#sLkBZZ4)>?$b z&m{t{B8V0`OQ=l5h1xmkT7OtWR3cc%@TLcTe!3nBeK`}PHny1t`@mFJY3ViL6s`z% zd!*r6LB^zO2$xIX!D1EcpPazM=v=8_@^ zXC!wrCFF8yl{*a-IL_C|@khzR zKbp_|r{Bmg&DfDIpHUh1^Oui*dkc0zylm{a*e8D!hPiw+GXmy_9`~|u&+hCSN0_mj z>Z%EHZ;raz-pTPOd?@c=uROt1apiiJUgt=!pxi6+6O0#Ha)r7w#yVuB^FgF}-#6VD z>Nt$h7hfwFtUSV8AdVQKkH9g>FvyL=`dz&VHi2LzG>-Dsl3}X~;!vvYrZOEQ`ay>v z3hR`4;C|W(iIg7s4y>xgA*b_1F|^1DL~E<5c=hj+|H}+F;80)UFk><0y(ima__Te=Tq)c}t_b zZTMLCJZeOnheYYlW{9@|nK0odOSt(HBJ3{Vj?uA5$0{hBWu1o_4-79829u_<2=aan zYk}OfLdL|4;J2)ZCX%*Diuc1UN(uIN$&d$eo-1G!rtz52B(D^m6+2HO>lD7 z^I=AzyYi|Y6jFO7CY0BejuZ)lPNt>sC=4i!XZO0i)%`L>O&w)&7J4o6PK}+G=sJLX zka$&us$KvJpN&1A52;VGt{qIsfVG$*B@j`X=@3?dSZl8&-P)r3(ahb4`GjKDut$)} zL#nFj{iv_fV#OdPKag<#a=+B+caU)*jYVP612)m784qR#vT_WgLyAexs2Pf60%oIC z?y-ObQ7RT}MDR_G(5fj-XoStqel1af<@@kF)H0Mo=G@k8Y0)tWu|76;*V!I zWjQJ|2AKlnf;{V+i9s{etVX8YhdrNENs0+vuuH=cbr0!OphmO_rUWbDd4i02)Sv(h znTLgAXio+wvHuRr?df{%;YJc}I%r(c)(CYpQ&6#)a3Y0QBzjEr8rON$hh7b%%M;ig z0mCAw>U<(~-Y-kjwk$*Jty+t+*O<|Q6hnrM;0V@wSlxt#H-fA%0a|EGwxFw}Om(bU zM(LC#t43CYl}MUK1haFaSlU8=rJLM%h__$k;ROUQ#Z%;`<>N@c+I1+kx)W~#JzQ{T zLbEm;zSL9wVbZ04j{f5qcZaYaA#t*a{ zQFufYYVXXOWvfrFpht)bDV-qh+-Q%(M<}MGsPa1OXVeD*5r?%z>?WgD$xzS~`9afiouol6%i3G$h6ERKb7f33eqns=pQA8`yC6LdE&M55&W^n6>K9^Rb^MXz@M+}g9G|G~)FhRg8ZhaJRxV&y< z+XPSw>=VRO9i$N*K-eLZFLUXanAXZsVL(u(fPNk%aucMxFk3C0u!vot+7sNc4nen| ztY(?0m{HxEPiZh^A5cIM-rY#vqxSDkh|CD<&&m&e4kA0p`~TE@Lu!pM%BTd#g_dm@HY{I(Rl) zWEap+Q5#*6vX1;QOtEFEi^>u0k{#w{kK>S_rV+Tp3b~@2VQ!j9b_}N`3shVO(M%%L z;{zJHj=+ajxkM>wIhbT+jw}Xh7mO|1_Ap2y)9gjPD3}P=bie-*#W5RW8tpEFkay2m zjY&lPj2A1abP*&)Sad>ww1+NBX9-V#L3(=56V_J6x)$T4O4-b2$Y=x>m1q|wJ@=8| zATw&*qCwMAqgrAN={mQls5`CSasv(o=NfRfUYjji^|cM>x4QZDxBg~v0od7vG-hpw z6KU>PIzWsws6m6`f0)y9m^!E(cR@|YTblLLlGiAelSmO>ar2{?Bd{D|$ZDEQ)ugIe2Q&<`!V9o+CQbd>!^jSX@{4k4GBE!n zqnWmA!f1(TNW$wZVHlb_A>`G~o{wTg^hawMW6*XGGQUqm-LGl2q7W7ZkO5*`Y3&6W zl3`eY4OiQ%gopX>4XUkZRkp28C7~#WosgPd5Qo?Kz!voI1WA@@G{)6c;-gSO$Kx<0 z-vi@Ji>r-sPwuwYVnHbs{60TMu9Xy3A_I!THWcgzJn=&*QR}?K&_l@DHc_2TJ=#W8 zgbY3k^+gE-gO+klP+f)UvxbmaS0k$6O|x+9^#RhEX)xq-U9UZvn!51Ye25p!{Gz?^ zcv;g51+|M9;ALSrJ_Ug+nRpYsJgYIRarphG;BIW}}<=$TrZFx7Q3XW4>vg9+--8qAts5JN#67|kJQ3=Zlw>_{G{ zqMq^vWyG@%B6M*<@MVICnjcgYtNg+kK1kMfqW0P-B-7wGs5XY2CJCPhot9w+c7@VM zfijKCH9(o2;5dgy5ZSQslwI5a3&ywx*y_utgz9Abac_|60KC ztvknQ%|W=&AokpugqQC^*gWvDOj*UrI`Ak%oj_Jt9Q7#5B4@M590UzMn2|#Vn9UG5 zMH&347u|&DTg)hjLqyRA&GQK5qUerTh6rv}P<081^-#f_AywIosRw5)W;*feC?aEC z*YK?J9-9m5po@_Ht63Ps?$NZ1r;yF41M_M>Mu?nd;mXEAMQF2-bZOhZ){Q8?L*alx z>SBi;X8etrPI`E9DML8MWDJlYm=YQ+qG=M;(+6wOL4*?S=S`~;!O`cQqA9#U!A6)P zhmV$RKb;d+VAV~Lpe%Tijf!*!1%)nq#H8VsB0m_HA`G@*&P%BDO_0D5(%L^0`rDnI zc(XZ5tVjsct}5d=?*ig?k%^GYz)vGDDU+loGDEFDDp_K0K#IT5(ASN^6psNy(j^H| z)z-iP8@mLB@aT#OXAkw_6LPt}lnvK@I~Mtx!{CSWU5aV#g{tFnj^;s}hX+Gp@tq}ea zs2t>U#R6Oe3C6zDYb7JFHvYh}$e>Yd8URx$K#f9FyG&a3v&iy-&6SM%VZjPM^th~9 zu%-bB8K`v*9+92-aM*-A&JVmQQY&pW- zZxxM_g%TAGX_^7dkity`f=DY9e%i6MN8=(AeH_+k7LJHnqnTnr6JR!QEfc^v|P|goGF-l8zOoCl!V zy#+N|RSYRVH60#z`~^BNcHn|x)*qxxO`_a*?20Jx%h4O>yl3Uq#eTsD8Dme?8uzLL zn4mLBBE8&TiE}Q{W`Nuzq6;AGsv49?Adca$7~xqmj9(#r8?Q*Bw@~(_H!bm~(b?@w zQlOV%z}OXK#!jqowt@&F13GkLh6;U?3UwxYDvAc`$H(li43W;=n&?051UKm5aIQh; z=g&V^=-T-joZo7;e);s}_pKcC7UaO(b=gs7e5mcLT9xp}3N5_Gw{){YWyTo*Ytd*GvXsU>YU(t=iry=GpBQQu3 zm4;{~LGapEHgB>!o(?583Plvo3Mm{C!XmZIGK_ekG%Xl%Ctgg)xolu_aUGU%9yS41 zEy)ILfx(cVUJuKBQOS+GGzJ*Mpl(W$OnT;WZb`3rSeVu<7$+z=jph)`(n%KYR)b!# z^26PiDF21jSZw`_oP?>0Kb-1Hu#qV5X+@_7?yl(+^km*)`dWbDII>-&SI|^21oW3a zK+9I*qX*OL2xj5s6xnEmM5DUN0fDcyiGaV?@bvf3Up`@!%}W>IPxa@YK5yj}e{a5g z`WyChbuS%`w8e+6htW1st$W@QEEf$q{~T?7f9M8aE_t`6P@TwhBAi+`BSle zhoqKLra{)?bLk`Z<{}Fwt>n5!5k?|{LY4u6O}H{|X<~RgUL*4YO(;{-H%qcoo%BJC zsRxF<@AT@Z8K!LABF^%I*-dmfxzKE_U1LNC@(>kqbnNDOo?4a$OCh?!(~6dTgVe7= zJC1H=GzFqR5~C?7K}LpCiv9wGUx<&2u9-;FK0=#Zw91>tI?R>!DutXQr#U-WTToJ21 zk=Aiq&@x^ZqD0XITUmnHJ+G|$LZIautX>*x`j6-d*^-T4XB!Tme*4%rKixto2)GNO z%qBJ)p&qGRHHDK$v+C4kzxT*47=2MXbSV@ZbcZTh8P?>2vA&<{7d=0_L1;NgxgOs3gUuiu5 zdxi7~9lg677tEcm5Eu{Nr$Iu#VrE;nQhS8E5GdBEipiPQ~(nyIcRIu+l4G}1R{ZXpSOk{`Cgr+YWJ_WBUvc^}Y#KFo-+0yg-HS}&BL znkY3S6;Znadv2?)jl;Ga%UKd%V5V$|ZXquT5JN{vyR4#;CJ2pXNKgm(60|%TW3I*V z@&ZGO(0lIHFqUe}pT#7cH1JyqzHQPJ4L3aL8S%;)Rd;c=qD$XH4IWw^7-p?aaDZDq zR$ZK>$Rw*_8b?~xJjxT4=ABW??cMDv>3?lLjpN5l{ruNfoJ$@OjX9`-xP?dHeHR`X z#pS}Imza{X>dJ8D5s&UvU7?WPg+^$x#dJ0Fiqmq);CsBS0D}65Fa^bI4Kr}#086|o zTG8^n$rT1!1<9CH9mN1AkFc?bMuy>b32JgEI&gv^7Ar%w_s;ixlOrOq+I+|6zIPG= zQ(ZJMlqb-7L8ew=oo)**g6Fj8l5(G+2x>-fbWn9rpoLLo$uLw)Q)dxHzAJ~aRHv-= zcvR)-$?UAK1XCht=j=KRD;OVYK*}D24CEBZa@I{lI)tP0d~J}Utg#)G^G2XFbKxur z5T;jN8KQs$UKbkIsvN=Dp=)!3hJ`*b`OY{rExOZ)8H7tgCa3}wdk#917}XMW2^t}2 z+Ho(#SQ*TQSuwS|p_qT8FnvK5`CJV}g7HNd_t|7sS*GQ8&?BJ!o+jK!$IbTDJ6z6$ z4txP;Hmy0@6qY5O%NC2y(R~-Sg=jxS+mL{c9Kv=@2b2nD zGzyRw)c%wpIuRvem?m3HuK3XjBu1*PO`A)d6nb!$kNvkkzL#kVTYcv(EIG4nKmr>} zF7fG?k$6rl@ts*W6EZq&97Z#jsjNK|Pz_^M2NglipQ5UlRiy>Ck6JkF`7k~}qO;sY z5e`riU{ItEB^3<<&uQW>9>K^eS)ez$F>dTxr@=~qSul0b&e6K?Xc4_@=^{D%jbnhQFC2N%&w*a zC7O*lh}`$eB^`pW(OBfsE3KPZk%eOl0A|}wCAv^riF8{eFdVpUQJ(TWDkOY#?d%>I zNep**U8frH@`y(;mg8<*FwD-6vAD2ribTXnb)pTp#&8|@@(aQ`%4;xM1{FRFmK}Bd z!Z3lSN9WRvLWrtVVbCl&CPxi)Y*)0eSCyodtT7HSsnXNtrLXAnZ{??|`8-}mtpEJ! zhYgMJCq{Js<7>)zYfI@>5 zTM0eH$Sdr>tT;M*R2*ZZP+u(ST9z^M{_D^0?*p`qK40HQ@}v7kx@ti)oE{^EsWxkm z=^k;;JBBnsNDH_}==Q)aII8p38>>P^8TC920SD_BeqZIN+{M5ns9{6c0GOC3fiK^>h(D{u6&g>i4e|C@$7B}X!BW{7kpab$(C90i}^{z_{B~Ggn zo$AP0%X~rM;-W)dqB(s*m@mNqGa43T4&Z;mEFStE{UcS~jJasW0YHW7p}uJo%{5!+ zEG^AG!54??{H_5$)?NSj*D}j75PE;?E2bl>0s$#jCt-}nt$y_qr0u#^*Cy7SVIWkD zDZ0~0F;c$3AcbHFng+RO+0(dDK|huh)1EL$Jw{VU$VyC@LLno{0x?H#uBXtCFwxV~ z*L-_^+Z&Q!aY^#$Km95Gk^C6wzZ~zq#IU0Qe!|M5g$-$sIr<1>qydJH1&C`P{DrRI z>^ji(^267$-0km&$lvzr);j-9JN=9I^*^w(dF@+YeemoU=$G}P$(o{}1cNSMP?jsYXr6`7Z3z07;X5#-Si|LOn#_fMn%7IS?1+sCo{fFhHB z*3SSjl@jDX5ki#otv~S+_5Uw>@BZA@apn7-pIOEKVF$ViNk$(CeGfwd-r~TFVQ(QRW~(bfxzDVSiRn_=1M$%*?jcr z1G?`#`joHKQ{u=KX0={I7owj)%0*$$zSRzU=qP!~b*sc=}=&hIh*C zEzdgZ>;H%U0}&4T^Zx%yn89Bi$@86;ulN3Lt>)k8zwnjU_HXg@FMsL17}6tBEr$54 zANloF?|9fB&+&lqF@E=^H#s~U4MzR(Oy7r*Ho{O(zyIoWuRkT5S|4nZ@8cxjfAy+2 z{P}D+m=EdHt-r#U;Q^a*J~`;4f2IlJz-ZF`c)4mV@!MBBy-#>t|JrE8+~clLV7~3m zCorLJ<8N^-y!!oDz5dzRbn?#D8Xm|8uYBevyuBWoKK$3G2>@b%qiLU}PjSKezFdLk ziM!J;-+%RWkLOS4bj%kBzRpL>9<&n~Pfm}z2gAeRbSi+wb734tV!r-hFr3*Tl%pUP z@?n}F2ya&&s_xW=XP@(9Ad~Mm&U-LDXYtvF?T38R{$zCyE%|kC>pzgo`jDW=vlW0f z|3;?e&)0Xj`>IDQ^63WuW`ic&)>`-;Wv|WZM#N^_S2hl z+K;^J8<8Y*&@fdBfBO8}m%Y1K4g6EmU30G9Y)BhBM05kS%Y$}QZya~{y1IPrhj0Fz zBvZadE%W*F9`}y1u;(z`p@) zS=5`}2Vi99o{qhQ5$9ItL_(*uEB=71D;Ca)uC6V>_nK0{NCRVLw~Cessk zGCLc9l^(GKNtm<;`&HWmt>>eY>IUt{s;p?+sE?6tHVld|-`#$bAD@0;HP@q*d;v9( z)YUf|JqJ6JRsUQBq2lRxU-c%ZV|WlBN_J(igI&x=??x5n*LI@eko8@ol3V&ykyD0uYQI+pNE}2D=RajXlwYSW zIL7G67;){)^!w@XeMMnyB?o&lZBbJ-RfXyrGYnf`XN~*& zL!iV|5j*n=rnT*r`slg_+h!usky@D?42OGpv-2jfU?UE43H4 zw*QjB#$MC83=a>nqCtVm>G;nZjdvnS$5~B9+CPWbYbsKGy@KS|J*6Ww%~e}ogEbiW zPo^g|T=JPs5H|Jmu&l~?+UPavuWh6He4^j_o7rRK$23&@aNE>=Vs3?it!dtisr{t? zlj-@6;U$4Ig?^I^(-g(i?D%yW+?6(hcvRocUx8O-&p5}%hnW4IX;a|nWvUOm)CIvu zjIe+_v%|rsU7~bGjGCpU{*(}45wO&%_eC&3qp3_mutpeUDsV=n4pFTrn`$uxe&72B`4ma-{@ zMNI=ETXLHwn7tp@TzAc7Yp&2!I@Abwesw@T%fxLxML?WDfw}5xQUmvt}vhGyuQsb3LXO zo_&w~yZO1bF)IO?PBvRXI@^tSxR(jbryB}M^p^hT#8Cbxuk`(c!RBun`x9&~jQ>6S z>=Btt7XPD-@k4F_SN!kZy-)xD_}{++ubzHMysvmSoX$uQc-EmIVy7UGFgZYHf7Thz zCSC3i`n9l?;osBrbiol-);P?(KhvZ_}Im{ zm)>1Jm>+a(@@{u)x<8pJr{r$2dpcOZTfl=Jcfpo>cZ+vaoZP$L?dnlm9{CGwfr;&# zVgEq?s$hfF&(FF;0xs{Kb<`}0076%nICUsFv%j8o=1j7oVV>A+QnTUw+3uTfBQQ2J z)ODDP%hV_*ls#JeAWW1H}enU;hW>(a6S{<0DAYko#*R=*{oB{S$|lz{$5_1 z4#&?rGrhq*eZzUj)@*;*pY5^jgP{aj_}_mOhrpqG*gqMKKYUSePx54T)*lR?6#rdt zqv^YKHX{ctH_O~zUeXt%+3b9H*M2*l%*#)>!)?s}qT9jrXmr{g4-e;G6!(WGKGuBQ zOmm-idFj6xVRkm|GwJmFv^zSrr^ch=EG~_I(^&J#*%!sV^#>ZP_CIhfOt?RpqtK@3 z!UhgThlkz%!GT8G$3i`t;{QD8Qi3>{ei51nvRQ4-9z7V28S=jhp#?!zU8f_={=zOp z?9PvQ{pdJq&tYAfNhtT3XxNAzk;svqx=dm#?fGTT5#j zTPrL6dwHpRQcLrd%@4|TweKsfEI#TW?)RP^jP?A%oUqPl7TNq4PdE5R-yd8A4>*hl z+JMD$5fa8`5#B>+KhDh=usGXoT+6n0>w>ac2_%LGWs9P-lJ&Cx!#ThwwcXdPZWo7@Z!B z23)i=ovm?nhT|OIH#DQCggOjy^+OHkx!@SXuUb(y*K4h1a-L9VcJ=8DFMil|6dD>9 z-Fv+GX{TpmxSsYb zKb-E*&Yt{16Fl+HA}9a!w7;17X(9Xj)56qHq}pUUJ}7<|p%nfw+%L+NbR>6}h-V#d zX6I+UKTL;5PuMUY7qI&AV77iTn(IV2YE!*LJog93=P20o8HhnVX@1Vl8X(i&aBzGI zJ3RW(>Al?D%chkqXxYr@bUNyhIIlU{0jkMP>__g5g!3Y7@U$(?77^ z4u&YvKg~Kscf;o!44bgMdoUTCE08$r!g4>qN6BNS7r$Fx0tI|@vC!)Pq?|zf@O^(e z)cB9@J-QFek6&?$5Dtc#GA9wX({t>if)vj2hjr7@HqaK))VSw7XpE>I|A0N8k^PD) z=f9#+fq0%R+uxT7G=PY=!Y2)EYFB}xNghqHbBS%Zyo6HLFJ|ZapubT5&DR*u^JADN zKWK6;$r_G7fUmQ`bhM97FgoRTevEKiFLuGEqe&ms;%hGB8=fj&Q_x#(NIppzazaZ@ znym=q?xI%=hVWiK3Do4EDiM^i(KN`36Uj4{e}3Qpkk)fL;bzgv%xzs$R%Gk17pPao zaeu}PP!cO09i1{51XUk2}3L7DpB!4My7m-)iR~@f5*VF5DF|7QX+%Nb=?I1WCv7um%={ zR&A?A!6l{u z-h=ajm<7udu_8T0fLjXa13P>$c(_L6nW%vJ-2o;A+Xasf<>4q`Ef?@sMnYsY&AVVh ztrwM2<{DHg{PQkcG1Ed@x4v=5hFAcY z@T;}}d|WNvlDNS(;|vDp95FXA>!7N{1u7)oGMBCcC83%`Az3*$Q>ZUEIBd_hDmO53?uP`gIw}78-7b`7=qQpU zUtKZ+s!V&j^VM()Db2dy42KMz{ROL-7ya?NO@+)V-cPuI4Yp;~6NoN|*~>!2n9k2| zhh!&u=erlrDQ$bqn7ccBFHpezZyU`{pj?7}dVDWFT7*CC$xKP8IF>RJkjxijBV$>< z-l6+?#X_GBPkON6ryKms`pJvR@f#Qyt8_gBU(U|=h5dkL^Iq|SHfZN0KI7#6{PEfo zIBJ28Cv||p=wx~(ALNkvLaDp11T3VTsZL?2lagHEz$Fh*x{WE|P6Y}^w1!f3NJ0iNA?1oo|9>O zoQoD-c`#9p>l~6buStWJHZvUp__@}LtQQtm*opR)@F^Jmf}l=dGX0aQ-ko0KD@}O2 zymixg>!xAv6Bbn?azT0)gD_hdLTH4ujn_iJf$?i!a))v(|4mCQP_Bz~1cBL=?4?aG zWL!s)0dbtjh0qKRM?dq;e(x1O;ak~9f0m|qhQDEXsm0-d($c4X3^T_R5vg8|`Ui!j z3z%3iDhgVOYYhkjqpv^+NC*^^8KUF8&`5-`(ByR`Iml2jc#g_HMC;21X*lTMr-+AK z`7>cuRAWt<4SyI%Nx+*=2Eat#+o<1aT4V@@*q-V3Z0dRV$`Hp~y$P)L{$_UKci>3G zQs&bM7Oeh}27xrRcd`0gEuP>?aEC(n-_7;L&=Z^)gmz?vu*7feFXOCV>zfc+&4JnW54IhsV=m|Nw8QEJed2uA-3DEa6DPXkN_w`7VjIoSP{(!Y3(t1($$VCs1mxGIoPe|? zeY)Yu59=eJjR+371uY9ft|oy;y5SY&Sv;*XAn`gP;9)|9>OO)?0LhTeO{k;dIF`mDMhs^Qv{N{g}Kn5azq0MteH_4 zC-DFw^n+dB`$m5he?|?*ZEb!bDCoKvz1iT042H&WdH{Xy5LBKVcE919kDjN&5#^Wc z&ii=hQ_8{a0mz+f@(a&NgdcSRGgf(G)OjD8*};UUcbQh$Eb>|9=wV-((^ z0Ozeag>4iJ3?T!w`;3p=HlSNjjb((%l_0~Zr_~r&yZse30Ie0M4!^Io&3yjt@ZBw+ zb?qFszJDdZpgdIB-?LeOzT z=xoxU5i$b(?eDOK5ovlIDmEPErp3dvGrnaAyUa^S_Wn?CwT)IR6OsvjTPRc{f~*#N zKbBgW(^g6gz?t$<=Rwc$oSNA(iY>OZ%&(xM;||_arK*Ilyt8pWBk%@gUsKaqjPW%- zgChh!GngEmDsIq`_rrX{{`W28LXF|zI~<=Ae-N4br(2(|a_V8e?&z@l4k?t)TkIYd zh@lbY%}urMiy&UX1|YgmnIGbIbZ01fy<)BQXWiF38AS2_sm0x8&VI9}x8lSG~?4o@l6$HO%h>*BAB>67< zy}T3|)(!Wpql;649Uc@jM6r*{OPNf%nb`LOi*sl@2a|z|A22dn_+cX#y z`KvNiETWqv*2U9Q2w&ia`0XsqQ@s#ZEcxa5L2v8q43~wsui?+9X9DUWP_cx;kDjWcd!Ev*2$wYxfaz5g`>!%c$Uu0m zd?8tu$y!)m3g2A__$=$o+%Jj=fotQBwuSyxMzJO37%TCN)wJ{)5UxDw4wEKzlMDoN&yQv#w{E9Dg8x{NjN&B;&K?@L z@)R%fDYUVpb7H?x3uZ0pdYaU{qT3{*{`Kh<2rDeb$G*2%!?W)$zak|epa>~I;0b&v799fSq|Z1WFRUM zD=j!IOi%{b}hX@m*mRHi~sdS0eNu#{yKmEfO{ z4DCe0JArRgYDijVN;lLIdKnsV(`p?W5lr%R$=|oc0E{8!2zfrCGNml~CoE8yp_uId z6T~H|+8>Xxk44oOj$=y8@*oy9FH@DM-8q3BJ6m4jAtJ&c@|2VUoY&Sw zOGyML8e07Gu7!B7lCxzS4tpbeOUYYG90_?`o(WQX^??PKfu6f2uG7vfMjliZxRZ16 zGOZ^>%g@Xzt-hpIedMob)ibgMR6=-8rhMCBIPcDn0j`R7s?)Q7gGVE#+^`yQ!bp09 zDG11B^&O*QF*@z@a@`X^#qw*Jy>{O7Ccw%(Xp(Uc~G&PKp44hbCg@_Q*gVV;D9GF@) z5KD(g_CY)tY81uPh2bG?!bW4D#8DNze5V9R#ylatt54Iq6Xl1H3_Bsw&ryhD;eeS+ z^Hc|S<7-&4MXva1mhHIFhFw90+_SY$gu{frb% z#=WQYvH=LavG$lXOUWsrcBUW|gGE+JqG*u=x07gtCJ-=Y_4AIPdP8L_wvG=_HaU)tn9Cgc3WNxf_@QO2+}X%S-RE zwhu_G9Fwwb$$J6n6%k*o`q0MmeTxsI~l8ohCV| za42oPe?B{tITDF*VTMkzin2j&EewuKxrdf3tEED6ejXo+zW~0kCbK?BfJCU30eRNB zO^M1z`m~9x!HCTNs49nzzEOfPesOaIsP@iuk75;r-Kygda?KrApraJv@oa*uI)Wc4 zX_y>HP0bcO5$?qE&W}eX2lSH=yE2H7+d8<}o>?2{D%f_S9KuolnvkKq5| zYuJ}b0@8;qeLv0{bFTJ|m4a>w{8mvgvrIfq-=8WwOPN^{D<~b0lsKwiW)Rd^>BZ1> zU!^VJDCJmOkBJ4AS`PBT+?U`E_J@s{H6(e>xg^ZCOT> zvkQk{G14bEJxy;r73#ITL;8JMKiPep#H4tSnhXkfWA3^hq8%rh9jL9c5kOd+o~v>| zm^Lvf0>T{Pa#jFc9zc!30WuW}{b_hc{+XxKQ&2J)qXjK5k>@lzIX}5u00N_;Hbk~;m_DFmp0Ur%90M!^>4W-8DK0i2J@hA(zhR9cIA*lBSV@7#ZRrrwIv+mx z+y+aJN-~1p0NYg{J@jBR3^*Xge@?=1`Oq5rSCALzAy6utstA=+YJsT0<0Y`rn6K^7 z&hUGnD=kQs=|UpT&!smeOhaMkbtW=xol3uwrEnA>FF!JJLyx?4#F{l&tqa{BLb#p+ zA-c+qsb_$btChW_5NiV%*|r+co3q8+WkO;~sTw$dHU-SX^dZF(qbk{4d~OwolF2E# z(52yhe7fmN>?{|APzmQ#{*ZK{s0=s_MKj3skdW=p@&tXEeGUps7S93U52B6PFs3Yb#OGGSemccR6W^BRx%bAr`NRvqunlgcx?t)2ny+1d)>UT7)vGT)X@gq3 zyt~!qKJl8e_pn2q2wu1;c*S8_Uio@)$!_ea#oO6Ya(cEOd%U@!tpcsZ3u6LplHP80 zksyYpwO6>}6PedaVzJW^udphP^&BvzP0{}#T|!^l{>DGVwF!Rpjt z9G7g|ngja~i?{y*4VvVI8W8N&79hkGnRyB_ip|AyUT9%e-5)|+_kSc577sy?OS5l6 z0h};0EN!0kVoOcOt@RQ-obo<#ydv8!j9l-e39LMjr4ltpw+r=2BAugI6qXr!>#qIEzt8v~A^?ImgoWPm1lh=_nkNi#VMkWbEN zV3qn$nJ8d23WXMr;SG^O`ulM^Xma>42SaS6R*pHMju>j1NcD+;=SbvZvCYAkaI_HM z5XeFm^n`?XlnOIMG;4A?URiKh(t`57KnJsNblJ!v5ol4u>cbff-!=U(#gME9>;^AF-RN=7DSL`^eo*(%W;@wK*qux_zQk`CzAo&Q(DJ zSPw>pb8slqtM-C7&=!WN#lgOI@|NT&*`v*BzlW9DXAcb|nzP`{Whk7LSD40smGiJF z3^DWJLG7LFSl%Bt&qe+*iNfHtTzD>7nMb~Bt<|UP^s;rkP^cI?>qYgHvZre+_QNVs zBA0Sy=uKVQkbWI+qaQfB=DiHO6qT#V=OWCupEIEO-whpU^;`L_CB4A8sTm@(FCgf> z_lS+!_YkAd#%y(&m&PbSG`EKEMxt3nx3X%<=?ZXy`Z1!F0K7OhA^eFmN(MlcRYz=t>XuQ}M9%N)<3Y{3r zO!ioH4W`H0_+iJPY&vSBS@CU5Z)h(&RxGViHT{^T8EH#WE zlyG+mfu9MPtxE6`oNyTPo>SO1@xo(mHWe_xnqV>k3_xJEg(N5Xxa0JCksq_*=2~MA zOS;hv5&gv)%*`QyA|w!+E8Kqtx^RCOK!~>$&fIBckEv(;Pryx9uQjtU`+Fa?(mcN9 zyS^MrYXpcXmhm$(hA6WZf|3^s3F1oR7CtCTTEu~AUx8fVWjpsMmJa)~V_CsTV#FUx z29(7U!!*YBtg`X#_s{XRhgt*AcjM?(=y*9zvJNK?Tv?&3%Od)G!K2TE*UKKwXw@jJ z#h|&(g44&33@slSS}LZ3hnp@8L8M3ELGPhxRLZS8#bkPjCC0==@N%poXA|Nxyk<~b zA3WDHZsp63+2Y^3%ya-krdehUM;7xt!L$B!2Ga0TLle5y1_?KSDa72}SGzbEt1&kc zT~W4$zX`qome2AC%9R0`!4^=vpgKiTD$*(TAPAD$g}+{j={byqzeu5CbOiAW4%7Vn zrEE{3+#w4wYgZj+?4hlL{#oSP+~zj?S%(hyUS3k*wI;E=hgK*Mu2j!nDTE%43=(xj zdiP0ni%yypM9-WV0z-TtJ9n{UMrxcpF08FQfG&QCts4Sqac+;JnHtOQr6|9?7J*7; z-Vg7iV2qa$E9N>TWszwUm4h|mz*}@kGe?mrc;$LbsZ%t$uE^!urgQ8QHKQ#CL;S0Z z7mQ5#2cxn)i^q7@+o91{bs5$=N9}(4$?%al=@+r$Ljoj(J(rha)rY)-Lc$1E0XxcZ zAz&?N9s}1q9wE>61b5OdD=~BQW=RYC#lTd8#Rn)-23igtULYf)!!=%X=@Ehnm96WJ zI^8gh+6D2|fUuSk0bx<2$k5=ivLPK^+T1>Fd~6z>*78zkI+@_9f;pAT+KyF|zP@}_ z48r7Pm#wfblRe8zp}_AlaAr~x%PR;tJ^^yL%UqwLUZM5~p-T~8>nH;s5I+*Dh_$i& zh<(FZ1-l0>ADUdkV+@_)O(h6EsKp2IVVQjvuXj&rG?s0mSQ3;l1{KImc>;B2yQ9AZ ze{w~X1IS>mRLUeET?$CZu*!%tzm|=mDS=#An`~s zjDp`eV@=t^8E4YK(p!|--w5`DPx717bTTCP7=IFb%GP7tDQEvEWnToLnuq=N@a9-h zml)kKvf8cjpx}uvOVtbw4wa50ErI-MFZF>d@zM}7IB@g0;I3Mph{43rMYi0Ad1N6` zN1g^RY#H0Q5g4cym&9a+IZZ}PF6{t!r{+@2wpV{xtm%sSp zS@X;64Iim7kPWI#CLs(vZhMZdwok*;b#sahGpZz%hCZ9}K<>)jx&cI1e0uVOeDF zG<(BZQ`>|`*Ra1g&CuC&LkBB?RZEg-#<(>^pZf{29guwJb?*5dP@O?s{>x8%)E2;t zwEwU(0Vt*`B}fO13tqS#BrO&4`Pg)UV{EklOGef}=ER85iU!(CVcSetN+t&t$LpaA18y>465ZN=QG(<1^fRO<3%7zCkD9 z@j)i*KpVb>^!T=cqFZKp0=pRrSuw*|(X6``2eHXzCJ8&^W(wykw6-)kzkdv_BL_yK8|@P(l%gAT=szQAc);2qUf)hpHpv>ao4X zne>hcFI;hK7YDQqLd%A8aGQDnDpq&lT3+=#3)M@xuoKdImhD!ZFPa5qkUq+$Hq}WQ zrgyv%kx)QP$RR47txA&Qc!n|3ep*l^rNWvtHy^)qTo$iaoO{WSQKC}GfYSgsYZ>%eame(hC*xMqBeSUzhfdEoHYV@2fj=|{vZLdCu}JA6Ph)U1 zk;GTDNgY?P9G#M@{?n|mgC8uv#Z#iod>3>2v@ls~z|KC#2pJC0R#+Jn0!co^2W2Ur1DIZR~j8dp>7K-RB=}-TT<3%G!TWACbi#-wZ=Y~ za4*4ImoxGKq>W6NI43@6T z+Qc}6G`H_#$AS)VX%8rCZCl!da%~SPB#P3+VWWdj&QP^Ww#e?eGmPbM33mdwlr?%}{S%Hclx_Ja+KJQs*?--8(S2-+s6K5zOv^Av zW)jSz-Q{9E(^;L%v4OGYLN1U5q<9g*OK=n9hEmO#(cbnI<+k2f6qDI^MQZcs+|ZnLrE;|wI&fg~hzoJSXlN%e| z53E!818)?*1tHlADu^Q8c6@e>zfqBU7Bu@X#{0-;i%X@2=_j5AOTBu1G|1G5B$i)L z7f#SJ)=WhWC`$N2hJHeIGb#;eu2v>jWGR^i;ZPQCWd2n~;^F(K5G88C5{9FB`QP19=$2lGxe>fwG&f}g*u8Lj$(4AKLlDQ5ASIXSZ_%r3$Tg?oB z!^&mp8EEm`XfwK60#2*tdm4zCvS$_)9k(ip%LQETXdVVbOAC*<)%D(rBRrME-&tK< z`P<5`D?hKS6{{;R`E+8RzTwk&Wev;0$Wyu(TwKnMnF{B``KD~;(itp59%13Cw72thi{D`ij z_OEFLzgEG!EhYoc(?1T{UyGIX6=dB?*Z%tt`|lU}Z)IrI8*(DRH{WvQ&ooiAgrt}3 zsFmC}shMw7hs3o@Bwt&`Hj-j)GkI*Ewp0 ziWIGWBvvH^VJiuz2TW&CN+vb&X0n;Ui?JMZG$g1+5qub2r{1#;(S>GXZ|D)%Xg;`X zH{t0Cn8-BCj#C3sv)UEptm2B9u^-4X;(EQ%@YMb=K7zufl@g7nzmF-Vv4wAa7+U#; zG2$m5z?4-S+tx!AH6~P7VzSiM@y70f4BNvN&HuEaa5Dl>hejWvk_(B-(=9Kj+&S;D zDUDh69^4FSnykABLpF=+gUI&?>P*T9#dWEtQ4zn|MT~M z=%1WD>ByT^;hC>@l6RP9Jmnu+wLP1|KYKKNv_t!(&@{D02Wfp#nnRYi(EiaF=dW(0 z7oC+(xs{b`ZDo0>w(qq5tdm;U3e&XjB&H)=mxD%938HoL-c^txzEoSvM<^fx6Lj_M z0%W)VDF4RuO$%%u@!|!Nr+3@OR^v8C$D3D5zp1==dk46xLwNh2ogal2 zvdYoZOZ2U%UGAwef?>+AS{{Tv*f4}>-eQ{9YkX$9ZOnnrGNoQxT6E?h_Gg+dVjRT& ztX#w#3dy{VNrJ6VrQGG4E99(URmF>@!^UXJGawZe8{e3sEiUa@&hs@EKBn&KR^ z^4QZ=s(chbuliVrmXvK
    IOwEnP&xCsF9R9)5!J1S$8&ZcB^hisL}837eKqltiI zy{4p502 z)iuC=x2n}}MxRF5MNTESEj<06g=HjzM_6e=7tySVt)*+9;LYFxnX zm{lBJ!>lh%N(&=ggQhku9JHCjLF9MK=VJ*zo&o3Jvtug-MbOASJ>i3jTygAV>(_~$ z&hhAUdP%zogkf$T<7U!Q6w-0GO;lqthV+EoBFldd#M@e~Q>~>Te^@%mPBU%7sHo5t z6d;wg-?8>WJWR##5IiCRWeDdj5r{x$_(Id6KD;yAVfd!!|Jo%8c*?{BL(kPLl3SAL z#o~rOa`b`8Ld`X?V+~wIKklcS#aO6OmmDlXH|rRog+QtOSSo!T1vYhgGJOx}?B;iH zbAjh^gCM!YSJdc@AUt;!VEG=)MO4;slypnWV1wFKkd@1-7f=AiXTADD)p92L><2m- z@DP%-e8`B$N)`jS!!q(MW^-G_zl`OTP=upf$u~$|Ej$SR(=gtTKmH*fji93f;oz^@ z>oP#*D@6k?IjBxwkRg=hDtJjSH>LEz^mjeECM-jPr=$El)(#5-*R3S5TjyIr_(zo@7r#Q774Z%(ZN((f^yJ>;Poo%vvPWf+XD zI~aQ_v!LPJjfhn}jG+D_Yph|)gp|;J^cF`CGDbz*m^WqMT#z&pJ_i%7J9ekILd4p4 zpKviI3~3BTUT!sY&97drpL~HFixtNL`Jb|{W5&j1 ziHg_Dm4nVhwCVo(XG)CoqE{UB-ViXe!IBzrN?i0=N2WtiEE+{BSYhQzvcob>2c^&C z53&&TAZe0OZ*+aI2(?+$F{PHqPBD@rylK6B!tLPj26FysF-v?ykCc-=A}oD4{d4=QW9sJ42xn`iw$kZ3FhO@Za?XA=nyU4p~X-HIK zQl}Tv_y&%=p{BG__B*6RkObZN@v{y&JUHCy~xKO7JH2ff0Wc%i>7nYLhSpa<_Z*mNkX zt@!Q8Z*$C+=k@yb%zdjPy6<&FuL$Kz)d)6wU|O)Ip#B(BgX0GHNpF)12N!l1XxEl* z?oMkzpa^bk${G&KdY5)ye$G)M;o2w_#QY7z<4jnjMkcVbe3sfyVvb$}`b{;l z*SnF!T`BwN^=^${d_-KvnA?|x&?ChuN9^_PUVlncYy$J)lUeOmmKGCT2p&{kuChAQ z=q~(y<;z}e`q~Jv$uF0e^jK}%H8y$O=y=>)g#DE=tKv(wrF_H*RDh$*sx0{oZCHuR z$dr`2tj{rdS)a4?}7%2{mir|zgIkMm=$gi;*LiuF(` z^M12vlEv#?Py7xo`_RM;18ZV5LcfZbXi??|WblyxektU_}2Xcfkk_BkOBblViVmK`VtKvg3G3Caf ztNzIr98`TPIy8GCv~5k9dEm4=#X>osxqQs52~H5HG-lSj(Oi|N?7}y*5Y;)Q$t~|n z_?x-e3~-`y!2?rp6qj1!=rTA_NNR!3z~`fXt60_hN+q7m*A$egL_)|v3|Tu017Uhs zxNrVj9wwsQa+SIJ@B}ny;*&h>TVxpJ-iFFMR@>}4Rh)&{kGRQ2iAa$R;dapKvnMth zj|7z|hia+k2$QqinZ)@*vc@9EA?=9Jam!(@;yKN|C`oa+HOoBnq6AJRnVEp`XS@d! zqD+(N_~3W?rG%4{I263tSm#GHoU2uZqe9U{t$Q#>YKVaE(^vu{m+$!gc%iX*CmL_>KqXfxMV7XoiX*QdsKx3$9`O z2U>d;b2hhuBXn%;;kgjmokn`aTA8~JP0^~Hz=23Ss#H3Plrv%uVrpl@eOm#vd0g}c zEdc$7oCE< zdYC?KBzB<ZOFov- z;2iE4Rx1@^MtA&^;d=3gp1_=GZGz;VQJ;O13C>A9u+;;0cX0}u%%;0gP%Fe^;!P+a zSZC1nP?)zD0^k%_Pbi2d#U$cOP-B!@g!bb12@*ovR|Pot&BVLn$P+<0q~=K2qTV`1 z;WbAo?|*35I}LSuC2NlzVP*ftc~O5jFK&pYL>vwl?iWfb#~z4rsJ;YhV;&I24N~b! z%xO$Juu&!13EZ*AFUYQ}WWP>Xd?GdtcEG7vU*GaZy5)~ZCow^892<5mJ&EqY+$0*j zP#Uf=1U?b1A)J7NA0t~g{<;7tm5zmyw30{(ye>)4l$18SvJ7z5t_u}N-%#Ha&y{Pg zGj28B;3Se0sRLB>2p>u1gW2WNoOKrBE=5X;aRZ`JTN}peu?RsCulBHK9y z!Cpxjeua1=A=MV|h*drWVp<#3(APp6${|6zJGn~oHi9YT^_Bi~+Shk^nX>_GVnW2MsvLfWen>|4PL%{z}bi`Uum! zVl_b*(giISS04nAFRP)2TKnC4@F38FD~{P88tKA;*1oLmDfm4uDgummB;!O$E7wGC z?_a4(CcIOd_ak8Gu^5!6T0G=?zngOx+2849e>8y-`$zdHdr zENUG?JYN=zqc4jA9Mv%kMMj<8SqYP8`-lj9aRO4yaOOD=5__~ zBcO_vcYap26WsY1H9r3mgh6;!%~jqGJ9u?|l8%?pcDj0&AdhNX^8nrAHwctvRmSuJSSpPx=%Sty{(4S#31((;z13%~7zOgHXO1;PW zZ*rE;W3FBJV6;*zv{q1Ek@A*u@_61gfY)5*ne#ByingQ5vZopvG0>*#BM7b**N4wMhez3(zJB>A4846?0lc5(D*{#3M6QW;Yg(+D=eY16QAC(s=_=rs)2Mp zN*a*A@RtY@)`d{e#yXR8cKQ>xXa-po@{s9=>KabP46K&tLL}K+BKA?tQ-y=CK8=I% zqx@tGI2VAXW|o^;8@nQUBD3vYwv%1R5!tFV+$|D9$n0Yo5Hw4vjbJoO@6v&hV( zQx#@zcj+R(_v_{eHD7LWomUh;Kf3E^BA+T*b@Njzqz;!&C0}lKxh~F&N~sW4lihfw z6rHJ+O@;zOuhk;nDOJ3&Z*d1$3|=rOjkN9LTL=t3a zyYFs$v$iR1hjtMwaPXRy8Q&*wY<1;rJsCIWf#qk7gQ8SfnL>IAUHQh7Wu6#g|N<0KE}6W<0#Y**eA(}FKf4ij(kb)rASK`^OG7bz2524ocH~3 zc-qxwXAz}d?-XD8_qoWN{hLtB7x{k;wJPRR$#|l9e5Kr)^0mf!+_*D+A~TuEK~+02 z(;@q#jXk2ON$JuFcY4_$dbZK{YRB|#f42PGHf+?Y{Mr1k)`4q=UM6H(&BP|$%KgKe zU+;`gUz!Qv(;J6=O_d-qCVdmEtqm~w6}b|a48I-nW>SNT%WY}9p=P?usbB9{%li1^ z1q?fOV@k$tdv(IiN-!Q<@0-Wo`-XFiB^cf=CKYF4$h!WBYKA(Ub$cld!=U;qUSJ%~mswFlJSR1_w@jt*ZJ-pB4_7 zjp*ZbMLmYGK(k=He>b$^I5wqy?bSazr2`KgSg#^ z=^t+7Cd#FVs}|o3A)9TgP$>aNT`I|aT0BdW8#)El^|p*(q&xTyAR0|-IS=JAX6&ND zJ3{@e?;z?66Wm+hEOu;t>U!`5fVnTIgDrqP7KXj z$a@VcNP|(Ph%QYzo5mXN`g1W2d%yR98e^}P@(~7UF%;yOvB^EiSqd@)dDX&Va~60nIjs0*&dU4- zU2mGNBMr3d4n4SzzROrxev@Rx|2udd{VvJ;!ad_ST4h_3+GQ{zBN3Oyvrda-u7N7^ zNqnJvwx+;dUJ4RBf~|OZ)=RY44M>yiG2_3+rlV@1ZMJb~ARO6rs(-PC;1N_}CFoQPFQXWNf8!~PTBl&1nERD>A5^KvUHk&!y^$q8Uo&jS7&@ zxSq=k^L$1?HdbR7S!PSJg6_bo%VfypjJgcbtU3R)<Z4Ae$xYDFthIz=6VA%V9yf zRxKc|FtJo;Z#U3H5w2B_4|?vb zFcntX@+6FO)j|O&lS-9VWKpZaK28pHg3ms&IC2z9P4R;dA6&M=u<|hn%Q}4?+WAIS z-@1_Frn7Ib+?~?Z(SEQMW|G+IHRL<~2XkKrHv|)g4%M!>XGpOpUK^nku>smAf9!6Z zCzmRlkzO_DpFw!SBKC#Ct7+)B1GGvhw*b-CJIhOLu7nz7 z>kLB5cwheW+!0RetAx1H0%{hA%)Yu8@Pdm~USBhUxp-2Gy2*VV-rA-Z#hY1UxjNtAQ^#@*RpoHHCWYXt;Vut5t(j z%lA_1bPErPMwPS}%(rTf5?%QlZIp0hneo@s`dBOgRo1|}q?%egF_YJMH$ZsRZ^Wu$ ztAD1`%DW-g#+^+_9F@Kt6fZ!O^lORYbkL3!(gt|4dBnjj_wh%}qy(u_{3Xrbf@rF4 zVZBlJM=)T!MWcW&GG!mJQgO(hkG4X^DAw<09sm^e)=ZEvZ=gUD^mZZ}j?D8G>kfgKj|w|wt#7&Dtm)j1K7R!a%x#m z%EWu+?UP#Tj46K7;%d>9w1@>PthTmG0~XN|Dg#lhDY@8dvpKLXE-`BvunIkAwikKZ zE$VP>kS>~dRhH6KxQzvzB6aFOGSv1u)!G+^=U0kS)lQ`?{`V1SVpa8KL6oK8Y;-e~ z6V4qJCcV;kEeu5lJ^|{{A-bjC6)F~3qhiTzQ#tM5A-UKkcX=JfeYEkCz^X7Ai-a`o zl1!;J=v%hGJYd|6cwjAHTvawF)PnpGu_05x)iEYSM%-!DBHIE(Qnw%cc3A;lEzR0w zQ`R-CZqYBOjZxx2*%s)=oo32e;GiwvaHvFa4B32TblnBnSV ze+^~oku_p{hC097`DW)O-ResuTfOtzDpl8b`NuFo#K7JKb(Ywg7xb#ad_T0TE%Uo1 zekI>(jTNDfQbVBXE!WoL&rye$UIJTJ*Jt@jF@4=k&WaDx@UGvj7oK~k3JM+h5LP%K z1IQR&@LOpi)nup&5;%WsvP~tFsUT!P_KDRa;r@|pNi8{@mDZKb++w-XHN?}CRB8J8 z@)GS1xK^Gk9r0bq6iCVEfUT*>^ku_~~&^FxJ2F>MW~ANJ43_`=QY zZc)!l+Lt9#s*`VmLtJ-$U}`u!8XRN$q|S{hp8;`B&Vb!1rypzs?I$C5@}cZ*YxVmW z0wpX1s?U%=`pi1{|JgYMXRp8^e6mmlldvy%WlGqk6WeJdSh5LLRW_NAZ3P9rCbz8S z3tWR+KD4y3hn^N@t|PFtsWhw?Sw`eeR5$`oW$2crMzZUZpUwo9mc-pHFlfI_9&Ji( z`5i>qv9QVxtNECdx|%~Kb0k5#wB^`R(@9aRrYAYk$W)BQ*mdJ5RajacuCc%tJv)bk zzTzD+h%I-IBt*hF39}t_g&fo2zs^U_jpPjJTetR;H6@s4jdiexE+^mH>(%0gDbZlM zQhk%Z($IZxoEuS9zySN>Y6~>jDN=ik}CS007Arey#GYpKahkR-3torPM) zB?kz*RJRiZJ#V=tf#jb_?#q3YW|EPTiwCn*tSgB$1$$ocDWOahU8J@jmdYw))0-+T zT*H}?VhP9mk(#MX$%wxYrUrK+TeD?KDUcGC;tZ;@J>nsRYqnKB&d=2X%&*DV#dTQ9 zIuJ1}qy>VG)7fBgLpkDOpu^4?Nrl!h(DiVaK*05j#V#$@svvD9{TP6jje++T)|ilyJ`p zZ3Zw341Zg>L9UcA0)g<+RLun~8Zq;_@wsaH`Dsio_*9ollqyp^qlFQh$SuIKMvOe; zdq$>e+VfK9B2&f-BFH~8%`(?ngR+U*<-Z9u%S(nnq;M#N>}488k&Yqk1G3BJ z(+MRd6ROrmBW8J(fn8>rcXcqww9jH^UmsJIE~3Y$8|*IaF_L<*n)PgOghi7MNQ&de z9Cc&zo$N#W;;89xI8DA^W_PKjyHj&I12b}YsO3=P?;65b!-+f@ye|2`^r;88tA$c>Vky( zfVBc;eu`iB!XU|rvz-ZJSt2nXlm|&AHARmdIzdwRv|(9&KvIZyU2?g(YOx?=k`e7T zps3z96-@;aslpP|)y&t>ARkyb z{(%<-pi!6#q)QD&3Uu%Z=GehdD);>c~4VI%FQvX1)N^;LGkqrs+-JF@AkQ*99btP>L8;v zMqf2AoK3iE0inW)&L-zm$W**M(_j`lWKuK)#l7N@DHUwv&6XI{AHVN^pdF5CRh9mk zGNFtOf+!IfZl6fQl(icUr`nY`ckZkgTiF7?SKAXeonJ?%P$hBzV-F8>dSbNk>55s9 zubR|WHzy2NG8HL$V*D&2m~2O|P7=;_M_c9Gc$?NGB1PbyiMtvf&fzNFTW7?ktU#(- zW_eDj+O`}x=_3^oNDg&MEygyhm>v-llj0oZ7e?(tJVzsk_xh-Ix--<%w8OGRQRc@2 zhuD}ziIBex@nCKS5n@_t5*d!cO3id*#ZZ{Aa|UGWrrlFNr=;kJ2`Ofr!q~RSbO}25u==@p|!tE|`i>7$c4IxyxR1x^lpFX&9X4 zph!pJ_p!+`&#d3{9O)!*6LG zJ4@`F9L9)7Cn~YVE;^C%biAgt8kNXQhb(;j@3>6C!6A`r0g0`>y>E8k?1|Bx6h*@B zVl`~$*SL+fVkLfDK@|@1jBdA-GdG1ScT|MY(qaH|wv;&UT5;#a?%#H{^OanrJ(PJ! zd!;R!4k@~qjTCdLS^;~_KymiuDn;_6i$R?r1HfY0&4a=4%#JPC50G9UXLmFh+7u-+ zzypHA#3i<~guvD_6EhLL$pcB^OvcFz!$gtD0$+g=>j4>U%MuzTRKmJk(+$Efa8D`B znj<{IT=>`-&y=8`D#(LzeoCAGqY1$|Rge0hZz7B2ps%X3&|)jcM7xNVVB*g@pOPEp zg`~OO-JX*TDY^-pWMl&jX7~De!)0)QBu30BDlPtqtDE>(!XN*JX^c`fq%o$g0}>o$ zYFfwh!5ERwP-VI?tvT(~&L*8#jCcn$GaL!9xVpY8-I7Fr1Bw#G|Il_fmzQYw+npT3 z#R*F_v|Wku63WVy3D;ldKt}U}jmzRuojo6`H8+O!t`@83nyXa`<{-|=Y8WYw1f~v- zgDMrYYw*lf=z;F2G$uMd!U^?@4S>i z13f{&7%nmy3{p8yAaRaq+*-#Q^&8%SB804QmUZ3lb#+_HyY*hvl!_*Z>dVj!qz=g8?cpI~ zk2O=fN4OrB!&dPzebsbp1NShrV_yUt)_Scb*LaCSruL<(_U9Gw$okp&|a!{V>+e zIDG}$rK9go6;wq9QaBF*kyIl{k6KCczk#&n{fK$dd|et=JQ?|$O$RMgHDG<^q3pi- z_Mtfo`B3mF97cV(!Q`Bf0j_~dM|-g8N6c{g7{(JDQD#zHR*o=Az&N5H!m`6mI=-^-1PH#g!xv$Vv|JT39kf$!{Yz1r%2 zv-Rrh=ihGc;5B_){ld1Lkj-!oCybd%^z9`YVjiH7L!iiaz+KpIc{UeffyFeiPrP@x zgb_)n_0u;mq|;M`yw?brseC#|BO1s^tCMPu70XKzEppQ%O0e71dn(khiqaVC^_Q0( zJV4m^ooWvr7iD$(ba=x04=&clTtg?BCvp#XxXU_h2AKu7*y}2|?92TAIH$TeiFdz! zaB;WY3Of5}W*(R01X)-XPc&coxXk`+))E;B4rmElOUD4IPbQ4){@2QbRDSq#;e#Gq z`=qw+TKmZ>TxZzaC$pwPigI-h=4-Uq%{ClIsV%lvPGB$mHb8dLdv0JZKO3NHbr8G# z(|&j^e9}{mMl^p#Rr1H;Cp}i%-)$d8x$=nh7k9&LIa*;*uNte>pf$iCg` z{>P7N+|l0e{!;#3Tgt7&E}g3*^2Kd+7eX%KgaPv3!Yx);Mwa-uOWvPd3SEWUf`+^? zJ@2DAK*aBeau>JlT?Du|O6`%~>n9c(!4I5WbtTgQ<(721cstIh8z`Cl3(SyE?6jM{jXf9g%&mUwrir?{B@*0X z=SWsFK>I4U;qsbOR$^TMtz#`7D%|7ej&<|q)(=$2kRyXq zi!MfJqa0QdvUx=oL|?Jkgm9OthZyj~qZ1rZGwyJ5AECE^nYk`WzKQ*e9~{JU?~R}q zxDLc(MfE~zu;M*|0<(iH%uW0Wk5`N?tER+^Stl?vOUva!POTO!RdPmE$u~i#g+)Z= z<8%D$osc;j+S!D?$cj#@B}nK;N3cI%c4fs82$7;A+2Y<@Ucxy~o&?C(c7`b+C191? zT&T^+` zE*8dNRC2&)q<_8hjGjmbXT2BDW;I~{8-?b}>eDlecne6+IzyTR>;VjFSq485x0t*? z1wVf-{&{|K1|=wR-V!$Ok=Q!2y)z81Y=#9VP z_DFBEzA!t(Q9OCpc|lbU*;fdq!Lya|a5NjPWV?_-8LJz|y+8Mnsi)_sz<+T3JDOOJ z#*^9P?07W1Yl~41u~$Nldw)5fx)d7eOb?2eB4ows+q8XQ@SA>!=Z9%LcN#h}ITV)SJ|k_Mpn#&Im0Dfnrmf?n zUX6~9$t_g9rjS@s2=SKUsi#p~KN}U+hWlbXnLg{>`TX-k{wKV9I`{ zFJ8Ue`}TkDyejVOzI^@7?hbvoFF07X$DL{swu6)y5`5=Y)JRpo!jG2RWU1;LB;C@8 zJ4*@ z<*+-V`byi)rnQG+UuXqe@@3Q4LF=wO16S_DuhnPPP9t(%yZ_Jf4k7p8Z+1t!G!E*P zDIl7Uq?4-*iC-*^)j1lT(i3V>zB0vd`FfYGr2>ywiMhXtE)yK#9Fhn~?8kiCv-KYX zb}V>+VPF~A#>mib$9z=c) znFl^;jI_b+lNWRCYXSrOqZjB(ea6E(4WG}A3u!B=_Kwg`JKYoVqTR1|U-svN zV@!9yR>4o$Zh9!}mk=DQs7M#x%GJKd_HWa!!ij=%>U?6Q!#V+JMMNBG&1>YE40z4Ns^s7!8ohSo&nI!An>7haYI(MN>AI?9Xe#)PQKE z(S#Kmhj8JYD3vini(=_go)R&0H-`!`LpmkQ065D5;NpFC6$j$4xDwY^S6=erL?6E4 z!VU4Ub~<-Hxed-5Lw)>A!}{vXkNR0pu$ry}1Z+uRwrf?MxgHR%Pr$ zDr~eRY3GXj4buml44XD=B%1AB0l(*#%Sduu%{ID~HX&u+MhgBC1To)GqFgo?k&hA# z5yn)=J*-1dBFlUx{b8UgYwarUQ~?@Bh7jtQ>|)AviQvXTr=7gl8mt^Q=#!JI7E~rt zwrWRfhDHh5dHH(p@7lfzzkIGxqRbkDn(^FipNhJ`xit8ionqC;xo`oAn}|nZiPEe{ z{rJKs1DE6=>yrN}WJy9K<*HtzAf3)Pp6iudCXz(R_cl@(Fe0a^LUO~g(qbkjD^XWd`0M>S)O>Yn$pvB8CO6V z4Ss2%*qT=AKF@_o6W7l|`pnALKJ#o8hi=pq6C%V8(Ev7rPAQtc76WYj8g2?N3qP69 z7;su2>FnJ!Zr}{-nfIt)Sc9e5d@<_G;Iy=6)oErmoJl;CT;)R((?h#U|&B{AIqk6(^{9wHI#pB#g2SB5U)X69tAVDj_Sb$h!%@8`HI z_%~-?^AwgW^$$hGNY4u&{i83=N=F;AF*j7KuIlJ@%7Vl z$IK#gqXwZOiKOB~H+wJ__DhpbTs>as=#6$<%HF0a;5x7H@a?BpYC!7haV>&*VWB77k4ua)=GM-X~(rQ&aV-CcHp(yPf%=tn~X2snk?Vy`CUEq6O! zD#Fj?|It|BH@uZBTi5e3J!J$HAeApmZavAm8o*i?)oP|)rvIB3WvsJnd@=fvmY3v_ zt?03s2QVs00pn45Yqhkzj8FD0iM_!m6=kTZ-BlFqmRRFN(}HTmk+}yW+H4$@GBp(g z@b$7vO+Lc+tPT>xffc%&l3+SB%vtg*FykNZYA480zjh&QCAfPePM>u=rL*Dzm$K+a zYrCl*&#fYIjEy$z{nDW_|DveLBSBH&A8h?s6xx;B69S=Qq=r6aIDMjSDZv!7+fy#t zwP}rQir8ga3^}NVTF(2!{`h7@VPnm-uVJ>yglv0z;9|Kw%UUS&oudNjsV^STPMHk> z<;A$lIF39#qdBELD~lzn&`T5d5wgV9cd`qVd=BrlF?X~Nk+@D ziS6C&S0#6NH1g)&pqePW;;x+sNnA$8)FL$_0Aj!yE*QA}cU1G8 z)+Yx;8fhGus3xhSgW#H;@E0G&As1&K)PjuIhFF7{9^;{{x=a;2LRUhj5HvGOFWPn{ zbn? zUtq5&H$V}kI0K(OSAdDUb=6HHN*?tONo;NT4+809eNe@*33h+Oi) z2Jdser~m4lD~e#LH^`w9pduY9#BHmH3X+F42+1?V3~Pl)9!B@Rd@&JABipzJlEzoE z)-ahJo*?0fdyTRquc9+XlNvFD}Muk+ZTq{z&ru^GD{6%3@#2d&07?o0S#g z5H-&FBQuW4Vv?ZIDnxB+$F?)Slw_dQ@mHKd)VWBAIW;kbSDtnFhJu z7|ys_&F{)EVQ&N10Bx~bs7dDa;Z4`t{Tv+3Yjr^$VH5%RN* zAAa~@l7P^PZ=EhQ|te>!N2i~jY|}?(jqP;d^3_ISVmt12=HM2epHH< zm*hmD`?DHecrIHCSyh#|z{+w=3rAE_1iVdxYGmXZ%d`EioiM4)2ksG)Tcy+{qoXHZ z3AU~>kL;gNWsY8Xc0MJLE7!|Demeo)oM+Ii-xC%WOQOSE==(ws~gp~&HK&cJf=?U?T+hPAQe z%4Jl7!pRxc8d)V&VJchjb%pVP#9ZWx>(kO3M9zvsW^JX6KaS`uk9Vzj-=BIOm3mNC zQfBCRkN1aT(FpnKy`0XIUeP<8?j6wR9%OQ>*Bg99dEk~l6?~`~W;&fr6AW`7{7&G{ zhVa>txi&S{_f2z#qnDPm=s+^fK#aAf6IABS2rV;5KH6vwOY_z1%-+I!n?^z|C(Xvl z-n(U$AYPqSQw`dse>JLM!miAhY=0$Cj7L?oKMY|j7C?Oe%jKn&i#3LwZC4ObgTH^C zyyA<_uT6WnV$?QUZ&cwfE<6t^T=w2A9+yc|D~TpML;qI5*GOTh{T>svLS|Lu8MXB- z_Jmw11unZCFQ9B-i+(AQQ**k-BFpOlVi!hCf-j>ydEpPV9)(OMHp)23B8(6pNO6-2 zLlZ+(@>UrtsD@4g!wS;8RB0qa!-ic-`A%I>=gLjuAlGn&7FQ_Iky;W-nWDGCa3yFb z>H9N!Hu|-AzoLE}lPXbv;`cJT*2&%JKPi46usHl#e^kE`D?q$}jhARY*+KL1ffa{* zT3Smg4hhgJT~#_@6d)3L!Z9u{J&)4&_*2}PZRH^`(pixeS)c1R8#10-DSF?;%%NO$ z!Qq7+CxyaTmr^ZDSrg|KW?75MBqyyb#6ywNMa%^pKu2vf)^j;+MdHUq^j3kd6_n)~ z#j0%nN|qUg5izHcDJtzh^R&Dy#W72Zm2A8;JCAF@GIvKf8P7CBt4_ANwutAqtI;-qj`ECp?Stod&TgMzR}&ZzfJT zV*mT+BVEj4s-$^#pir{v)i1Vcv=68a8E}hAzc&K~d?rCuS~4t3$D(XFVhJPSz*3%V z-2nt%p|4@hd6SZN7e5zRn<5OZbVd4IUV5qg+S_!flmblqd;96dolK%6PHD+CU+s45;exqy7%9kth!LEE_g8YvwYhSiY;HzpVDh4ueT-=}} zf+OCrlWCYu_48_%h*tqoQw?iCaPTz1l*}6ZWlcD#`OD40RNC2P3%sC3Fk4)Z!|vj8 zsFsmD3=qz|hFQ(WuMOem5FQ0sSzVx}*KTVs*#LPwRW&hb*#$K)i+-vomib_7e_Zs+ z%39OO*RqWpd+I;CC#Uyk_e`5#;HkfeX4)&G+P=NjRdd@8tG@r+uR2R7d^hO6-Wi>K zO>027gPBP54O9mR`)cF$PVq|JF;0)JXBxEtCX1{Y{TexONKK3{8BWVEMnT3S|EJQ@ z-5b!7GL@s62TRH*^S7p@|9M#1;_i2Y)UqloaI8rpTu#!}%oC`5jm(AR`calAhfbfD z?_N(Kr(eMqnLu@2%JO~C3X{f_C+3BejnW1PYE&Ghl8GckC5)b2x$!{}9&S9-D6hSYdNmU|2(R>nzDz(7!qGI0!6zVbr^9j#1OL88hOg-`Z{hY&?6jlHsE1?Ye_xNh5{K?<+Z*lgdfyem-Q5gq|X z%TTD4XADkmu4EwaRRqWhRs5dzyf4723+LF$55@Zzf<7A3oCo3O*y>Yv67o}gL<4!j zdw4GDgKNG+E=98fD71Io1uGA%Sb3lOBkU?ttb>&3nr)_mg?3osoXBVysLidCK{bow z#8!+e>Y$w7Z#XsU#B21`p22GFbW4y0)5784;*I@ig(BwTt>j!M5AR;>uJ()!ttJdp zMy)=)Rf}1RD&f>}%t}tpy$KY|MqD?_wkZOnu!{4=yR56S>G~1-;pS%<3jHRO9V@9y zKSe~o8wi6V(M*KHX+B`26Jacuw;BwD@;}4&2<8BTxaj_oZjOL1vaK1{sXb@wQKT?Z zi2{YsxdjBG^_$^xw}Y++>63iOLn4ptDG^D$Yfn?kq3$qCSRH1LecF}h>3fl}Ypj7U z43%}qqE$SEU{KuS1sl+^HHwnkHClwT|YoUh^{tZF|RZK0cRfmW2c2ay195Ek531Vr7n)2V$-(r8QsgnXc7 zD3O~E)v(+;*I05%@)K6lnB3Xk3|11=*-$r=&Cfr7eDD6|y)MW=t>K-WP0MfD^!yf_ z2JC{HU)A6)gm9_!keW!?Y)@H+5|&)LreGcv7xF$rSc#>2bsL?;Xs#n#N+UwOR!v^j zXrhtj|2n<`z%pcDI-}`*LS^u&@@w6rZQ=nGpt-vxnt?iUK&HzHm8l!kc$Kqe@yMDy zubyVgX0FKD*E?iF*$qDabV%;?_1mQw@%%*kl_e;pUai}!3kgiH;ss4cDe3zWB-c4u zD+b>5V&Gd&IuPI0(Jp{hu0I%b%kykM@I_Ys7r^}`iVB!Bd;*gyX4ni6b zwN!F;Ay!p==~sy+=cuPNoMyNCSjDt5Ky*C-FSfiSmME!_&F+OUVRFI+Cb~|^lNXl` zg8bLu*fG2|yXJ2A#&1oE9rG>J{-THYE++S2W?tHF!a-}Bu1$>`@N zjKJr?-2PbmDE9BL z9K)37Ay`q#ZlI`Z`?!ZvWdxxs13Ip`n@~|Xc-n<7wR|qB__yy zU+BW?RH#~1&ud~bT`X~-i~BMyI;0CNjOO8G#swQi)y6?#UJBDG#Q=km?xqzdAPdcl zHWiMv%hOJBDVC>QU67xE$!8KyLP8#oMHvdwJ$pfd0*d%>JU*jXm*qU`eAcKF?oS9RNpc|C_;OvQdc=zV##5*R zN_tx#stoWel)Jb3dF;T55UjgNh)cWOYKB z_AloBJc%Ub%MRv>l1^-W8k8ecZ7-L=rrj{Qs^-Yv=aZa3o?ae6sxm>AvJXC~$7OU{86Ix*5(kQ}16%79|arQIX7brTY`z6B;#3G3`k7d9!{-#ih{S zM-9g-rp%L)&855KaOMfytW|=S?#A{ze$4@c09@s8X2+ejikf9NJ?GfPPWtC#7-{$r zZ^|bH_Rp~*3eHZCp{ z)z&LjYG&`%^t+EyUM)o}G&EiiEvx!}s+tPF<;p8-1pkat@~`k~aX77!O$^~`MJT2o)R~mC;rCRufDgoDVPKMJX z@`H<(H65fj-#XLunqV$9z%=Q!Id-cvYYIzWiuXy(1cHX-LUbv#ChW(kS2XO)^f!j% zs3{#H%UM5+|BtP`kI)0;A6z2e3>h=ZjGT$_4R=)rtkNs-wxgjZ5>8f{*JWL9?w&F(}( z=$46OHdO{$dJNN`o-wuI?XnyL6wCCtL!*-OsVUjjw5ma zK(c5(VilZ~qPdHbe}rb_ElDt`72#%#EeR#cwKHW_hi4iR$ydm<8DNsB3=Mr!`Ll^o zjcC}Nhu^VCLTed_~wu?pDZ!v+naCuTi5IlkpclucK7c) zmv{dplfeSJz4MPNCxazf%0+-p--BeNvHWfjvPo_CuZGbdKdsOj>q=mq9AAun37`^; zT1IYuH%NskT@9((=QI|ATgZ;txCpsDjdm;QuF-RpE0NWR&WT%1jmX-jrPwaFSWQcE zJ5`)eA|sq$20j0|*`FfQ(J}9(<+_ImMfCmYo)%Qd($P})Jyao6j8AT8Hwjio1?EI` z-i6`9BhgL!5e-W*v8rftsmeH^!;xDUgUL1&%21Xm31w)m$I*TZJTZda9I6d!ekFjX zQ27GtxU5R!vPsk?7*`G{;}wG>{Rf{x%IqC?pj?L`QQ;_AL1QI_^!-#IAL9_kDmnxt za7&({ZA@qqmFzWv=?KNZuH4lQPa_(bF1_vlKR<52Ri`_OCb{b{uPp`XUAH*HyXSL< zw90^uT(K-o8fH{1WYQrBcii$ENYCAm_*xw<#(k$(mjjO-E_4FXuIPukHvZlKZvnJs zrb&m0<5q3Zh#gb+uY`TU-X~Qu2C({hb4n0_P#(PWn&Pn9nB|+swlR{c>vl1sXkso` zY$U!Tix(BBChexHoLY%9*(TU!(HaB2Ltbx{?Iw^ljEMmR#a>1x*&KFk$Z9v;sZrb6 zwSeNijomyE7vsHbCPRm){~u3+3*KQ+#a3R1-{xM#PepVk+NjFgWF0q;8ThzOOo_|&qu)*yttr6h!lL$lcCU(WhILb{5JYUw zQBl@yaeT3R8#8RUd2VXmDW?*_HdBIg`Cwy8Ut^Sb^?IkXv;F$jG{5B)!D= zhjIv&3!7QM>Z+|P9NHZ*y+}Ds5=&e!YKCeXTxO_5EUKiMlR%g0G#fq%R0OLuq$Px& z&;&CFqnv*svFI3nd$Ru+mFD!4h@i5Ku41UH1$WHqGp`tmyVAo=U{A{6$m@w96Cj%w zD`5o}7xXqs%HIt%*eFul!XQ6d7(hd^upSYW#us5zP^B@6%feSJ5M6R$smBDtq9IoY zcp?h@We;aj0bu*8>G%|t-(=NjQnpbcfjm2h>5&2(+mV*oJYipI^?LdaIL*BZ>;)+-fw)+Xs_NI<61<5i912fRxJenU*`R%V?Y-T zXn^O~tJKyRH4f-_6N(=b>jiut(WUp3&|kKw6de*b;0zNOAzn(FOvfS(Zvh{c!_O^N zR&}(;cAQ)U2v=V6;OHt<&?s~U4fGa{@-XP8K~3;^zkYRnmLMAz&y7n+x6WWn235*m z`5^bB*>y}onZfCsgmW+s{Z8PiGjD>Ak1pW%wzmJHzj?p)?qQ=?5{PG{!vYBHlb{rP zgO}qbUF~f^65(tR596gQolKGvzm3XNc#dVkW)i=8U1igcG%4?X7q+nVa(FN}`a`DE zXy23?EGqU__@C$I)cOyZ)RyvIimkHm#Z&^ACe0-(WRDd}KeQ0_ZDWK2x&yhOK93~u zCkz*B@)8{qZy?gO+-x?$!O^F|7d5euL<_p3F}8nr^K#q04)!oq2;yQMa<_<8fn*D< zvXw%HaadSkp$lV-ndk63c#dwbOC^p8O*mk_4oUb* zZ}1{LiGDJ4n5v){EZiQorMKEqHu%OoF>)O)U}%=GHXY9L|M)&sz+7mtdQtBJz*>Er zjx`0aZLb*sg;_?IWK&r;N?6SC;f1O=B;1JAIZVTH-;<^|x;#*&l2VEKB=e>7ArQsV za&<$TKs~kCAh=-5wQLocMiy?Po)xK-aO7%jE8dyUNg--&6I|=_bBHytMnA=kX<|-+ zB?(T!AkHaBenbzYMypC*Q;DmLYkFOf%F+On0zK9FDjWWjxOp9pXJpKam7joZK2Bb&cjU#2CiJ zWO5_)35HiBLEg5b*S~}m7|T{=;wp0!|3|ML`ycK~`(NTEu<7W$$<8NdZkjXN| z#ylwHb9QpL|H1C%(b4eY{J$*4ez^Z%2UK%8C+hWKt2Gy{DdMskOoDVd7)5bDj+kwl zgpuUd+?&_$c0^bpJxL@>zJS0(^Qnx4swi?(5H&uoq=4jbE<1mJC6WG!(BQ0PtP~9A zCT0xFI)36=$DN3_^Tw;Bmiv2xUJ+E5?hr_Ls@|`bcL{P-sc@L>o)r{}EHk>yr_Np+Q48`!i@Ns3 za+@u&&Y0{D58zG4zlnK@m>QxjjGmH@C(CN!W|jgQ;?pDDD$_W(KoX0vNZIUzwSeYxMrG7%qAc_H8TxJx$pw3o{)Zmq^ui!QI> z!M)GRi;qCxP5sl+KW$0_hNY9}a9Z7uFHjFWjASFP-8X95+c8h*M5p(rHwd`t>XoA+ zCppe}7H<5)(UpBGCg^=tYp6?CL zLM|3*98?t&1_~k77)WQCSRsR709smT$P{4^uhyjJVzob$B&IxuSspZrwy?=Jg#)T) z+w8Z|%|jPTC!erLQFVw+#hG{-v^8WCO~fJJTi&GDsA9d&FA5+!=+cX(9xP8pG+cAE zRGga-Q)wrh35`H$1xjrt9Yhs2#*r^t>`u)$+V) z?ETGyLG1=R7eVG9TePrZa0EpkpF1?qqok2KxMH{IaCy!@FQtU2*%DG8v6jSiW594_ zQk_sw)xQ8`?1Br@XYFLDE}Dre$V04bYW7n|GQNQr*0z2OZSTjBtgER$A+zI@MZG~r z2O~OS=IKCgjns)j^HK;!W|#m_~~TLn_g$RDqHa-hSXmpa@1VI_WZQnYNgWlEjBngX{|ChaZ3XTb8N(hHq9 zVD+Q3=X3Yk?GOA>Zv)4U!=rn9P=*l%~bq^&vJTZ1JbJ0cX6TEF6iv>-eA3~l< zm#}w4id`DZCC?%}h%ze#2Y#JtT4LE%>_)Z+CA%A@pJg-s6y>(0=_7dUTSWod#q6nG z8;04Hg;ka9^y?&SzvCA?zVf23mKE(pY-UQ>sxT zd*l}(yrnnc>5=4dI-)HTua0~v#|BJ~UyoH;aBipYbv~Gr7;#fDEY0hPpLC$o>@Ro8 zmDkK9OR6$F6DWd|%q%ec9DyL-@kCJ0v*Ww&d_+%q7$mG0t%f#fByMT2N%!AVguynzjZI6wi$lge{?UQMsTyR z^BkI{+V7|?A&%gUcj_rlRU~Ni?ZGPTH&60aUlm&r6EKCY3GE`L%v8WCKlXbe%HLkS zemezE^2>pXs#jv^9ls`w`Ey2@@p=AVj0r(9HzY0zV>0vw>-Xz3Ro$4hQI&-96{M!3 zCfhd8xflM|i0Xqxi>f=vr!)(Q?*s@B_;-uW#);5LHq6b^kflgZCmBN!VA?<)n-eZn}Z{y zXEU9feb;Tb$xONJ8ym&yu+I~QGz%Db8H6XHNa1AB;xyriC=&*7ZG_^x+~ZO|fU__s zZS?gMkV2>T8&A%!_n~Hezl{Dzu593a1C6}0muniKf?lEVaj@wa%K?sImbX23(rYzS z3bzS^&!{0KomVqZ{cN|tmx~z666`#|T%JvoA}rXv@u;DmiJwB|A&^>an59CvvaJVK9R-lRX#}Sg*s{HioD?aAQL*jUs;%dBme_`CO zV8k9`93u*EixoReB5t}l{R!K+V$b6Vk~)k7#O@Sf5XIYzj%wzvifg{3O}Y&7MXyA0 z=-^4~gV?K1>8J*%(PCLavBzG>v%@ELQVE@l9&nWjd*U|eq6FhL)zP>3^~I% zSNQ>A*k=A)NP2vvr5SbjmvdDiDbc4NznsgeMM-zDZC7nq@7{S6uD9MrYnSYT%h4_v zv#RNR!xzm=!QwK6+rIN0r2yz{#0YWcld}ybAGfj2FBS5yDrwuM&^lJy;k%T9Y=ZJSv`p+V%*U_UxeV^LjGR1 z7>;F+;b3YOKa!)D!dcWATxi*8Y^>XYP?${7Mj@{Xd48Nguy(&#5DghPHhTSHb7GI$*@eUKDu+mtr)sw+?BXiGv~X zK8MIB6G;bG3Dlj)TJ?gz`9pd6&Y@UoTDG==S_P(LhdIZy83_z}u0YE%szNY8|7ubZ zh>DB$4$LJ@O7U-(C;xegc?d*+R;j_%UxvrdV88t3F&}k6Es%)!DN+%7KhjnYj&~0a zE)gPVk5|wNwh^3Upax{16>s|ERKm_^tt8*N&ioqD^zZxhs}SJ>+G#2^yyIb&P*9R_ z!D|BaWmx!Gb-39SiZ5{rVrp^PGwqk?i^6}6_Z^$et<}E^UceGPdIT_lLh8az~_cawkXN+`W@l`VHO58lG`|Gv2Sy zR1`Wq(=b+lh1cS~$Ccb~y4seUPb5z$ZCKa<+4yOz_tf)_1l|H2S)Nc;7>Zcrl zf~gH5HV-y7FPHwUglxhlh1EiTg23qsFzMCue4-X_FLohJU4M{Ua zD!DurtC7erzJ&Vkk-4_*&%)y|uNXr3ky9!&2-#CifH*vg9=Hq@a$wLPBSff*pkacx z0v)JmfEkhxC41WB|F^0c8o#VokQU&p4=cFgq;EChr1KPfw@GFT>IoP?jxpkC9ekml z{H)IJhCkY~2IUm|jXlBIkel4oz8|=I&HICRGe+`JZ<50BH*Y@P(5eddpgj+4xnf~r z&oElp+c75MbFr4th}zD7*o}wAbLeMrsIbkGcuPbVepag0(IKhK3?k*ZfMF6Cf((~o zQb03wPj4e?DSb4n`$W3zuJFO=?UioXn-`}GuaEGV zsgPqa>2KxVc1G9;L3)L-Na-K`HR!;K$2TMc4Bj+6vDPREhZHk+zUXrewyfd)B))i6 z6hOhUul<<#Gwo>Wnfx6F{eNO2lFOM%gav0@<|LoV9|8#y5j@pTWl2-Tk+<>Mq&Y+> z<))}?w5>kuGu2rvo*T!mTi4S0?yFGwp9X<$v47wEG|`OeA{}R}2Pm~y;Y|vnzLj)a zo!eRD+s3gj*)I0fBS#H$FWmKl89WWZu2hU;0Ji;z)+C^p4_A4*!kMSX!Hp8)|{8~A>e1$|xSWj20QW+PmYwVbODNq@e zv&=)?3<7vg8(SZYLdkn=^DD9f5~8VQ5cJ=y05%^9i!fmX)g5nBUo z1WoM95f*y;5HB1wFAPpcm`kdA{_}Da@A|eOCd<~QaoPJcV(zNKh+Oh-0?qdwO>1w6 zE+r_OMOjf&{Ncqy>m8Cdn|AbtHM%2x^3av2|K-|&!(mpkIsDinyqc9*9J!JacdQ^v zQleg!+0Jp-*i@`zj&W>vMEvp7#4qWLHQ>YMi$v9#rKQu}{W!rO5kl4H4?lPwD$-;u z+1eUW)%L)0RWC*b>WdDL(kfL*4m9ir(v}bFzMjuaueVYdkytVWU zd7LhJ=?Y^gPJ+x6g`FQWKjAvcm`S3Wj`7*CTH)9NCQ!?Gc56#QX>ZKt4~Cg;OKCFs zz-?l@V{Jz1JK5UwW4nJPn=xWr@3wJ-EB1OSi#Qn-972H4!9G^}5*}1Y=>NQw7UMMR z!&s|T5ZR?t!377~^QaPX#k!FJgPga-bM?CsLwHy?q_E24Xzlf*TcalQYG0X8!+kLu z&}$c~^*YY!c-e(}POoDKZ8od{LpTz~f4%`d3|6De(|pY92;o$x<}Do_PT)l^G8?rCuDTzX+{zbK;HqLq)Sv zQqd+0n+3Ics&KHZzZE3FVxK0;HDdJv07f=t%CfyO(ItpHKi;+{%t9+Yn$C!P-xiDM z7F~m%liBi>o#>xzLGCVx$zF@UM0?4pEUL(QkrgmnApwcDvoE|yqDx3+ywW8)My;c7 zy(ZSlXT(AKP_;*^G@%*xO4Xy2o?Gr!G!PoZg0xU>lRV*}I!*%(y+8xb9~@mm*sqF9n8@6D(xjRhY?zX#X>rPh99 z5T4XUC>NA7vw1R^o0t&*p2WOAq>6;CMZtgsUA{O%{FsNBoan-1U1>cC4xp9ZGb=B2 zP8H17=+t7{jdUYtMeamhD=UbMs0MGqKy1(Q>ptQY?o|07xZNTi-6w80Akth?c( z4*$o?!Nv02!oospejXZ82VaJtiwb$~{_$?7|3ho8P-qF{S|Hr=@!|PM{~a;jyskyM zHm9IX8<*+mMJH69PN-z+g1!2?@HUM0S3xtQJD=W?=;hHE1UuRs2?WyR+125p^KT~e!M(?3cpp1RW{ofbC_k%mLs!3eo437B%u5p_jt}~@%#~uA6;d;MmN{%47 zS{0iJwR)4Z8lY`|0;HoL-stc1m5?sB)`2}!R&#plcRI*EzRRD%4`*Kt6#vyosq6=R zdV3R}UHUu$YOgj27Xzmd>7#D@YqMzm&5@P8a`4TR+8eg@TcK)&PuGC1zBfUu3Ax6; z27pBWPqO2mzSz0Oz?);u<`P_tn0_gkbENkdiwN9}F zCn+X7ShiIb*(4OSm&>GO%^s?mQUEQ=%@?6~5)KJ@GG+xJn@Por$yj{1;Y_e>CU5;@IuX1NHUQub<4Fkjd=I$@c3;FL zhu@_!siq^&O1%$f&iPPMtuiBHu1rc$8SVaW3=-NcugNNHp4XV-n^(XvmF=xS2*rtH zGqDvK4hC15M8M=QX}usTOi8p=u_1#MhdS5`UMl{CK&8H$92_!qdVv#cz&ILlrfWJ2)APh@Pe(m^H=hidIHjpJ{a^AsHg+rD;BHxgC zJw+($F_R6Zv^7;LO4-GN&J_rUWsO(5{+s&QQrod`;RV>TJ!L03Jr(jagE^Nvl% zKSsm002U5dt)7j)so3~GOhF52Zf`Dk=id>uc1aSX6)d$?qXhwHg0Bf5lCeg68!en( zd3!-@?Z(Q|MA`yejfB*+g+@pH@I%sUP*uFoQ9)bko4ypJMB~mtN9-`GmypjQwwQ1t z5a(79fuc>|07sDf-zpWWnl_H(Y;m2gYu#Ma3U85Y;{wJ^B`zYkwwqPWz9LNOzN_Xl z=@{>!@TJDr(eU&jHAPb|Lx1FV@?WLd+n zpm;5Krn^i0&`k~d?QEl4r>^j^d0K|NR-?T`#ROX77XZxz4Oz`!szVa zG5K|mmBYH!Zf`6uFD=d!hL8SwufJiPFgLst=8tc;3sZjde2rx&)-}{_@UnGJnf9D% z3ZPq{uz5Kpd}ipKeb`%GY3t_g{?1nG?N+a)@8$xe#KNiU;FfS#Kk{TnzCt~MOGeLd z!Ja`SHzywP9Weml2D5%*vwq^QKOPP&Z_%7mN>>eW$Cg7o+Ev&VYf)N_qM?w28mWzV zv;5eRAn!s_+W10Bwd7FZ*29JqzHxTOa28-i63^g3qHIpHJflEp>>v?>%n(D3iQNm- zFpK==e!!y!aR|Zkj&8+p92tYx-8(;?$d^%&7F3E~K}&;v)!Om$Lg~A<;`2RR?=A#_ z<1wcxXsr*13kM5G$#xm#LYbeby|FxeQOB~1=bGf<-X+mF9!v)i2o3W%5?5ObDSq0M zMJ{MP8%p3KvNADzG2lL;nKa*Y$vnaoWgWTv8pK+D_pD%ZE3@66YObyJtqh8MSg%F9 zku%rP8>qppBH0%h*xh*=ULM{Ab_Ufe`N;ZXYydwH@5!0|B@@6_;K?XqCQ>57NGX7t z--PkUd~l_H$E(~4mt@~L6;W)A3Q#$)#?uF zh#6q?iDQv}l3mIQzgkrX1z2_7^|rT*J1BN_1R@yb#2T^rdFvg;h+x<|6l4Ml@7`Ia z%r44RqGI>1rj(6J67-xC;+kGFkCPWqY1^bBdUC}`-M)3CV9lZE}{LB zqA*E2wOTnCKF&{7-$}<)4M--=OC%{STS2oPlOtl1Q1f^Qs7?;gk6RBcgyoRnxoR!G z*D-_VFTW1XUVQ)kUmoi(@~Af!`Tl!>K*(4osYRJ85tve4VEKi4l=2v8-LD|#x zY23ivjBn^2EM0a*R-SK*9Sfqw78>HPeQ|%QiVx^mN2JPTA*qnkPNj`akkT;ov=Cq1 z252xJV}52ZO@t|rZq**rk(QXoOPB!XJjsbBrv0q45_LfXVqMf_Ds1%GkiMn(=>6nJ zxlgnjqUfbTl!PTP`74Sw8i7Bf&(|mfDzP2d9R^s%u&Ej+n<^dNf?hVo!%Q!Oh(PQ@ zP=o?kR*(uwa3@ynjaky9>U?At5-S=N7xsrv60@KuRy#QrV8gZtv!%*YAY92H?g|J? z2z#bpF)GW0v5@V9OF(~%x`HCllQ$~X$ReP~d*vBbQSm!$>4;YkJam+C`|qS? zOVvx`*k{Az(XUp3&g-G{KNG=?#AdThI2?q5!esb|gv6B583=?Y^F-I}LB6c3qW+l> zo*~<=Dm_rKK~WmJqr;=&8OGa-;p5i~@x4Y^K#^%SXPG#@TW?x(^N%0P66@pgx&I$K zl%j%#(ccO$Nt_umEctDZ`>#k2NK-9CNNzhvTmJm1C`KALf?#3cG39QEL@SngaQLB| zf|*0G{(ope4ref<&K7zi!1P6e@WZ$rXmw$Ql3hgvqI+mm4xtjoCVPD}o(w5zEO~r( zIB@!*Vb^rSY(LsQkpUI;XxJ(*)VCxNON&on+c4Nz1YBex^aiP*JG6wz(AL zjkc$tmS`eL%JD#?K|U)C9WH#8 z==Ws-v*3hLiFplCjW}7AOG568$r2D;xPTOXn6dal(#It@aSdd0r)T3e<47K?E4N_Q z?N#A;+mo>-fhD@iCr?IvTd+Y8iIupRtl0>@S!J+jS@3f-kwj&OPYj**gx{J(EiAXJ z9L1LPL#ZOBOOLXkfEF7Xc?3qtI*VKp<%M7{HNFqk(dL0QvEBq#rLay=&@UQ}6fqM= z%u2Y>8BqwqFTpn2cB>5+jAwnfYQRo-IWKr)loGOS8g%+*llaDD(-N+h7G2BfAP2?u zjislFWJ^~mmxm3b)nhu-L%DbLYzmR)cTo{i@+6SiQ#%Fisp}kQojQSDVTQ zza4e05GoSyxfk)1=GP#mK|z?yl_RJ5z9N;J!6Q_u%dzyc4DQe8Ky)pgf>;i$tFx>p z3}gmK8)6J><9Cm5#5Ey{<3sp%-5jt?@yZS;GtB<>T`f3UE}8fdyL;!)kN@)c5Rij^ zMAJ?WmV(wq2Xx9=xt`XTnJYy@IE%Xp>6?UvB79JWG#h478uMl$46eR(jQ3C<5DOgI zkuub35Qq zF*D^`PY>r_07)w|X0c@Xw&q~HXwz%Gjjoe&Bq_uepO-Y_LQeCsFfimDf(N22P96(H zv7roMinL!1gX#@BxX2G`6lG}5=F1iu!kADO%~5DSSLi@dSkc6z0XZu{E4gA9oGqA_ zd4*X&3^JQy{P?sZuqgq*~vUN4uY%F8f zz^1dyG)+ncW?o}kZwhAkC>7nn1ixnPv^969sZxC~XqvfI(3jzLX+2rtWkxk`)C9@q z)}F{GrVgT62^kO)nFHm!2xu~^@0`&Q`@FP1Cjg9BvXo&<(z2XOW{yiRmP5vOIMfug z)O)DZ8U6hb=rGQDvx69$7;gr%{RSgVSfWu<&OJ9>DvBGVxyb<(!Q!Gc@nS)Xcfh%& zD}YkwB2|OMF6#=eNM=`-+}CILFk3$i&VE5qLtj1Iu6))Ef)awrF4jl+S z-@UU{dojqRvATw`t(SEUZFe4FPUWH2_miKQUtU~P!j-8(jtT@8W0AEm_{eq=Ro|g5$||^DHyTgUN;)Ny{OIIH=~uryDF{ut^54 zlShb}dO{Um8JPxWYP?_o`ZPK_l8sk1UuvMGwQ?9p5Z*VBdxKa4kBUBOD?H>{!yPY7 zI6Jo{Q>%7%R{iX3g*?xlttFUE0#-R6id$KsZX=EQ}q#D~T$lE-i`HAWbpN(vmg}LJ#WTU5?;< zREOZC%79;)ZcW@ZEk*6Dj9Exd4W93E;)M-7gw0xln^WKzwPY~+hMH%;vVw`@yx*8H z$IT}u^VygZ^5VuCQ_yEC1u2KyMZ{@X`}+S$I>&!%(ya&g=k#N=Ovw_$(maYQe6ag; z+kds@?$1e3E92|O@1@NOw_VqSYz93x=N^=^a#+ulN|zQOmXOr1uZU`V-L$*>h!l!#Oo(M3Lmz)F087FDVT#$}D+T@{Ma zv~M0-F#`tiX1y^+3F>p2p@_Z9?<$QB7^V;dOR~9p$1-|SZyzIV;oMO-D->4M)ur{Z z%g7(|E_(dfdpEzFepltC{rGHs(`vjaRwE(6!MGEI+m?iX{%}NuNL&tjtOldY-pP3) z?pDS-2qmq@b&*)l9jaHQIc#jKwda@O>j?n}YfHc7`CAfn+Jzg)`2YhE373mY`qNa1 z=9|N$EJoqPLlov(4?IMnyogv3jI|PsJhUao`AvhT5Y)iMI$MH)vwbk8x%C3X^5XLH z{J*_PP>P%Q?&INyi~sd++wbx(#TNH}>=h3oz$5n@AyZ$qwJvCQL3YM%{m1;`^5TYq zHJ?lpCvM-eX3gY@wIEMeG^u1Jsr|(C1xY5+myCBvJO>nDC)|;OwulibzpvY@`8A)< zA7;J66KRGh9385TA5g-k^~wT|&Rd;Jq(GG@m>>b%TEAGckghck=_cIQ<`dI5jt zd*657zU%M2GXXzH2n4UO@yTT~LW#B_%12r#0^u%5Vd0%xG(w-(tL71ER~6t8*`daL zQ`8LGE_vGS01;#{QUr)qX@(BXeV(6FR5)JWF~K}M?!gG-8{)JR}5}xtg@h7 z40XD0f{*BSQKrm;g|y|G!X%~%T0s~>W`LuaHlWftqU|ybP@yfb*G`y(u15cN;DgLs<~-ea@Vku3M4n5Qv(xqi`d4atetLDBNK%|mhD5d129wwo0JOZ?t| zh%CP=YOT>cq_83D!eA8{i}zOZauKDDuSnsG3U$4CV1Kb++t6Z=c^n&VO>i2Dcm>xb26wII~-tSj$gt(p$(} zwF%F?usxjzih$EDYVfMwrHjHt)k;C5tA)LgkCteex|OO!0kq=5uZO7;CC11JlnIfLWAu?|_2HS<6OvQh2xiP5 zkV$}+iXIW2#@HqEAuBD4bj6hB4js;qfk~Ww{)kDL&eFGW+I$kBI@uLgB{0f`CqV`mX`IH|8o{;kZ=;7P*G$@h4SBy%4DnQUI=?)|h>FF~kTW!q zpvRHW(wR;czDyWoD^xKx9INt@p&GrwJXwVCm`+UD^ZR94^8}C>L?mU786|OOcSv{0 zLl(ue9N;W_h#;jUH&y7Voe5clg|q@&B@6ONAi0K%-y znra3_9W*eNI>Cyz-#?OTdY_Z-4j~(L?nQ#Ppx5(oW{C5J-Mo+Q?DPdfqUzp-q$$?; z#Wg!?;cIWrYo68n>Pm5LIa`2j7GG?Db#R!{rp+o6SYSTlu&2|Iz2U?Yf>p6JgNv*s z;e#12TtEX(*syRw2l6EIA!Go_ zmd<_o$x=$(kArNrH=K_JM5K}ag2LzQa~dMpO9A4wZw>tfgBUs!<{G8_;$=N2K{Frn zfVO2xcknDli#RD&;%kb#2x;^o1}u~gO~^1~;gLY&((ncP-SCk6QUmrCUuHnSiX>Vo z%YqzM;nc~cemil$EgxvVy=WE=6`3fI)hrHxqaYj!`s_qfj=cwg8OTIj@_r<1@od?G zp=2i}6of%~Ay{H);%KEqK$7%ixOzxUzBeBFx$5}quEN9cW}7N!+uD}K4>9cJMfebJ zNjL?8c~{nzm{5J8F(lbXSJZp(6D%kO#$Zr|kb0MM@&uLlCDsZ@i4N)s{;zZ!C*L}1 z@4i@BSiO6vS3LgC{o4pR!3WmRit1DS80HNlb{-W(A#~W`8I?Telv}LOzg@dBS9eUu zcIYKXFx9Y(=X3HC49}E`@t2D;`**)}Hu@wj@cG;_PmhlLGu^^tJljQa3-3#}tP*ok zjVktIZy`kFVswlX%0qpGf|CZT-`5x{cQlNT=BjMhyjF{wGG~_cCWgAjaj&@74WsP$ z-@NPP43G4q-HP|fZ9V&wv5rK1$+hb}KHT3wg5_Pg-UwDw{5~S|RMZWWl!_H!cm*Kx znMD_)Xi#+8gC$msANvb3V!Vo#J(l9X(byqXk1RkRb=$LXsYvlv)t37aV^0 z5|1A|fN!@valhNzZ_KaClk&S@cEy4#-DGxDxp#Q-`w_q0FQGP=zye^{Fp`P`2Eqr^spPC!Qgz}vVCl8692<^@TPy;{rdJ0{{P&F0S{u%Rw{>F zSqH|Gt;a|aTJP!tt?jJ}EVl0~KZgpKuZt2*poYf=)s;bVY*|Z2qZ4|Y#5zq1!%psLUawvBa+g%bv&4<2(YAnQ1t&~5nooY5X+32BC7{z#6ot^afADWw}rAe=Ild7k2$?5(73YTo#K2rJyYN_0eIPL7tM)D zxt=LGq#zGryOya);v{2IrPAM#pQic%vZ& z``W}ZyMP)~kv2eXv{=Lh6DD2^d1xpUrnnh)(DKW)9WsDqLm=-T9(}-cYR(o52ghCd zN#&6hnH3+1(c{qvoa%4Yqp3+sMj=%{-m$lhCAWPs8w?&b3OrnVEX-QH7)aSt*~T)m zLrxzB(PP(-_u!=Cf~{T)eo)j_Y9Kyt9s*jEF;mGEm#5|Ej_PxCn$$05i&7Du!@Z9w zZ#^$8T6izdsb8Vco^elPOxtv8*$srBbqf+$_oxm`R$)!%rFCF8fXS8`j>(f{k&5_7 zyf#rhzqxNYQ9v3a-S3p<+jp3JXu%DGBS3>KUCux^tNV6YNWwzvfpEi1xWq8N&W?4> zVM5P#V(WE0Ys1f&ulCI$5keEgjrk_Sc!$q%j4@CB-h+oZZy=`FB5r`A6`fxRR0|KO z4wn$EJTPMi*@`u}^G@a7@ins8{3tI^C-K^YNKAsYc*$|IDwv&_K!frRuTD;2hR9#` z2k_X5)n-Mq;(RZFNufqoOede$M011j8FHA2ZusCd5HImq$;MO-DU;%8UQe;6af{>x z@o2#{<=-5iMmCe4O z9zs-LNpf05EBpS$NsN zV`)zgfC2mzage6J;=w)uK!|dS#cX<%TLBs+O)*2oqMM+&xOWdT0m*{_Gj-U3Y%?1J>* zKmtKMNK$Fx0$`LrwEt|pD(AtfsI8|o37*~`9&_8xgtJn!2$XGA-9+6So5ZsnHZpfm zmUx*SgoSEyh^prZVwXe_X0ozXM4}avmc=)PKr|yO7Cje(doC@|!b=AVf!ygV5nS*o zU3?;%rJFw5|AP;;B#==?o?J)TU{)6`obYB!-(;=>{6n!5)#Dd_W+n%Y@wf~AQSSE6 z-#*>!;p^L@P;}Zvj4>%c()Tmi96Pt=b%7`3!*}~dq5cI5_N3gDvp<7k0Xc&hL5eu z%C(kmcpoG07U=8EstqR5>>c3*U%DiI`BOd$$g-mU*ND0p&EtLif zvl4l-U~5<_m5Z4XyJ%pJ2m_;p5xOOs9fxDge)9%N+*>Hc15R41R=_k zoDv8M);ynEqYLH%4($=H^n3002mV+2#&NqxP=yr`o+f`-Yi{`oS&+D11;-GGmI>1H z`-6E?Rl@Dhu3X-%bc*G=xjhi6b%ni~$-NTVb+kOyqY;sKBL)*lh)7hXF?K#Ij7f)I zFER>aoUNo?aJpo78IP5+KxNDw2GS76gs5%d1E_!!P6CnPP+uh~j&R><5}9RtL`c;J zAnODa5S5zYbwr)^a;Wpe#D_Gh)23h(Gz#jv>S-_UYEM6~nC&kQi(0R$aeU@R3r#Fg zYtas9u;|PAXitWz^0jQDNxmh^V5|Mw6@3w)B86N(uKgv$dq9^br8;xLvUxQ20i^3(IV zX#Nf>+(=wogUJLQ*7kleUsal&2 zK3rG3-wy>s7smL7mP-%$WfMfO(oSZu6gHV9R;U!rgx{F44^L244nR$ciRg8RS5&r% zVcLpoEQ{RgdzPs!)DiR1z0b>w53`&pt25e|*@MLj5qVA0KEnoMKQ*;OQIfO?d1u0d zR>>W#(ty-ng%^8ybROa>Vc{r7T&?3Qff{BL8I_9Dk}`E}qXk!XYtIdQPAtFjK-4YDx_5Hd(7gzU=%{jfK^PEU$_%CvkNb zzGGGS&8Ak_j`9gBz%t(F*rN{3x}4jw*>RiRkB!QJu2a0tP25__AL2Z!Caq<4qyFt( zl5GmH7mk<}FyflOl$|ul=wxyREE5C^=7&q|u+&F!ip#_EkFASOBhg*Y#?@#JQB+Wa z=|Zb$6cb?63{(}vd_GslsWuwMSlp(ER>}41p;Bb*xU}l zm;`;1_S9oB&659Fb(+m%uTWthT4@=1u|%%Y+(E|Xblp%mO{1oQ*SkwR1;v_8 zs)3<|!Ij9Zxzfkvi!rHyBa;^ESs{*96@0aYV3t|W%Fyg>XBVY-5je!61QavD z8SGM~+@vq)LjZx0QF-L$uDXMXi!KC$=Hoon7VbIO!QNu)FNaOgbtr|Kbykv!shV7k z+bp#w@yAKfde-_#lMf1WJ&ID!7`M&E)Fg&TNSjwThD$0~$*({)3YnBpJFQSVzApLcocJpt`VhQ=cnzk~(Df<|@f|YB`|tla&174{%RB97 zt)=$rlh$|4>e)ieF8lD;sd?fIpx6}2K!UXp(9d*q329mgS1JHQ)we$c_R}Fi6QwU< z^?#x&AlAh3ji@1VmDVe8h~iBnD-i@GgcB&t#cy@OA{NKm0)kyFsf}57TT%s)BWS+*eI9`5=iTR3&U)p`KsVc>sVzceTfch?&h~@O6n>ZWo zQapaNN=`B#=4{8vLRIw&<{o-kiN5e|labcR0F2lD-3T*D3M$yMdt^(d9IiCG)|D%m zL<}h{;ZAeg`{?im556^<#|rUg${`i=JXFZnFO@h zd59qb3X*$ar}7A?&pyNp2h9ruDzZ23kpvYkxvw}k1938dE^&H!#)BkDQmSM~#{NX! zU??|GUUI1B;^H;&BS_cjWE@gWWk1A=n4=`QIXzS&a_hoXonY@l29M~NV$=zZWcp-+ zciJiC`eJd`z{l#xm5ubHcb5s%m*g9~_YPi}p8}*zlMYwehjiIi#Gb2-5xI57F4OeQ z%$X&uX_ZD3K(t7XH8^g?-OahpB_qua)nxeLFk{9*_eyaUVxxBARl#?#dkDj>e4ydC zxfk3RXhU(Qj?^R9sm&`LNfQ0)ykJjwb7P8+5E*|l9~1!4RN?I*^EB0mISv})1Y7AX z6vPcn?4n+rX1pD2R1sp$V<9TqjI#*3p9*j(JjeDf;9!(xA&-b_V~P@fJ~`@|0-974 zGXE3BOY|QF!|`@JMCI0%CYBa0Tb@4#0Ava}`KZbboFmQPftnP~Bdwlz=Q2_LNPbdH zsNJE|AjkK8wfqp5IvTSk(L1{+NFhFiig@< zQPUvSdUZ?>n{+kxN_$xlFa%1|q!p!i8iNA;gh42gEya#1% zHXAyqZdzX+y8+~DjKWCGQN*bv13LjE zRiC;s7(~sXqGdy4urmshnXmO@97W&6%Tm+kxB{0kzrBfpVrTV`!I6Yz&_WA$hcr=@ zL(=D*MQw8n^C+8CM$U}Fb7d)kH>t=w+~PK+;Ei|e??V>x9ArtM5MCqLs>%WeF|(*K zeIenPb%~x@1Q0OOn~2ZC0Fh6lK)G}IxM-GF=&?2|$=uexy6xF#ZD zQ%JYe&1I1c7yzJRlZ{)NI|Yu-AV6DM4Qe)}NqA1NU)gz;in$*hGC`?SIm~I~U23(y zkHaGbqUvWlm|B5n@GJ~x+)fdatk=-S?GOIei66j(?`g|_$Rtufc^={9<|H@W z__!RW#_u;)X4&-k&^O$0?-MxFPV@FFvi|YCn|2zpaHiCbD zFbhyCk}GVysMzzlNn0o6fD~fKKg4in!S7)#s`uPN0a;d4_5QMVQuh~HiivT)mT&Q;giT4!z7J*E7*`YEH@&HTtb*L$v2-6ls-j|<$hPjR|6~Y0S#nW zK6a!(8>J7eXrcI$bo7PUcZkno71(Ks0ohVEGTWP3zz{!O{XAA7#tB zhA4xcO~*=D1538;kn|V_$BL+jT+E3V=Yy4JubeZAD@usH_*cw=&J>*idLRdbpr*K| ztim(1PZk_knj)GEd~^W~Qy!%(gU0j9fu3yN!`KiU2T>5+Xl>RNxhL868n=`NK1MXj zuqlUvX_AZh*;?QQd$$?x%s3A~O#MSs9kCsp1MPA^S}at4YKV&q#cStJN1&nGR?Bcx z!Rk#c$;QNToD9|Drqw6Isc2Ewai^T3B(Jw$MqfIPEiK1Qz zL`TZ!tkkAp$CzeBDGB?nzkFOSsJu+Va0F8p2>o@OF^_q?1hy>mRKYEG|#IzAF_WxOU>{StL|F3E__5>pXKx8$5V*$;fwk;;gD}x$ zD`c_kg)HRwy-)@k2a>kxqm$Umgk?_LUUQD8-NRPz8fb`4igwU|5J4rHu$g1^uZl>1 zwo_*dFzsv#_`x~-0Y0M^bYfUUo4|F6IDy6*U~BoTh&!;L)CwUj!4X!xXYMWwR3Ta@ zn6Uj2R17N!M%s2e*kN}PeHWLDFz3a(IT-BiXxk8Rer9 zl~E2rS2P86u>8L~Rvd@$PxfG5AVV|ROalsi+mxLi%jV*fHfCl?RkjYf9tyG99tK^4 zWYPKiE7{JDDBTJ_v}j2WwdVp_SgaT&Izi+VcIFUW%!$c^#Ms9;Lik=hZ&NFbkZg(w zF>+D@GQ`)sS7z?9WG!U*{0=d1&0!wJV1O?`yw|1|zK_qacs&x*Hzc7eE7Igv+>_F` z#D|bi>IPecyp#RG5y6ha+s(=cWfxU|SwgCN__kt&iF7^O4_^-Bq1(z~LEPFg#tSn{4M_1T1C(yNgn zldB+IIlM5<$+cgoloQZ&-FHesa$2-hCB)PreKz+<{uNP6{&XT!XksxyH-g4VFfqBZ z0$hwIDwFzpp?-Q33Z4ZB(%|j1sv9y8&^*P6CA2iaz}5w}0Pkopuqi9b)eHL?G?yd-{juFN~?lDqQHqd zh@3KJeCmzoj{3s|A#~!JsVcO7sM5ZOe(}*)1h~G97wbn|mY1q~KEe|b?B1Drde&Yh z1z#4J6P<^W(xy?`6f)zy4gktXf({$`3FO3o3Xv6@KkG-CFiu^Gvu!QDGt#&ZgFOZJ zuylKj`HA*gKP?^QAV*~rEC6nnm)5gx9xTHr0L=u#83TZ{{Hx<2PG7vUsk*iI6BQGHgr{r4;15s0 zCDK7~iXN&t%+ZjaK^U{(xbik3!ROWiOsnQTS#|dbXd4=V1!%f#M4v5HcA)z+IVgjJ zVX|hJT=MA(TENgT~I2#1w$Hr1u+q?j5L8_mxKsg zeQgTBqK3WLewQFWbUP~E@VIMWM_>@i98BXn1G0TF$FVhK(7MQP3tD$yZoLO zsWeK(t!$pt7}xurb7ep$m5I6FlCryBG&QSX2LH^e>J+(M@Xx4Q|r z2DFTZ*P&hn$pkDBNGdb|Nh+FL;ei4-vkx?tq=@Ko1Ec@f9Z1ooiKu2$REW|pB({jy zNl&@KOpDW2A}=@(jSSceJA`JtTInaD#%NKH$p~G9!d{KhbIyCC>2BICrNl?Sy)d5H zMwZ4mzu*TQ+Ar!SGdDJfD2tk?AbcxJrsQLI)d9Tk@4r8p(I=$(eWn{Co=rk(OgA*i zP|7N7&P7nqID-1%fPt@<0*^~Ko`!_RCV5hHOZ6KhF-fN?g_cfOu2D8;gWY%u%NWig zED2GJFGQ`YrKSfyQocV7Nq9naRHxh~MqZ?(3=m?-k{0&Lcu+1|U(B;Z7agK0gs4f^_XGnTnzG`;`5WzI|7%E zX000#j%Ca5SoZu5vopG`u4qrmi{67#rdB*~a>KzlA+H0HtZ{J3T_TZ>CgEZJtu7tr zdo5g$M$b(^l!I78thya4Q&M*MQ^9gr>u@5g@Yxl#Q(6Xwm0pp0vs4$40} zDF1vVC?k)04eK=;X~ZDZuLt#DEau$t%Ze!)}J3w-VvCek~C&I z9`9S9KI}cU;i8b;AQerwNA@&`s7bdq()B_1yO=jY7E+3imE}TuT1e6Oi9oG+S-6yR zo|7AebeP84r6&tY`+h+W5WN^on;Bp&A#z&GQ1f4zFS$*+}HaFcbm!ZQD^P$*?P zMvT6PKhwqzWpAghEGn!SFY1L=&B20I)Bcni;4bz=V$@pAc(aN2&fP=GV2vkqaqcx8xlu|rK8GbFvT z+C;Rm^s28H)Cuz_g4Z9Y$uD!JVy3-dm`oL9!9x_@4KIMOMH?W3O0oX;;nBXT=W~KE ziJ~|Uj;JR>m}k;?K3tqWef*ez7Y@%47l!+nj}K2|wH}^7J{+739&=2Xl4!vBkjHVFP4i_i(Y1WM~4ct=!BlNIz zuCKJ`M@RD33V`okx419^hR1Q7xE3@0v|@|F?Dtp^zpQ>^ad~NR9!EA=M)&$FhT$uY z;R*rw04=d5koWc3!QkZZH{)PtKBQ+Yp(tnj;ab|HFw>_??TroG7!UD6_GFNSyqi5K zsZTx>8n@20NnFvmE>+KtMC<+EXvFvH;OD1QdP~x@q~z-9G1%72r6u*om6740X*R^*d&#a>wD=reJ3DVQ9yLpfh}0VhNU+I08M{Cta8 zfQ#1BQ#wggW}09U=_v8YP20O`F$p@T317Q+#ZfVTKsqSdMNm>24$xPibRJU&XNBVd zW`mRxS{#uH-}rb!Y0-l()JRog7B>2G4$comlpRw2Wd1b-pYl0?s4PYT17I*b7Z3GV z1Zd}M?<1}vN{Qi=(FSw}=ajs#u}IRx1$ESV!*LmopA_obmq+D#aElJEv>u#N5gl6` z4oTx2t>@w;)TOn&un4j#I`u8_7a%-D;RocJ z1U_d%I|e_wdidP~fXMCy-+9*y>I1OlE3NfuiE zY?LH>Sb~WLKf}%a25ECo1X?!W9T9Qm+JZ+SyE`KqBnM$+A>v{a2Tj&kf7CiVrHqLD zwh}E}_?RMg#MO1>#yvZqe>n_isJIYB|6;YR;@3y59j%vw9Ea!&Iv_URmkTY9$)OlP zTq-CmCdK5>N0&&`5UQxLb7mg=_;&l@Q}0>~gL(IbIM>fM)I0TOz$TfOZ(OHfQKd-v>5mzT#!&khbWos;>0 zKcYs=45u*;cVj+bu^=7_GV$FYNr&<%RtZHx3_n2Jjw}LTGv*V$@hlrQ1(n0Yyi7zm zKi~U!cK87#*`iq7(FJ*PNWVV}5UwOc?7@D$)COB7;X$b<+^}^vBs%_#9TuaeQBtL^ z6bfAwMt)hiJUN^P@$IpN{o&(eLV0{T5UG5;dv-Y#NFRUH{1#4Y&G2gm;=D^ul;%^= zjs1)0Z6u>}D6%jndB+`dfdK^A{OJl5aIf7@qY=fegGKyzQ2J*?20we7Dg^dROLK~6 zZ&SgU#{>jXyLOKg=rKf1gXgi`9C%+5{68mAH^>3W;--aiTTHl|YA3#su z?R0wY^!zfPp<+)~=9e~BF+b4uhLDM~dHD0uK+Kr;B8S?oL^TCL^_6T8A`K$1R9VD%)GEXiP9_BChrQl_v^*@FoK6VMjjVA<7}s0l{3e~_TkxcD zm~Sj%CDUlpqPfJC-xKBx{}j%lOlS<~O<^tpzXE$978+n><;2KP$qgmmH|?Cf^LogJ zn`VP-B&(q9^3p19h!TXlQqb3#F%$Jr7GXUOTE2Z1q*v1$K`$X7koaLVxFC8R{DKAD z0KYWswNQvRB`DmE4r5$iUVhT9!2%C^GK<0Y*7Wy@8GN6(!Bc)T%|IODP&ekFtG0uyx8FhL_tf$xeo#T&mXha_ zzUL>Xul^-a^pDtK9!X~W;Zp7bJ5U=p@tz14*Y8QV>D%o|dwfsYzw4gJ<%OYf-3*Pl zIaMod_GFCE7fQSA#S^slaJa^~Zov*J7995QMhA<@&9I4n(>L41Co3po8Z9#2)-soV z&x7^Vo5=j0%6o}oZ=SE1__6)w*L5rip}wLiOFzDSc|RRGajfqNOP5mCMZFlDkqd|> zHNJ7}eE4FJ!4*7R0K0zU!cE_9<2D+so{igp*I%o#-)*{61}Os<(Kx#fTRnLMM1euh^Swtc9P3yRudKj{Vf{ z3ZZ^aUcQPGmcFx|>j%Ch_0(*=H;qZs!L^gt+UYG{>8?+XoK4yeoE=(5gSo`AV<#Oe zEDK@#DN{8$SgG2XNug7=F<0tR0=s+f7KrYBtfia{e|>zixVYS2_;_(#StGBz8R6k} z9-7E>oOQ`!txvd)B#tQi{guA{o*2g!ZK}!8_hj-wASEk{PIgYH zVg8AY`eL$~C>=&hw#(?BP4$!(hbDqE`KUd~2mDzb{P4cv7a4w1^7;H50&0nM^O1iz z+#~6CBu5^;P@*+|A_ltk({Q&4a|!^!Ng`~|e@S%@wo-CA9?P5;YR)ZHX;B@5RDli!*5W?S zLzo~2xfF|vqf@!&HarEJ2posb<^l0adrlxpCparDKU}}(u5|dcj!to14&_!WhDAo# z)Lud$lz-#5jN6)`S@*{FmJQg|`~HTv!e&JZLRU3W7zTvGo#m$6JfsH0{?LP}t0_7L zafr!J6KBtphq^UaO?15hDK4G-wvj5_r?)+vy0!@_;kEIfmgcnLX-<};E}%OOuz8+R zC-=UY{|yE7qYQjrph01qw))oQ;emS4>lkvFRO15ofc9LinMh6ZIC1sz#Ny}a)>Nw( zHd<6PG-O)w`Wu(7e`&QNoJPH8=l*oEdw%+?{5O^Yzu2cP65jk5{s;s?Uy5JLhx{`o zFUBtxPxyE#a9drJ3FI4Up>r?dCmm&yPO7`G_C(~Fdr^GRy^V>53g+}ESAM~DmHgvN zBS6+>0Z4qP@n!%vzWUifLh*pb#nmP12P`hGb$)LBmk(r@*ZuO7#^v6k-MzV9-@UnE zm$%k`zI&$}Wh-3Ys%~Fev{9B;mg>tZs~Tl#rC(q687^U9tp;CO?bz*48nDtu+SUvcZ>^o>aFlui4aeXsyFEd=&d!ppqqAJ&aA(=p(SglZGw-a}>FZE> zw1jDA#emmYsqw9|VxaC6`(f`~YipgI^7PsFbnfl0C;qA@^`YzdK6N(oDblkWoAo?> z*E;PQW*y(PPGi?PzH6Og*K7t|pFww}0fervw%9&Rp_|Z5-};LsfV#dZUC2{4&#v!E zx3Mc--<57-SGv9{U3R4!+&87mrqqM47@Bnp@H4Y+eH^-LhIHMv8tJ-ghAv%BNj;{6 zzgw^ffvW3Fq1!km-L^rn+pcL_w`~Z}ZFlnE8l;`rfQ{KU!q{!sw7ct!rrW9UqT4ZC z>voDUZIq6W(rrwkYqYyt?4j=N`9Ad;Y^3LGq{lW@(C97M`g)Ch>iItP8mFY^r=(Zx zhrM&%$ZxL+>YjtT2SqA(yXT!yGr(JmSFlD>G0b#Z4=u1?V1<% z9q;=MjP7@Q8lB?&>J^>D^z7S{YF56l{Z1x>_Tr8Sk^N4=ZS`=s29p!EK1dPAw43TR zID_sZ^rV&Q6t<`#MTdQ`ax7I73wzSw0=$96I)$v8GLem1xnl)qJaN zT}>xOcdn&7ZK7*J1FjZ$XWvULD!P`dJ^xHRUFmD-WSla668Q9%h{?4uD zgtvT@t$h2v&&x~8>jiZADw&f%`BnBcpG97=OdzG*RnTlMueg<9WwBjJ1uMw6+`Y26 zRSCl@OUA)gmX>RAtkW|z*?a;emc70-?A*+LQrKdm31xj26H4)1;C)x+`!%&h?Y8;13pUME+L@*5;r5Ej;qBFu1aMqyy{+wrnf6*eq4v7T+3oer zyKGFC-`ez>jPTT(x?~F{e$CfJ7VXCh^lcY%+BK0}Y`bNrjb$edWiNEoPP2>P__HjtU8O?2EaTz89Mf@oMUIDnq$WWt`3!l<0>_m&Z=!vr)hQUIG%KBCbigk zVq4vL($I(%jwW5uBmq^IGbHRT z$CEDp+;X*DUu~Dw)|c&7T^5_hv{BZ?a2C4-OSa1!hDY6EwRYK;*Da(GyKJm-vD?Ig zZqLp^w^*OvRbxTVSKBKNg~sgp@_MT^hg|HD+peD*>+87*x!2gt-kMn;dRPHVX!Lw} zJ(gENqvz}EHL$$r>+3b_mpxxzuUMYVeAC~FA-5WQ(}Z*c`g+4QeW9C;ecbeQZZ_7r z>FeAq_YrVxuBDU3xBkjfx!S&4W90op*84r-|2O~kgT-!e$R{x zeLzyJ-Q~x{t)+YbxO;1*5Z3utu3>8>$?A4z7H;fob?~>`qPm4Us$BOL0XVw3uz+cf zTW+M@!q$@JXqVIL{VrG2I+=pm<*t$Nt-^S#XS-R1<*sGd+W1q_@a4sR30Z^)oeutF zR->=&Jd}Oye4T`0uEQu=ZOkHB*c0DYqClk3PAoP|80#)ktk9lyQ9?Ilul{O(TU?yle6Z4Tab4&QCu-t*ggjlp|9c&~By zrr*8U9K7#7sQy|$6`Fb9P5b@L`m*a9T&6B>wr9bL_|UgQ3|6|>O=pg8 z%bQ6=vv29N@NH$IymP%YG%WU4vv-bL`@N(s=>RPz8Ut)s7R@chpDfvZk?EK(do^!TTE4t3j&RlV`!lZCc#Ep1dACYSdZ4@8L3LN^Yw zs4BHk2qXj*6!9b0zj#+MBe{U@Qx793J2xrDhf1TWQIwV!dM?u3g{B8;eBcoyD!_R# zvT{c+NsWFDG%t^mP%^ZIVUTT2#0!c1su)0+#ej153zR6Z8i{F= zR-LIu$p+p@%E&HaE7;2}|1n({pQ$1W}J^ zDM~U6G~-)Oz7&TjKwP&J==g}b&3glNYS2y+@=E|Jn`oWCE%t`iVk)XZi3lNkCxtyd zCEid}$K@LN{~xh&@)z6R_5ND;t6DhCPE`dC8TS`4f!%`$ob};nb?aom>4{*?OwaZQ z%HcO`ieX)L-oal*06m|x(u&{uz;pDdDrx9J8{}2*%PHv z#D7_J183YlOElyfIr&5mw;sHriqk8vpGMn_6lOTiN@8h{8?%xsf*9IGLkepr=M_5% zWus_1-Iy+SDUK;BqIrexN#t#LgG7QHV{SA=g?q>H!q zy@j9x{Su0m#bc*#vXxO+!*)9Ce9rNW*8DFttm7CSQIn3eViA28H_kaUR#1d@b8z5y z^L928NuJM*a4E+=I#4Gm;HC-MGBj}r3uhBOWcIFO&UW=wIZMAEX zY5?}>i)|M&)aW<_u^L4ODWIQPp0&$jL##_z+L9q$?$hD^#mDD!_|LwZ`^q+5;&%Vi zf3>r{^ABJ#unuI#0#9FbfY7lD0|`zK{>0~J_f|ISzlX880I~4Y4&v}y;}_Mcr;S{F zubz1*Rv(HpA5gsWj5vMYDFR$=WR0%>9xiA`@TR#I;G%jy|6&Qgdxt+~{zq*~0n6zN z&WX!9b{yCrgcW#>Bf^>(1G*oAuasiUNholmOV1CPz+2#zQc z>nwM7$l;a{iEPLT=Q*WT#2#BZt6T+{G$6G6G6|Gn^f`|J4d4zE`CO1yfA`&~H5X6{jrq-L3eY^<01TVDlwSd+qiI{x_#{=%RZ7@~&cK{vicx)01=j|+1HySzI!av8 zR5KLzmh1&-7((_S3VHVfng3l+^HNK+@~;S0v4Ga|`%4Rp_y5ZNEZcAYviZ8V^N%#2#7odRg3YSSBI_Pw&c2*Pp&z}>~(pq65E^Fv`hnAU!GsZ1_ zC}$yo{(Z+D#1R=9$$jLb>e&_^cE{uv<{&}bgzgOFIDKC4Al3yyBCWRkdUmnqDd^vHnlvo~n3lIoA>{)U2iiePs}~7WD(lf`^oxENsNk2#mP|@M zU_?citDKb7FOfVg;e+AT_S%Wy#ivo0MR;lyfS;fhTE?Pz91TuYaDn>OBiZGc$C2^k zKt&%hU7K892;PkSxS8zbgFP}S^ET4&W!fI;D1Smn8V=6jd=3UwsT01$H@itUz^y_b zYHW$nYKNnmRc`Ptl5eXWZf-7^IU1gy+`nki9LF|AyVTqw$Vb?jXRRT-i9qVRvlr2l zBjHylnrvv!wkN@EZ*O?2&b}hxIzT_j@BuZT$o0{ervQcYZuCt#)cALBc`*WLnATG7 zN$M1;L4UeW+Py+(!J1KdD$BgsrUF2yvx;eZE{ ztIF{~5~5dgM*KwbHiM{nu~>)h2eIgE11*sZWJ~ta>UfLFUN|;}*$KiR51&c2);!Z2 z)nl@r<4HJ?*45M;dGci@eFyRo9c!FKtGEr?Bq0Y@j`=I_CF3iS|Lm~(RI6PxafCIv z*e^*T>^Nc^4ILO*(+j^R?CGR^g$m2T5`YhkB%iprk zNt$ge$QE%WlIDFW;#eRdj#8BrhAf#XT1gTXq+t9&*R7uOZiCn=e~!ytI3BR(oSZ@I zb-BQwz&9m=#mV#L(%j3-zCgZsgSh5;c0H38N@Z%XP=<{S=98f*Dgco^YG~c=*EoLV zfrs3Us6L`Y;Txrk#ZyngLQLmB&~GD@w#l#3j-m zz8_DBHxU%!`Wz_iZAfnmTmt$|B*5|48BIL35@ifuwC4id^yi6jH4{-N2(wO23Um4y zRnoyA2e1V^_Fi+xEXAYP6hYo3Vc*aA^W>kbNwz6&9LEQq^V&mofcjf=~K%?aOcj?6e*K zl(>Sk%HSQHS9pIvH0b)fWV2Zn6o;&y%5gL*@0TPmh!-jvdJ-S9 z9ZRzwHebF)X#K_riVEXpwVOSJvnwvF5#==ZldUIdNa{y*^*TeR9cBS5oQ!_{^I)>} z-g}>~>R)XH#$^gUXgyctRCfNH?op5U}u%)B;Ih5Za9*zHdvehK0N5Amd{>)bc=3s4DR!9Np ze2f$>_s!D1`wt#EFG0Jn9L&+NHFH$INKc+j)**~~0d~{UTgir5U9z48v|3+(h@4ZHs&ve{*&dP{4 z=ADE+@uidV5jR@`RjFyP0dr)uW-GMdpBapID0XPgIDyo~p#rzF5U`__(UuEl1EnmGy}Hr9DMIK*}mt$J6c7T<8llL&2rA!ZsmnNE4mx9gW@3 zh1B$B0-~XCgH{G!hQ~cqG1FC7q4;(ViLB7@q-vsookqD0v(drm7rxnDd(Ka2ce{$b zHFu93cW#*)?2ZY4^~~g_n_DLBvl;n1NVhTU_3M7RQ)R9JJ^Pfli zBH+ivV|tnzH!!>2!Qgm=(9F5TX%K$RbI78N#`ukw3zCRKF|2?n&dO{geQimMZW%Vl zWmi-9n~M|MzCxY=0VshG7SziO%sxH=IpY)0QIhvG+9CzPVA8gk^>njf?h^g1##Ah7jr6fGu7YMMI6Qo|;XVY`%s{NRW%r9LB_FwN!vtCw7g`@X zM>v)d9MXMe?h`j0a@G8ddrm0&5Y2t#k&K9@r^;g-?csS8TZNHJ8!|&#+G2tgd(}gV z48U21t`aV8kw9s5%em=noR#}g4u}CATg68OGvlbb!1Q}S#WGqi^ks^kh2Y=iqO({2 zlg!1zQDAIxDNs99+vD+afMt?egSZQN--o7&{m~W8F(3!ud&DR`s#JBATY;K^a!vJnc>g(jYzFgNuPZI2w=zI__sp6BKx0BxZ0VrxAgm zlb<4)jD|(wRgYzGBMw{~qWj*bww{lD@kS)!IFqCCK>G6e6v4m%X-K_nQk{(RM6{VG zbb#YU*TdL=OGCzV7MAOjK2S}oH!cq(y_?953B z|CN&C-Wj|Gkp2)f1Cs;#3Xm9`U@6D2h_TjO^RRt|1&D$hrQtP&P=RM@eUbv86|qBd z^r9wzX*wxkri~wn{OQ`$5gsy5Q_&K{_9YZ{bi3qI7_Sl=(c#AE1NAp}11fT`BsCub zdnaAulZ%*}6U;1i&V(8#`cc%fe(-MmpF#3)a7O2&+Q*jHE%T57L5Pm0-ifobECf$u zmaakA+HmF2nsCo}gVvh-vbxmia+J5zXbqTT;PPV#f`Cg!!Lw9IRD8V5-DOcjEpL}# z((!juHt5AQZu!1;6ts+lrJTSdxR~Q^EU?I54ZYF~ zg=0*&z2*N3tXtj~i~W%!8v1NR#HD|-7ztnZ=0k$4eS0#*Fa`2NQifag2TgdaoEIF- z^SM$DPto$AeNan;P_CQcgZq++w0KFd1ft0C=?MaY6ZpBWi=@kQaD-pEz?Gck#~z|8 ztC3!h4iTf4jpSN5os-~*5=yV(A_`i80!JY_2YPM8!cdM!d*@{F-6cyCuEI`jiN?Qi z>+=W6G^so2tK7kmPL~!>B3_`VXh@?+^hH_88)^hEz8jboEWJt+NKoDkEdGlrs|unR zpOg6 zd;gScBOy|;%S7VS^`S+N%i;{=L?&OSd1-kOs;d}-BacNgXVmgH*=Hvt=oLwNTyn-C zUu;Krt?^W;TuWJzxT7s)Q(@N1%J| zYH5MqU>Ld&xulZ_kj;?vF4&6M;2L!em(T-OI#<^75Kj(kj=M3bD(*hW0KK9>Z^FC^ zAG`^%eqRw%gF#y?mrhY;IQj~K=JbNNI>=zuIBoUBYFWGwoJL$vG|(hbE5ifpI6dEm z<@uw?lmh*XDS!BiUUffw6<$}h0o`MX{Pl!M_y=>0t~L74x4_Ls%GO z%Gj7MZgAutC`1p%*%_k}A%hIW_iS<(N3^Jk%uzB$CA#!i2*#J=>ENfk&C7F2#C$$d zhoBKgztP&A2FjiiZ~h8}nKoACz#iKf?!Bh{6xM^MSTX$h;&B>>HQqOD2fXrDU$(;M zM7b!*oi6cArc>J>OvxIDbq4wC5cc(AFg_aHZBj$9Io{i#gErZGpdDt7!{ujt|NER= zq`S>`D{L|53gl^jf(u)CN3i0rVAp9Az|92a9 zPyTNKn3rcLphRJPi**uj!1I^i6hF3mtbP5W_;KrKg#PfKBiU-qkMPyj^YUkgeMYGh zep{aq7$UeAKkpDN{%mlFA5j+4=Nm8Aiy!GXPLDUej@tjW%Z)8U1{6VG`QPX2QU28g z9{+NkV>$ozm_R{p%=bgA1}|M`jaq9)pBZC5`|ev(WtsRV)bnQP$@;VHr+;g{LM%MV z#*`=L{}?s@HJprpsUp*2d(L7Xe;uRSKPPMu80?RChs95vBptL~H{UGkOtF*SZN68D z5_||Hn!hkM`Vt0v%JIYv7vubO{Q9KX84+ZsmUw3au8W3?Ps~ooO78rM*7p|#!LB__ z1n%oh;5HPKdl_|u|G18#TP2%gF9|}qE{LAu@=CI16Sc^x(Y76EUyHi@00o9*>2)gw zz;q?SjI+npD4n%UhPwAAo=t=+bcp*%EvjZ6pNDSN`t+yfuR|^qoy-!IGO{yPi^Aed zcsNqSmqbMITB0Mx1N}Gs<=NoiKosB%(?pjx2@f{LZw8F{?eHAoYYJUZwE)F0rUep*2s<{@(KGLRvgBbxT|fc1}rRqbJ<_({I2bP!MR zEpV+N=M4IMe6=G3-%gjTl#NaObKG{8=V6ZRV2&nk|a6~$lT2{gp z>YpJlCwOR|l9;hKD1Ne9Oef6`R5t=z9mp3?KA8+o{)D=^KRWtsi~#XBSlMtu(Y$qn zJmD*_U&=@v%C2GOwFLRSH!{=Jp`Goq2Xe)^)JNRVrXOKwe5RB`0>XN z?my(0dwh66XFfjPyZ^xlAN@!3!90WC^ce)1!K7*a;~%)u5xZv&ciHbVU;Ov};q$80 z&H|-&7Lrx!@d66gvRuf^_ZC!u=krv1M>n~7@!gkCiI-VeUR{OtuCA)=+kY8>Wte6M z*6Br02S=-`ThAA05xvm0A<`B47mapLuZNGz<}x(78U5P0h9+NLPC6A5%gNq~dQa2v z6Kwh5Zu9fXL%M`7FKeB)JOof`Ju$I;gMO@kF|=WM zDh9}2W)$rCKLzV~7dS6Pm0ucFUIkDhfXq`v?IcW6H%74>9{uXAtsS@VAzDD%%Nzy*i>K;jh6e7+cwJBRY6k2uuw zC%oJ??Em1woZx@O5_WL*sJTBp_OZ@Z%m?LDzx&Sri0)H~guLs?`3YH4_SDg+In9e} z(pYEX(?`t*D-SeS>wh3un2@9pGPcum{}&cwK+ut`D%W2hjDuiWVg1qFp`R2(w040I z!tbNw!)7|!d)$M{+k(<66!q_~i6`Vl21jR)7dSM^PYm(G@CTWx9?182;p6+9kXWIt zL7%_AcPS>l`%XV9@hUrdec!Ai-$s{&%P$`_-|U^CgHIY=601D=`Tmif-w&m!f)f^H zkZ<%o)z6|kr;Csvp$1Bg-1P_K{Uh=Av(rbbt8d=CS;?*3d!KdozjTdRf*4^mMx#ik znt69-Emhpkq|i9RY^iMK5U2YTj6*<*LtJ?gRPX$GG<;KTNEcY&MWIV$yGOz*)lqRp zK?8cmQt(5rE`{xdX`a4^hlmbWi{k01$C@v#(CK(1=IqGz8eEN1GIuDzeO_vZjsOWk&IzL{0Iy^)+8l6ta5>|ZJdmDL#^if#n zDL#MF1fTh5-5~#Tkt(Q#?CuhRC20dzG@ek?`3L;s%@4y}S=GWy7Q{Q4h-dxmaTl%q ziNYVMWyD8na`x@$Xm7fLu}z1y+M4Pa<Hm+>pz!S`ZX-8Sy9V*#*P{ z+HGt<|8^TweA<6MWBF+~8m|oYR{n9yO8%2SbTkA&W7?z?v<`l1MzRSmD;!YxDEfdF zxMK!>7Yz(A{=PYy;z6aZD+^TgmvueRlrz9I$H1)V@5tpCLJ?J`3u}mU{`nfiCTw4h zW|!$QWb;!x*i*qHe)sM>pn#9=c8i&;vICGjK@8E-_@8|6@qNg4{7M;QdlJA;4mC^Z zr7gUhpT8#H%q#dyHTXLumc#|gUkHE*U-!8uF?%0>E@0W?60e6(8rVc9#k+Vy8JZf^hAc_GErRmJ7#_#+X;YNrG*mO9S_mZ#i%M2C98LC{2_Pho=UlS%a z9SY-Ad3SFJSq)_oF{3EVhOt9D0#{zqPUa&Z#4)W$<)Km}mD#|g!>yPzS4O!|LU8J} z>)OIyr~(Q&S9B$xg)nQa!DCWl5D?E;loPg5encvadndpL zG0TUL-D{JWVWeuR-SeakT1(wJJB0_SDu&LiSe9lD1u-lDixDh}N)ZJ70N6&5F^1kV zABIg}sX)dr)1#sA6de{=&!j*ac1XVF{EJe^WgooDQuIz9-RvTd3b`TqtE1W@^Gl$)WC}B-~q83Ol+FivLejd zCUujpE|mFl!u=3{I3_RBwFIX5r?0%C`{-9DY3cRI6_z3P@u!b4KYX;bbq5%-HV*_d zH!QA0p$Ng~b{{zEbO|dKoH2#f$UF};+`2AQm0x~c$+N&t6;sq$!ES*Y%tJLf3RiGK z$XpjN8%Ba`Z@~q>B7|rA>83%i-F0w8T3eo`Z^RKvaLf( zPe>|qXPa5mc|G1zrOFgiVDI`?-?JfRtUml2HnDD{H?-^+jv+ACsLw&An@>@w5IzxC zWaTBGcnb5l9IWVvee2r6uLV$zxH-$vy=eJ^0fTCzaD%-x3N$b4e%gXgf5}Uqm?szG z{C;zfdJ)6j$+=Q�*tLjXEQ^F)Sbnyfbg6XBt|Pp+z)Pb_x=jDVPmw0Ux^d#MjAb zwUm4%b6^unv~y#WsAV6Z4-`VYo=1VtyYCR&V)SiKk>GfhMFJr-A}jcD-CYxwriv<# zP8P*ed)5Wqt1kU}R*m9$`NFFQ^RXfbyhu^7Rym5H6x(h5wLo$0C|%5;j0qrk6yYF$ zahc7WMfF#j>NDPK87BmYX*95`Bj&>Ir>ie4G|B+&<1VbpQLEoHj>U~72n8yO;FD4r zWWhszpDTT?Ul&}|85|JND$Nd*67R&H3-S>+oeFw)A`)V4AFZ0#R+_IBSL+okLo~xo zorPd+E^;7(;E)0Yn5`s)0p3A{jIgf$ERM7j0GHrH#5CYBDKSVUnh&0bY_elg8TS?W z)xOe*ZfHyJwD(ndFP`Rm5Ce>zXOl6yhyDq!S$h|KtyfkJO5n-v^fV}e{>?oKO0|Y2 zmRz=1iH|Jt8~e+^^_9N4<}CNyvmmWdO+o%|G`XGp$A#l-l%N9`6cXiKSlbM8U0h&- ze=W1mf`f_>*JHmS!f4r|@v!Xvpg*1{G!NNM#I-%$)u4WVrQC^P7f^Kp_6DDXiIZ#Z z;z4T{=nEL+NvJwubo;CjFT;2A0MHJ71<@hSchzk?td7;ht6~=(`k|ZVQqYBGJNsll ztR?~ZwJE2m0Zq=- zE67ftRLvbeQ}hC0`6p3;t>aVLkZpx}y81p$e?>w~y?AySnSko^Dpx>f<3m$DvW4Dp zqN~xi+52}^QNdcvfDGlr_HmzquU`w&7pPp7jK0=79<03WPYC=_7EaXFVctZf;r2)lm#WcWe7XZOGx(SKybCAtTYo6ZGK-UR0klf z#lm`#waUA-brS2EoZRd@9@eyoEhE^XOH2II9T{`Qd}a2E=WiFhbF;4~Y0Sp> z7M;NYLP$pJ2ezQ9L1s6kttuAaTgHVJ!NELYZK!q4{nM?_SDkv8Z+Uc}k`p5Y{y1+# zb|`F!Bx#w`Z+#UN7eOuG9nn-8r52^!Ki>L`mD)Kv!3m)@t;L2h*FE(wZ+-4cF$KRy z2UtnKEB?Lpxhf5H_w6ap_l31D?cd)4dn;{qFkODJRX`NO-wP`BVgtLX9qS=&PzrI? zIUu((H7bKdMPr9-j1Y^OFTUl!xbG^oI~5e=H~1NjEp9Qu#t;#QwLNIgt8*@ zn*4osIPs6`!TrO^fG19|98`o%lrS@+iKjCJ)mh>|?7%uM)DaJ@mj)k@F z6kA~4n8dLzo~F9^0ymf6&N83sh4;hbK_9HGd(&`SshE=~|9_g$*zC#ct+UvrVZ=z8 z6UgQZYaPycHPv~&E0IReD%Oa7sEa8o=9RwjRF&*`LZd%S)_52v^9uG~7LbsFu%Sqg zVW}<9Q_sJ<5b#C3j+v(ArvclRr-vb8R5A&+*RR}b+R1I7MesI@^$8!4bi9=(x<%|U zZ#n8lO59DJ6`dNqb5FX(q)FV2B^E|gLJK!T8Dvy{w;#)G5tkaMNY_ufuh8mfLQWNCe?OY4!LQI(@ zS3YS`k^&~wg{?{>%~uggRQZZ)`>KU)!eW(az!mO(k1C`1N(cURYiQifo*|vaCY?6q zMqg=Q%GV^mQY9}!a)WE^-FM)Q!JpY~)->zl871|cv5Nx062>Vjpwr`}KRF!fM9w?y z@}9(yw9XVLChYRWkayqdM35-bkmzai$)gM@x?{SBSUOn{uNWIEXaw$a`+h@S4DJ0o~Nnt&rsGaXXB}^ z78$w<15#9(B0!EkL=0y)aW_S&5-T#L;zo*NDhfSHIMC3dpI*LXo@kqr#t?V&$tNFw zN`J5zffHKZeQDiZlB36Ryi=43>jNo+r~)r{3Z8#XVMl^z0++pXJRqW!lpa>C#>q?*Qc^v@hgHuzkP-CXDZY*x|24dw^jAkS-R) zjR|;=Xlpt9Ui3pQtfh}ea!KH^npo_~kJQekXEbyDwTIL~iV+EI;|tvacgJ5ivfqMo z#AC~tT3GXcH4O1X;j)-v!tR6LjZ+ygbrM3=cuWMl#e*S<$O|7+*wXzZ%4(J41V@RZ zD#+_a!3KoP0F@K?p#n4rW26v>P=psa77m!HG*3Oa3!hMs|>};!m$vd z2KLV^<3V>wY}ootvMKHR5%sbuS>&Y%)eBH1#=@27dQr$&BZk|j5&*kZw;hYctMJ#? z8g{JecXn$nz>mUn!awlR=t17ck4>=y#Lrl7hgfM3$YcXt^v>F2ykqRAX2o;7zf)SV zLJs%GcKs}bP(}0rvS`lHfsQo>Gy2{sEA8}7r@-zD`l7wB{1~DQ>(I1JMfFr%;y#h23(KU~=6Z zSI|-LCVlbXRfo_6g?r{IEFOavDy_U886VK9gh16GLT+o7Y=BfXHyx}sJ>(K#Ud2#kwQ-tVs8~uOCPVvXy{cr z3Ox6A)U$?ADbvu7o9S(*LcN}837k&rC(4{4b$U2b7}KgWs475P4iPLOGl)I0bG?xe zuYYo`a!tatv2hU)h7OD33l^}aF*rb`Vxd2&Jz2cSyA+W_3Ze!lErE~EkExTpKeT)F zD*^63>#vqmROftsj**01*hA~;C|Uw_36+`A%&Du=GZ?3Ds1^_)*M$YU5fVVvf08Qo z6D{am&`o7l!nVVEY+K7pa5W>%kr6bIZ*N4W(eC^0jRc&EXX5eKJ@4%h_(bS55+avh zgE(s!d|8Kq5$KEIP6)N$tEXNCmt(+e9fgj(I6m|;#D+d7_+EmB@WK$XO+^!m#wbft zTcb$T%WqbAiwrPGH6k7hYYsbO+xkrJd5dG|j4$e&RBhNW(E%t&uB1|IR)!XYGe ze)NEX>d0y7QE^7l8(_N#q=z2d3j^*EMNE@aBlG@{njQsyfna)cWEZhyDzO-y;l2@H z+o1vTd!Q>RNQuBgB9~YYaZq780y{_?xG#*#jw-=61Z(R^L~;y0%AkMCrlw&Nf)-W+ zch{5JK}Tm*VS{`T97IG?*Z_VsTMg*V*<$T7CNUw&2pm9~g1m+4Lx^P=dpe69ay&V> z(7EA#Y`W=7vx)hJxSz$-s%LE(3!I3lRJOfDBTNpF=*2@UPurK;=b*4S?Gg~q0W`Ww zX;*LMaXdtUAmIZ8x|+iqLu)NI9IzPfwLvS=@IFU zB#6hX?P$ldG2HtfXO<=fTHO#1ggZlF(8Xu2iZdw>vEWa=r%C^8q%d8URH3Q$)nR)xZ% zA)#4M7YdQ#XPt+tv(_F~YA7_E#4J%fnlq&BLdbP>(;b1;nHd7#DcPps>v+LUfoGOb zLQNdx;C#aSWYbyZfL}U406Q4#AmG0xd8Az4Hp+198y#h|karY?WPjUjE%h?37XhXQ zx8`t(ZGEZBM!ZeUlDMs|ffLw6S)dXd^_PT8&g9AGNWG%_0O~yoDi}I77EUT{F}5x@ zl2|NQ+L)*ec;LXbv?COj0?13FmEnr@F|dA2)#EHiF44Y~7o3+9*p&rAhZ+HZfZkE} zdrl*A5?`0Pj^DZL-jJsNyFe&4iFEeWS+AZeNQhD*m1QZ$(YZu8Jo23>Nq}r}2B4*? z1Gn~!;>}Iw>hH$wSnF+sG;}0Wp=$z9>Jb^BXTVCLkJ6BIP8^ABEVeo55{4EMzyz|8 zzdEM0HA00cBAPYn=5sR+3tCV_7U*C)&dCLQ#Jq&{!vzd~?%H9Rb;}`E+Y8w{O~--$ zvH=Nenf%0oOLLa3Y+=p*5xJ=)4@_I%N9t|_-JYx0=9Jv;tu-EdYfwezs?Y$^gKXyI z@3@g@tgVHiC_}DZ`1Ka&$>h;ywco=^?X$ZE(w#F);bkaXq*s{4KhJqs7KVuVaKH6V zaV&2Ry5}PMnAP3CA-uhXA=hdxd58~-I0-&zLD)(+zXteuOQmW^br?&hyJSV@41XZ8`IUL zUdGOLt_@+3L$ZiwWn}?lvB-TdQVKa}iCBXEWY!u}1Z!Q)yI3VQ5=OH0aNgAHY8vMZ zad$8s{+Tw{Nb>Sl>IkQI;5X`X=2h5Dg?UTH8SU0o+J4PLnAuZiFWoUBS^Sizr@xExWjh~^|^zzd>5Se1^yun6>4aCCzoFU?UQJ~Vi zY*36u{$MbDEuAva-}pnm&ZQ3~%o9i*mf2U|Z#-&Zu>ALm9z$Z>HP6^(1cY@svYm}$ zb&FZ5%o85n2sR`ugi&C=;cJh2AV-4o5)rxtI}%F@tqU>4O+XT~MF3(LUThp-EK6NG z!79#!M^vFnI<1U$svlvtoNUdyOi*SACPx)5MrPk`?YI4OvE}en58t;Q>#Z3ixU~Payrj^_wned~q>#Rtx#9$Fd!g?Xc zB^*ol4PcwqIdhuiSRAqtzrs|iu!q+72d9yWtW)6o7hN1^iQ3(L{AmljycQ35=SnAa zJ*0c;pQ_9o-}H7fNc7*Gy7VmC%$`$HKU=D`FeW}woJ(@OcDy^#3X`$joh!K)&;_?n zoZG!RCRFizW-M3dX$T0W+!hv#w(^LAL;c&7AX&*OrpiF>2K$~TVB$tzkKJ{Mwdle6AT!7C}3>P`3v0qF#fc^## zA0>{%m$Fd_2_skq?8sGwfGrnpbfa#$zqZ&DWJ9~GJh(`4qPF&nfhmK<2WV2jP6-}f zm<%Zz$A4528`%(4w(1?Kn87qEAgv!KgvoQYp0F>ik9|IMH|k~JESL;>K;RF20^}BWz^5oQh=Rsk z4Y!sCIv{!^QlZ0y7|E21w9Q!sX%8wK3f@6uA|K8mcniUiw^B-oxs=^*SK>?;WALV% zVu{ux?hY5I_duxeD6n>Z-r%eJW|CyyD=kFJnRjBXp_+?TJRPS!U1TMg~H( za*v3mVvdsXAV&swU}5zu9xf=81{U8R67StdoCry?KGFq!5_u}tW7Mg-o}+_Df$5b_ zsGY%2l-`nYnA4*Yjq8Sz*o=o5w}@hq*#bGz4CLoKsh(8EOGC&&*-J0SwrXF9z(hlO zvD}WZZVoy}o(?VS8QZ847${1TFO#K-_(dnY@UlJ%B!}A^I#k_>_d`mT>JmCTk33kyt=U$fAAEwY>km$p;&#hEZL{BW|1N{7- zdp~B%GM!EE{2k$ke7qn&u7p~vt4sT4R_4(G)zN9%SNDGGM#{yO&w5jClT<3K)2KU^ zy(=mnJbLtaju9(tg2F%t7S{YYJ_J)sh#U3F$@J0uBW_Qo$(_F`Je(rf@ga)-tAm)Yaqa1@)x{m^f-}VzKKe_%7o~ zkjYClZSU3f&>rM3$jP^Glqla9^u!oVo06!CkVrM%0)D}zp{}!9TbQ;T)1e_mRPO2q zL^$M!Ph_>5SW=K(A4D3w>O@Kf22gBu!GZH zHbVL(BWobje8bQ*UlKMAPIe?@0w!obiI8jc{R;dPhv|r#3$I91S4ZiK5(7LH?0BuTzZqwR~2#;6| z{L3eo8YAq9n`xYqP{PtEN<@%|Q%J))SVK&^EBdSTE=`_#-&MOx1!kZE)|1RQVt*)( zMh|x^%HV?R(SfR0_|Qg1pQ#sl>XB*IY3q$MX%b^D7;e}u=1M7S7CYtOHueoD>s`1M zR)6Qb8|H5(r1vc3rk*c~?b)Rpc}JgUBMH#!qv&>!Zi<0A3WdEwjQDJUF_KPtP(=j7 zOhv|R-TPt`k;Us3r!G>4Gb&{QnO0C@P|?|c66LC8smThrr5!x!ib<~lRB>87IX!?wzd^9`ut$g5FS9A zqazUF6!x!TY!(ZZ`Dc8?N;K!!C`GQQ3j5Yxrn`z2EzLk+wOlxa3Q_$DWetg)oeYV& zIk`)fHg$@|cWl__Mg|WaNH8%2c@fVcwLdS|#rqJ&5S*jxY6zXP@_mpHFQCYk%bT&> zi(i!XK$Q;9VgbrZb?HxLaoL??7wL!<<)8z)R@P*zs@PfeFT}t|HMe3nLK>BoL!;)9 zxSkABE?72vOtUBZrZNegR1PlG#lj}N5$PdJ)xw*T^Gj97Op#HHC$URatMf>6sKN*Q zGnR;wX22kj2thpsSh#@lx5Fl-ih^Vd(cEQ*w_;ZdoL?0e0ffz#^bsj%A~i{M3MUB$ zg-?bxlod`mw|W1b78^%+Kn`Bt2oLfQ4;=I0(yRg5j9#)JS&BPArq!kpL)1<6TLIt} z1zm@iVD;+$JVLIXZ^?}az+G8(J^}Z=Up{SqYu@{1-Tu4De_y@#%l!|Q-}`0r<0nsk z^uM<#5zRMGo;=wM-%zRFzPk4TUp@hR#z9Jfe6+UuzF|qg*Jf^}k(T$MvgPbbDEE<7}#W^__$%I_(ZCAb+J!}$GF^6_VZ|#91tROf- z>2uZZRiSD&37(h4OpuA|Ls4^U#Dnag!P?`C*K1qB!X!pw_gf-AL0-W?TDHOHm5UZC|DbgAUPq3;{Rzm8I^r0!#-A=1iiR|ZQNjHRtB0^7L=q=6! z5+~NgXATSg6{AX_CqLvGRkKNnl;zl}pO6USo}Ro#jzacyX(R+KrBUKRL4*99 zUcWu1I~E7#@utc>i*BKBLv9D5_stHREU-p6^) zS6UT5NS31jqbhi2KM>3MRxa`Er^~1(XOkdHrt>R3Xs?=3gjgn~WSX=BaCEXT7kk)F z8zo8K*$I{h;|3h=F8pe6PV_3mrN^DCD67Sw7;EzvxgeU?hgFYd#82IO`@$;uot{&W zm|SevgY|G`z#$kfh^;z%9~DaB7R9`g?wD9vw2wj`Jc#wW22zbJk`XBW!+9uve5XZ~ zZy-NbzxZG{YlB!8^^;?89JF^V(Ey zOEWb|kGRP37P9JktHd|-T-i%3E=YCyOz)NZsKymj1m6d3dBGjvZxUHTkCjfP0qvi(B{fNZ74D7P#@lS)})6W)U52|S9i>+^CUK^B-EZJzK)Rcr<;qLwmo` zG_6GkZhes|yb3>++KflI|DzGk-(1aY78V!sRu-?cm3QB1?K`c%sF)PC!ZiImiH_c@ zQqXu+0&BhZ!92(iUurGo1_DUH1T`3(feaS_Fqh%GeDU^R zV0f6k3kcFSjcXa{qgLr8={F}DHObSCdfs^Von-@{fw8sX8)_bDWswBTW$;UHwH9*& z8X!|BWF2>b{TDZa)7Dpm-)2JAZ4A-yWSc3;ISU zZ9mf|5`P8KBC;Jh0GPT5`?T^!X2dWv_<1~%*Ive^Q@O0jpbmX=WVDz>-mGU$Knz=W zGdem7A)^G_DM%2}CZyI7SE|Iq+H^9Xw;m%uP1YPq?^D88wH~ng%9ba2!X1<3WbO=V(sk*XEA>3&Gnu2=wvc9OG*`{$BdhsA*8?4 zKcyO&up!kQ&Ecnw{w$MeI;(u)nRcQjMHq86s`}Cd<7ex!JX#JA#8j%nuc3xpPUA_> zCP={ibh-R#UACTuTF^X%q;Qux9g+4*^s}fMwtJih)DjY+>x&pY6?~}8l2IPTVWCY; z(d}AK#RMcxCIuKk=%EyF7gVm48C#Vq$ap3SQ`}Wym=Y&Cx0N6nh@WjZEX-#0yKQ6! z6sCAf3WL%}M#Tcfpn_Iwr!l6MAs1pC^zq*N ziHagiqWj8GgT!0FB9Jf;6Ve}8hcx|T>;I=5Hg5p<1`kWJ3WQyhSd7lmJJ8fk3pMug+Zop~O)G~Iq8L%5ZV-9@g#)og z+~gP%61cf$+614Kp@{VC!1`II(53t}3l$5DDT9vcTv5r#`fyqk@itb`E)kTnOq`V6 z2(WyQc@@<(he>u(s19ooC`*F)gdKoEmdxg9brm4&mxVR3l?zoo9$e%2z6Kk{Dnx!_?r!F zTqs{Q^J=lbC7in+QpfcP>Bd|J?5QOVYOHh#gRfh2__W}};2i+&|#hA6&{p7m(No23@c zsf2`}BbZxoI^19RP!UG4U0^r!De=M+++&Xy#C-yH>_nL(>gSOp>=*2sOJnRu`raUm zK~d`Qh?PuBp4p8o8L;2cvaIQ6mVzGBDI;IY5<*|nPh05ZW+^<3$bfLwK=@FrFosKs zdjwVr4J0_)gMq*i2jJs=&|K!WpU33?JtO&(z zvl=ss(Hj$tpldERqBx#h#YXxpLBUiM49WW@&WT2bQzptiDVFo`Dm}&9nWH(c(zUO- z%JOn#&&^UG2p;A_u!tuH0>begalbL|AuZ}hCHjG4_&K6*oLHLsYQ%z+U+w*42qE^Q zt~5e&J8Wchjxh!|bvuM1qm5}UA9r}?_6W$L=)xbC2@hDRJy62A9B2$_A%p-MCZRfd zmebBUQ2nUva(ed89c+5b%WI3R#m=HO7A-9F&#Ix6r z7YLU9Ba*?c6?ty-IO{eR%#)xdAzb$I8VQXLq?{>(jYUY>&GP}lv*5UF)sDc;~_;B2(0fNc(J`Rm{1rL zr)Bu0V{Orq7zBX9wtBgyFDB+(`1|4)Ypv;9BS4$Jc=sJW*4lQ9HeJ z@(KwrCeg$m*{UCkt1r#glf=n!k)S8DuW&!aUk1eo!(ljaRQ~W?6+|n&OZBc1#0Y4O z@vj~fF@FH8o6cBL9slaf0TChHloPUf?098M4w^giLorx1 zTzKrY=|fgG)oRj3p;xMOG*;BB_50&OajOe6R~uk?ajOEZ9Hc*QD~{ zMcCNHJ*C(AM?bgLc|#tSmZza#-J2^CD(%E%)yOn%z`~8yKI(XuWX{FL8NE`Nff7BJa>v{)4^yD>(I(R@>3u~(j?eBu zglp@4O1UwG6dGL7-bkC})+k*YWF{K}g8^FMrZ7Bwk;)OtI!wMsR4ifI9ni6O7kT0= z%zj9*c_nH`GSqUS@Wg88ks#5OP|ae9FxlOIMg|k*la=rSV;40r5JB@W^H}>z%R;-^DaJTP#*?G{Kj@bXC%@v{;9Jt7t+LMY{ux|D7x+F| z-{_!J+i{22OmBOi93+`hK5Pz2h*ap*NUxImI*PTItj!v8>W@Kot;-1*8&SSBav?UF z&h6+HWLFtfb#aTX2ztkhp-Mao6=DJkyPqPPOb~+2ri6Woh-Ka+zcIual&-cP6rO{1 zQWdzOkqg|DVMVg7>T(rE7Qs`sLtJXj=ttP~h?;y=6*mI5)y3KV(R(HOMPnk^CxI(l zwr&CazTjjc&_Lsr4;H?_oHZvdUV??EYe7!|x(bV$v8T#dS8Rz}cRgd>JlE*r`(wca z5iCt;Jrf8UFiKQ4p=*-BV}&~w?=@OgcGDG6pip3EH}Q~K+!kC0##_{vn~9R|0{k9p z8k3yC6YA6VtSFcBj^0QiW_tnViR`YxnmpNINo$Sy2)b6fOHdPNXP4^cTd z?GNcl^&C%ox(svhk_&#?6C9$%&PKXg$hIjsHTC(5OvoXw;Fpu2q|>cK?c~okEFz;qJ1T2T^O~$Pb?+b9*mWA z7_Q1pm&YOmg}>UvAu-)**S=O-6Ni|4G{uAL1k%~!9kJ4B%Wq+&?+Ob+2?p+ud)wu@;tvyh zs<~PN-T9)mr(l-2D7~ntPnlM(iC*2GEB-#b)0+1Nuyjub!c)te@V$5UQWNThwc?L1 zP-1NtKZWwIcoa#!!Lajk)1UKy;}gYsMhD3k&1~ProB%GC9r1Op5SHS-+XHlkNO6YbCGl&*kgaCac+QfXda5BZNVt$1Px_pITF2ZKEYfeZzBkA@ggk*>AA% zrJO4B_mxLVLR4(RjGXH^?~ao?-`HF(KCejoyPV~Q$4Fof_78^Xt>j5LgXQ7vNfC|b z-2!;ahFnkAW*>CB~edSDe*9}ZqRTTY-2%0W%;dC6@g!|-^IA(6nk3# z@>4)iTitSc&EV|JH^GPSxEfumhhVx9J%jt9mj<^AM^3_^{#egox*Nq>qkDMPD$y-{ zm3%P6VP{Bd6|P#F#k##-AQtd|ZmvsMwR*+$qgP0yPhorcNZLN;a&IrTNbsrS0x7@{ zKm=k%?KL(qXeaEME@aA-^0rEiM{R@NP`r$#r4AZJ<2uxODz~!8aCq3_mt(sF>YS>l z0s;aR0-M+&qj?$IW|-?(Ys~7)W+0o;QsGjG!?)%(3KbHr!c(27;Zotdk zE4ca1#?IefY|Ri{3h*LP*TFAsQ1G(FCfs4atT&wsfcXKCwhgi?7;I zktKb}pEV9bNJauyxyeO+Ii!q#_+WFrkM#NDj7@&QN)|i8OF%Xc`}3X^-qzt!bxf_F z%KGp|s;mN8bk@!x;Psb$=>In1IM94)6q{B2@I)P`B4hHef6qnKf;-@`>i-`1IbWNE zbTR?;2lJGINT>L_2z*7+52U)D3LufXNTCe%H@jCbBg%`dWok9P84gbj52x{*UTigA z`u8PCpZY{9!=;qJn&way22JO$5fAtRB(isY*>eV>9aLfCL6usCQojfzEu}^OV3G*9ynnQXj04%d5sUYs~Ny!!H zE-o^_?UA4gS7OpH%&!!ra2K+&Nv~k6miHXFvmLq3cM|d_y#_OaCf4pJ?uzXC2GBu zo?`WUWqYf%o9l^pLFGcx3(Z^P5m9aki|;zhOcRky3dhbliVM9gc$Ojvl-^emX;LyT zx!>kn&IefB=+5;Hsc#nbc_({O%z6zu2~NjE*LE7Y<^wcFfRmQ5N zQE(NXf>qP?u6fc*V_1w8y3e#rB0BH-b1`OgckRK-L#v^k-hY|m7I)3WN1uhAnBrxv z8VU%p3D>GT@iOfwpl9?6b->_g>Rww>t7r&79mUUr?x*J5isnoTjFOP0k$4tUQRT%c zkQ`R<$d!$J^j*g8QqHY1hSi7$>Qc0$(Ut;my79x8!6=3l_prAXs0tnN zkpgeJT#s_X&h{eMRD3$!y5)c_#U3+>Y_3X3Qyjf51PhHKz(5>x3iptDVY1kmrMgVX zAH%*A_JY84^0_2N!h*}9T44ACEnMd0;q1D9B#>xTp5C(`_ z2hW0G@iARHqWAd1%Q#C|3CLz)ONs;WhTZk`>=$JsvVm*cp3I);w5YaIXR$Btr&pP| z5fnM4*i%w#W`)Ql9tdwSnV!qrMG=~U#s@vfPnZf3?RgRsx@4x=6jP#%4RSY>z8WV7 zJHcn4ShP4Qq+P~ineDu*Lrsr4SXM})E*jWgfwo4vv6VIB{v)i&22UH)3{q%ubUVs0 zX|2vWq7m?N*O4@n&Mf=b4dH%d$%HXn#5L2ff7HrujJaJEvwy>Fob6RW>z$W3-)qdk z4`@3ZnP*pE5f?SXW!|`SlD7Tj0>HIIzX&zgY!VRlXBT>>)Gz(X;~C;zBhuK!AP-h) z9i#~`id;(Hr>(gVq0+GlwQ&MmWp^SxGAtX&D6g-v_-tVvgh)!pSM9J$@Q*^h^bTY{ zm2ZO#hxB%}dy6u1=o(UU{!Gzh(!D6Xz$C6B`I?Y2bqaHCRF*-4tK2>{H>>YT^EI3J zc}VSy(=qX3Kqc-{j3$bWJwyN-bcnK(-TJ~aQUo}oMS&5T1m}p7DB!d5FfIq>Z5A&? z3AN-_`q^Ys#o7GUG0LfurDIc%K&!w8?w4vf*(B02`LA1*zHWc3OC!Sta?ecr*A2@M;lrzr5@9Z zN8W5c<@TgS2CQGOKx{9OM`a7eRVo1ByKlEMirjHV9-Qd4PMywG3l+~rKypqe+1e}= z@FQ=VM4LGHy1JX0{Zl+6;lhk_ofY1H-BKLgj0L0}+$q?49xqt?w}s{lSeudbn0m#@!RK+N?NBpOS#-o2rw^zZj@Jf3zvy>wAqx%cgL?N%5YvS?CvaL;yxe8(e z_g6k8``6ln(yR)zVT=h9e+|f*(vhC^$fbtIws(QSq@FUl0lT@y7#UWF5&{R4@i82r zxVQ>P!a(EE_2s1{2y|KYMhm4aT6${fAwH+54e%K_V5HNJJAodk9}LcqFhiOh)OSoU|gjO>hTxC!Bt;ifKQ^Z6^T1S|BPfdE1(_qgX#zB)q@UmGZ7D)7K(vbcHvI z`Es?QE6TuPMHVZnpdNTlS2+spLhhFFv>s+zkutiM_>SD{&1(*GvIU%HYb2?p-t;DY~t=cM-#T z#ZkNQfuJN}&Hb0*(gYH&jHQeiZPubi!V*4!Fb5L@%K#J!!27Z`1Sw1;1>voo%o zNG5`gsGMY)z^J=-973EuxLU>~0EXgHAQ+Hy5p7~)oKvw)No8p!$>1e*H9_2))hby= z&zmKLBaM`wP|1TVl(di@2s%|O28$ai5mOFInIp*AvhIguCz?MeOmWfp z%DN3`YZeV<>|eq*i%VdL0@IvM5exQ{c)X6O4UB^+NBXEBU`JRl%9ba2yucuVUNx`B zNBeH5c8>s(l6m8f4#yMB)%1dwOl|nvds|DM&Iq{hMiUiTGBd4p;XNT5hwMy#IwGrY z!dS&6h-Y*KJ9Lo3a9Iv#!Ys0^$6pjYDZ#81`GTy5)Z)JhG~~cq(=D^F z2REeMNXHQN!L=*q(+MRREeHz{vpj0m8eUOWnr6JvtY~}mS?uiFV=8jm9ILDBF6}X1 z^=7HazIB8}5DiF*UBW`rjxf8}hxo-|`wJE1Z9St3yWy)l?gGqk+%k!;`8K*(N-(lA z?9+OtT=jso84!-y#tR8l!)9n2CD zuV(GvSLxdBpzBtMGZ*dy)(Xn}6u({zgCrfucBb5DmPk~Rx7jwmGFB=BNrO{Oz6M28 zSW3HT5jrHZ_8XD>JfPhwH;mE%&>O3CiEtgQMJ=5p0;z`_l#HM>EF`<~?@)@P$Fh)D z_Sl5FFGR&IsO#Ac6_};n8oZYc-cifF_rbgG{E*v|H($fD)RI2wwuo~>c?pp54U&Nm z5zFF%WMkrfEd9bec1zRu{Pu<7bZNZiy@_GMbVm0$nBAkn$xlR^;P1bq8=@qV0eY8l z(cn$>@^V;2EG|k~1iiZnxuM|wOJt@BgW$U0g)=r&KqO(%6xMq*TOiRCf@VG`yV1_L z_#O78UvwwRtG?2Fvi>G*tt+$IuP2nIJc6IkHI;ONcZE%gErJmi7N0TM=CveXA) zkjkTK=i-%wud?W2S&mTc*fT-?J}Zj7<_ySXP$X3gH~&>j+eWCvs)6;8?qJvsyKBtrvl?P629{kB6 zZzAstSvauG%8-Cv-IyaLpGS6y0@sVW1MdxxnA8T!BP#<{){QdjicCs&6|820oueTn z=NTxo33~~R9fPf5L`Gp;f(4LKN1zLKPKhyctIeI4*wkI_HU>K=INAvJ3V&vd_344d zd^ts~s|+n1kc(_(1Gq#oV6HSjsHa1G!st_+&kf09f2DP8%ldRcS@xTBBn_xa7t`uw zhUqyX5z=R4LUu|7n!=NShT5O%ipXGym~K%yV|0{-D7)EpUWia#@YI~Zs7j6#BjDN$ z=T^!}ukY+U-F~?v3Uk~v39YDR?tE8H9LXyq8~H>K|qGqTBgET z)JtO!8KX*)DB2GD%lEg6Zj2Hx7MT9vMl%FUq>D;N*S}y2n4dTI5=}gjE>2OgC5JQ^ z;A4hAiuNgyZH@S|M3=O6K!T-4b#Qn-NX1g0Yci#qsxwPFwX;d*6+Mf7H++-;i}?jF z2|2{>?UT(SigADC-o1CGwdjallC?6F<@j%0t0|>yge3@{L~kV!@-Y~) z&H92Pz-W=lXDZ_YIKd=E)4%}n9*@8eVd~y%lRlD|2cEesXSN?3V{(W!t^SS6MY{QHN%(_dYx;RjB29%7M#z~{W;c%Iz4(F4TW{D1y-Yd%> zcQ9lqzFWWJkXJP7j@X;(y&TAj!TFsHraxJrOa|L}Mv@Y2;0}y7|7N~yHxK1CsnCxX zb~dhBJRYNq@;o}MUb*H%H%(+tU^R?PThKVkwwTm2_AZhjNYE>6IAE7DG$eC`AAPc} zDY{+8dJ#Vjnywmng5Zo-R^k9SK-j2Nv(ZkGe|2#bH#HfYOtJmzbz@)#+c#q?cIW}M zQw;lNNmL(DPQ|#IZZ&`FEEIX*F+E=F&hbDGin?)Z*2hPGCveYqqoQX;o;s9=!)5vx zrWLjJ6)4OlsGttinl6lgS+PwV%#AXW*g=_;%rVYe2exRpE>dNFk%?l(!FUEF7L9ityi!{p$ z<~C!Kz*>Uf4rm|5v7piw#9ag}f}#j_Fz$jd!7Veg#YqL({M#oR_dfpUBN7p}w|1U1 z{MqPlbBD-cC1cGFjWk_X)pd!17LI>n*iQqp$cm-sWo>ce_W7Rs&2%m8UCr^*ESvB- za;eoFFkFwqnku_KsNc<%*i~ipA;RRY89zlTXKm%(cdETWK@Wif;E`ZriQa(19Ut6( zs#J*VSj>a;1yv#ZC~&xs}A&Q{x4 zf4;u_bp82P8{ceh;k0|TU43Dz&>b+6M};x0eT--{P^XDwCmb&XB?IRol+scm_lP~A z${0hkUoSN<-I6Ph$5_+opkMDW!C-~*~%?T9ym*yYm$EPk0 z8x|Z5ew7q@xMG!VPlG;F548N%mkVn@h@Gil5B;2=>7%nRE_&lTP$a$4`@-}TZZUqm z@B}wHCR4C}@A2Z%kdDra#V*K(V@V!0bhP$0brLBMd4k<+@AV(3#xSDE9En|-u*f{={%?3n3lL6YCmF017<1~HyU=H+{gO#n$_ZnL#lw! zG{pNg6pjAV^w*=&;b&-(Uk;D|bbfw(^x5mh6NjjlWFt+XKNp!VO_t-lT+GjUlF8j! zE79RbrpGa4)yd5s2y>QIAwen8dgo~usZGI#nYAW(W2H6~dJ`+mX6XKfLBa5j2lLnAW~Wg!;9Kr*cSYbmP8yez%r3{vt3LLQdlyPAEUJhhfek zH)JaQ1e%Li!bv%W@UYAJy)e$$1C|Te=KOR~1DH#y8@w$|HNU5G;19{XVy2ozAJRcA z9!zAJ(M3zo+RojsQytpk@TBvsAOqFz9I!om@#zNT*Z*lgFGLeQ-m}=c`*mzxsFa#3 z%b*EAc=w&B5FMP+RcV>m_sxq_23)ySv#6I8ZdfO<3dI^KSxDLvS2u%bS_<1z&JbBc ztL*TAiIXB3ICH_7V97gH(N9Hd`6`g&b`jxp1$U8O_AwrVYKVER2S|hpN$;@_YvRHo zaSZ{FF0Dw7<$@qDs5AVRpCpUfT72c)*}+2|N(Dn0>8iU1Jor-#E6Yp9%Fncz|SYMVrCfk&joq!M&X96}!d~17ZWp=!@$@O1NS_tSYFvty-A`;y? zJdtjm_TXy|uea7*McaL~{cLcy_c|7{2`B6o+UyroBqGFaW4p!9lTH>+)UNp)9575T;;L6xlu(Y5XEbR{8k{U)n z9z^z+JE03%(9^>T-NA2{JKa>q2vrxPUAb9g3VWt&gUSO&zkbLqMcc9ix9)PkskhwH z;Plqw zqTIcDdyGO)m}1vpckRMF>UerPD!68N6!m^tNMTRal`|5yrvDnY%ZLb-4YW;(VPr0X1cMT4&ZbP3a6@ly0%CD`L7p5JL1H?* zV%6|C7R}HIAzRO0?EFpJxBF4f6=D}zV-OcMLS#-NTJZfox@M;sLuoupZRP?JH4%+O z6D9da*`DE(flGXlb>WzamSV1VUeyZ(q?5DN4ZTwGM9k29ZzF{PBNR;(LRX=B$O$$o zY$@8d#9OoXns^ZX4f+Mck2O^YCEukx1jG%y=d3U|9R+1!d@A(<8MN@*HA~4jt*Va! zWi)tEc&A387}HI+#*o)m-<5P;u|fs=k2{nx-m0Eq6ED72^%!VQ{LOc5bu5a9(zxLQ zuN1Mut)5ACp<*egP~{-UU?nPU`?bYix>**e`bB+i?4J_D#BRi!olao)XgPp3Qn?T7-Bd`wI$Tm020}D^RaosNsZY7@=*IXe zJC6cTELhD^C@{Mk7tKX|bQRF-uB|^e3gCoASmBdM^u=61W9cOoxa6)CRz9afAtpJ} zh*fA+3b(6uzuvqmQ)l$2gir5SQqOl>ijMx@wq48=F0d|)`GG{bII$wYNy zWnWKVaaHX$MRk`+weVLD=gL2K@ztcAC_o~6cIEN;gmf!v+R8+!`CO}GZKQfNEljG9 z6&3S3z}gc{*lQU&zJ6&1(esnhKM@8cW<@djJH8@Lgt?lReBCF$&RIgeSgu2UuI%>o zR#)`DF8H~+!GQvMZNa_zt;i0lzGqEHPbizJ)o7ozq3#0nOZJc?$eEe3Xm$7C9|8DTnk{mi|SRQE|bp- z@xka~wDG+Ajx45?oQe1jp%ND`8dc^hrsWlMa=>nS=%l;=^*ydpa5-ZQ=*hr6s75R) zopGX0MUcG$PvrRb^|DG$KEgJv(C;$Sz+~E55&ZG4b^;IedQWL9jjb!Bj3Cr%$P=^1 zv}nJph2oIa((7%RZFnfFq1KM{c=4(=k=FX79ZF%O%*%7Q#CV0Uz@KIBX|GR93sQ`$ zlC708W)G0b1vIX&xS&!vbftGOTz9Ff{!PjYiU!FEG1omO5)w+B&pnrQIJq*-hj5W` zO;-U6T+|Fra5e54%gxGgD1A(J0=Zs86m?J`3n>D4T&OBXxK=E|$XrKX&IXCIM@@@g zXqR%IpesM8s)lgL(NYJD43IKq%=`-=h!B^Q8^kv&tCqGaSXa9>6KL35H&X1fEe6!K zqj%aI_QyBY$gx(9TbO4wyV#ztb>5x@9%VOf6!1P{#XJX=!c${#Qn-#HEI}XTHp}Zw z1FdFBzTP~p9zufiqVh@Daa4A&gK2DFq9Nw06#wvyMa(QwW#+iVvZKE?3=li&f3J zYo`%nXOVF9aJ2}4=&*&J2CCdCqnpki!q;0z*Gi*laYj!RU{ru7@Cyf=auf?V;^h^U z;s4TQE$}rMAngng+)X4dc2z^STT>>}uF}CbD;E2vUR5m&t@9YaD6Wvuod(B>zT+?N z+jg;kwO~CcQ05rQEYZC(ZFVC{fEWSX@%NY#6xALh^rj?v)Nx766FDdpBQIazUMJy3 zLVuao+?wgkW4QNP)?v|>Fc-p#5fm7ZN9IS08hOVfM0vTjMpT$qSc8zfK+LdK$kRdeqR31Uu{4s6>mq5|xZ~+u9S74wZ_5}^p`navW=30! zpR}|unk9;l7fNfm<2Zz)8&kgJmg5ZtkJvPXsfD%l(GIHh+{UJtY4d;eL}#e0bq1g) zM!DFy6Z(~f-PxY*yx+aH%5q7B>5qnsIYT7AjB4-fq)cn z(#8-*dYdA2mJs;o{wi=1CTR~PzrF?uVF`tp-i(gPF;l<|ytRzi*85I-Em$=U7lnBj zZ%aMXW{W?2kbnef<7^>Dpv{fga@$$eMw z=8U59Q42F(J)Em6a)%D zFf9XILbRv8KRfO+9^K<@aC@Nr$$9J$4i@Y4VVHTzag+ZwnUm+I~>@f@#~>iSVJ^R&1!vW!A>r>>W~v%`KCD zkggRdDZ9)8Mrk|d4h*90Db-O!p)V+aUC8wE48AW~vw!yb@xr|iKKSFp1w!q-e1H4d zSLCap-alSA8XhP|x|vS)9xuE;J3D=}y87nLo0T^YR>qUV)orNTboGZHepr3;VDIm* z3H@1URw3|`9(E^dCPeUW{9^SINusM{V^ZjNBLvH+k8(f{R_;gAk@}XUP|WOhsHKf9 z=WHb=gd=KkJ&&SSUvVeMSzx(xP@25U)IbzE@%*3kVDan8WIRcf0f8jZwE$0#KPrYSg_O`Bat6!x6?PA# z;M{^!i)UB!ZswqjLT(+)?kUAeRI)x){k8yD~0GRux-KU=K3pF7ui$(4JaFKp;v13+nh>>UBRjzl>Ju(aDE)YZo$YgVJZC-C%=_-k)AdDp`oR-2(q9E<+3AZ5J z#H&nVQDe9gq?7dhDS66X()TySuh+P{+fV#aO3?}&X7E|_$AHD*&-SDG6?-!51#G-T z`Y8^Yj}I&W|6v_~AJ8R7fa!n{fC$D2$7oHYJlOU1!xwOucKT6RB%M8&Q;(A896wW5~FdTY-l`uR-L|fOau&szqgo3>R5x6G2@Itu8C^uHuRqk4WIt=A=2+#;LF-0kM7e7>5p-C~? z1X23daYKg|&O!2J6~l%sV)1}DM3ioZvDd{J-p9p;aGVCgOKf6VFlvM;tsukh`AAjE zywf+b60TZ$W9+GhL9O)e5g)%tiMhd=8nyXM+OecqS-k}}#9)llBYb%hZf)Ih>uFfC zdZYlx+0O;iBU7~_YP?_XzVl2mhPP@I3{jSmQ+_;89Dn3WEjDz67@-!x4U*z{f~eix z7GDI4tgWdUaEzgys-HAaHMGW0VCf|$1+amYk(vR0{3VrFg#REwL2(1CXBAJ^zQ#QS| zuWr!pwz!Ul#D-)a&K16Bbm89}vLlQ}$1uy1K6%fXeGQ3VOL7SCZ$Uw>P%h;aadd^b@ zE*?jzQ!QC`7(H3~;)7ZMRx{hGT9#PGa@OdamcILrS<_s~Q5_$9Jjgkp#i0t>#>%3` z+JwkrF`+`kAWu?5pfUJYA@>CU+q;RliOj1rXm7@JJXZkNO<10bZI!29(_00oBm)=H zamsFyR4H*wkubJzmmO7g$_?~Fi--z5qd(h+7l9NYCsgqkBxP#ulE-@dIdFF5zF)D~ zYg*GS!_U#Fr+UHs6d%z*o^u+Wi{gkk&21=HYRL_l{;V4X=&P2g14?RHnQ(;X6eL${ zro9o@OTszfncq{54#krRj$9GXhtusKqt&aAn{ezougZ|Nv=@~cG+#(K94vsTQipw| zp~|qzWB}YL%M-ahIxaV67b)A`{Ozs*D2qzVF(oS1*om#f**1ANJnn1YJl| zNzmdfL$1x3%s&N(=$IoSM?st;(M-58$^W&h6JacisB(Nm299BSTz7y$T=d{jS=-!; zg2EWpX+3A_k)tO@Fv-tDA9*$~^768!R;r3=1A+;7NY|ryN>|?MOagOFDSNBKEMaw+ zIhICJhK%n;8Azx;A*v7%?(LluDy#Uy`fM)PfR?RM6u&A^7l*sj7;Unf<^yD2|NOEj<*4fSS)}w3${U>ddJ7a9jK+ ztVeJFCE+CyGh(xjBixAiQ$BsSauyo2Smdp-A*;{2)35IWUX)@?_y?@o|7+z z8xj2J!7IQ}rWkW9d(@v0uBV~UQ!U|A>(2(+u2Minsqz7nrc6Ww$Q!rlH1lS#_1xP#4P63M*A8faW3-Nn?A1-f3tp3IILK2ax?_}Gh zT3*Z2_o@p}duhYfgnsD=Y79zy3f2Z=F(!3fb}+M)waZUNXW<7Uf}zq>2>{Shg>eV( zZ#Xka5~Ll-W&wLrqLP=(PBtia8J`3M)ZtoIp&a2V3B`*&FNw7D;z6!jZ7kq~kX2E; zzz}~6eAV#JMD@Fo)6QY7&GrzZyOpgrMoImirdn~sW{E0cCn|Y3-sk4-Qd8{<^<0yW zIoWQpo8D`dUQ=ZJZnJcX>TNR1Z*r#%@4QcQ65Pq}_q=7rX$F(G%No2{`XkS+9FKna zj1l;}cV>UwdH)gvZm;{g*N)5B!~Fs;yq(nDv^C_ut{;hhY)<7-ku;9A+rNuiK zAzPc9!}?elSNcC1k;Ha4+u)TqTP?g?T4!ZdNs(I0x=mwF|Cai{O|lBhF-+MAChE$4 z{CwPOo>QwAt$1ib2lM90W&OE=QhmH0MQ)7EF06&*1{Fc32UuxsVTcxS*IO0oml&~+&bspw!LBvouuEL0*iwZg6Ja-sVwr+kHMD@Cuv2O@t4@;uhb} z&xvCnASW2>K7BpGWo92up&lIBXDM2r_4svnljrx&e4M_Q;dkQIh{KXge;WCGJAaDS ziJ4R6fHg3xoikYq&x-CU!asRW#w%%(W?MkRq4@Q9Phr7^IBNAs1e2w9dP-%UV9RZJ z&+Nn$w&P;=8#H&)!qYTwKPjZ33IK}hZEN(~0FIo^}8G2c*v!`^K zMV$&BmJfH1t5DayG_ynyk=QjRi}6=!@f5+#COBpfnuID<+xisB$`L99j?(XGH;k?X z7sWEWK<@#t0i>by!LDoMMdF+ZrXJ~GIzLoO6fOjfhKpAA9NSUkm}(o_X4+NKh$Ew{ zC)C8OGpf<;#=dZjX1R@*zU=bFs6%$Q%W^RM$oHPZt)I-$ibXokdG&sp<%kEqkN!l|C->Et37!ZFJitb4>``ONbWol`MB;+XCwxJ84} zBj6DRE6e49L#n(+#VHRy% zRLE)6s}JW&sw&=V31BzyOj~YMEQLA4dsCZGYEgpUO6ig{f`3LR`B(Tg%fHifC57G0}21DyI)SEW} zGZ>iC+LK}L2L)`E;%4g{E(S92g#)XVOGpCg;ZP~KN&YK~=|N)WSmZHDFPn^WJe(Y+ zhEesJ_EYH~ooRYaTtx?Ll62Y}N3ApK5@BD8m-+H@b9W#^(gwv^bI7T2gm^{SsX~5Z zIF6d+3AJl!dnnpA&Qg*Izx8LLV679Tsvm{w+;c{oJDVi%q zH9_5p#f2-TEF%5=#v&jK>zdv(VrFE5wg$~lT!-l+DV-GGkZ+Jj zI05{FR>=2k{1Yw}bsn;J{qS=SwlJjY@x=(H?^~I4WsA`-5_j!UoWep3OrP<#UullG zKr4;$GX`BP=xF$hm8aHyjpa;BG1up|Y#VisCz2Ny^tFU`WyA% zHbrYwbJ!ZAX{fbyjrQupxiVYIUEW%t)yC%f&U!m$?rPhq02(o5uH=vE%@#;+fG3OF zo9!J{CRgRvnW1%i^Qsvy{npd|ZXO)2a#Xy0h=zhphE z1^LL-@d$^4^aqjr)>jg3>D!y%Zaul=e6XjcBjV}bcP%gfO%}8Dd}n*-Zx=2GiCE%! zfX$Xy+mY(?yFthnwY_yQjQ;h@3N1`71lGy%+4!dbD#55Ra`U@EDlF+@NX@^dPMl^T z+hyqlayvVKM|4j7WoFpUHmzTFW#v}wH1fmL_*NnvoL(j||JeiHEO5(or-};;t_!Ia zx#i1-Fg7&id3+T?QILe0f~zFiMKhX7m{QAG6IB{lhAVePH%;zC^-JgOf#*feooG+%_;y;<{6YZFpz;OOar%_zS(A88aFxO! zM>bjvGiuKWhjqQhlpWZ0L0pC*Q6VTYF4yM0C3o9pn=`z7yfC7qIc(&DZE4Z4 zmtrG*mmoZG&3oXwPQcgdaxv~pO6Mx?|=O!IN=mkhTFnUsjt}00)h#15yK5 zYv~%XFQ_-=MfCgTlpq43ym)Dv;uanB%%<8JT2z*v^pEX?nzWZK*#$7yUW$JP;HE=u0!rlpQ-Nm#-9s+R<@sXTv~?dX%H+9k##UP1V_=!W>yhr z5k7f{9G?PiC!1EF3XUme{QMFB?~Zzvn*r$T&Dd=}8YVBu%{)a)Wm-$YMxUntXfZZ%Kz%axsogES8q=httr6x%s{A(gAK)BhIKQn5L6RcpnT#v)^2;-o^7qL z&!XGutMW~OBt+qg zBp42s=;}ZIe!gq%@c^JeN!`hNuPd<#;G8|Iz4n?1EQ*6Gj@7IX5s3L}Obx60T$zf= zEP-$ur+zUcjKmfri8X%P@by36I`MoQ=eHn=1f;wyvd8VGxVC93T~wd$dKHR`7H`5_ zIkv@npYbBhjytUJd>Ru2*-Llr%Y`Q|MLKBEFlWYXU{{H6>FwKbesbq|nL{X8ILiW7 zw;f&K?(K-7MXF#dEOE4`^{I7ZnQ|DBsFGOD0?nn{U-(O#B9x;g2_evg(2N*9rABA; zpgx6{o*g|zT{*udUgj{*exV2>E9)53y38Ym`gmmy5yZDBYV%l7N+r}4J7EV`SJbah zirpP_{wOZn#vm=)7(hd6usY+FHWy)2P#giZG6Y!krNvQ8#w#V!AT+dO<^WH`9B-wb z0c@SpwH#lf(lo4EMRKVW(#2yHkx-<*J6a|1#>~NRNdAM=oT~avB~Q%`*^J7^gjU}s zhAr^Lj2sKqY;Glz1te!F_wA&y*~lH@>k>j_Z8+|vmaUTvHL<}7vfA2V3qcaK1rJ)} zw%8!oyC|{v(W!Tcqxlw;l6H^-?8cjB3uY)hF5XvL9B;>)6~8z6H}+SX6E$Xo&Pms8 z@G2-;0Uejv?|PAv#!wMnXbWlz0fLhfdA&budDWq`{o+YAA$7QnHki!4vT%bOE|E!j zR%+@chU_EGh&?M0vhsT8jthdPgjsn`qhjJ3gDWG-xFZ-R9IET?^(G6t$e-)U&mAm3 z(ajy*93o24AG#JOyQY)o42CQzA)aE>DPvXBb5nu}LtH2@WCy~!h1?FM3YXTrF(yNl zOku}P!njst{&a!_Rn@t?o~u0Kz%R%Zr};oAE=oWV@O{v{8NZtdaEQ5Q7g&jiBa&X1 zE<_sMViqhHnn$d9VQ85gAdEVeHXbc~rIr@yPEgXm{Nm0{Yy;GH4QhhVtF5^zOcd`; z3K}0>j5paFsbb@U+>4gU2^oG4zvy&VVDW8mgVE6RtG%cH(cit-yu9C1Gl0L54vSK= zzXYZGlbg<0^5=jgI=Ucb#ji4xpC)X5cCA82UKa7YmoGJ)VgjFDNWkvA}9a!`*(*9wdP*}s9bHviHdZ+>MaA7STH^TnSQQ^iUF4lCe9nhJ6id$3191WM z(qb>gf-To#X=EB%Dvjz&q*8txMCSk}hcl_P#3%DTDMYQOL~#209AYi3(NFPUbX=vM zslbv!lnjE+m|KtpaL=(vISFEkdrBGC)T1Dkr2!^`6hGCQC0xjKwFRhJJrOLu1J-Up zAfvYhxb|lrr73!BpGyCR5ixEqpJ&>5!iBVJgT;iTmIUiO2J+Z!w{<}sTiYyklrO)S zK{bnQ>JpWzg`)A=w<_+UK=P>7ZU$RuL}Dg9zFYCx;f=T=9c1TizVaV$Sfd}mdMxH z-+%V><-R!?%J=N%@ciPz^~uTb>hkZFXgxgo`!Q8FE(zSc-!u#1yM^{(5~Rz)C@%75 z%xcpjj3hS;FMfKtFYzDJ!+)_#Be5oPIXO>5f&eD9>SNOJxigH+Li>}+;SWc`d-bt$CGdMXWH}>kyDV(25pjHTjXS}Gq`E`_V)EhB{+M4d1MP@098kB-GM^X?Eo=`!T61wE^jbEekG2IW?`gr8n zNs0`Gmwq(pXVP&cp|H0MJTG=63W3<6Y9Kl6SHh+}jeKQWj6ImWPkVQG%mc<}i4=`^ zc9dlj6+V?A(I!PXwr(jND;?=enbWr?X+W(oQvrqt!SzeclF+orPC5OazxSmoiS(>k zNupG~CSuXGWUqq{=GY;6s)S;!B}I+SuFdpmFr)An)l4w8Is{X3u_;-OT08YiQkmXj z!Aj0#bRkUomdVni;p*gN2jrXrahnCxQPHwaWt$!_Fs6%3lEw)s(jBBK@YV3VS$EKv z=Z(w}@gbFg@pF%-WP{I|TvlM~o~qhgc@K+Uy%`QbSI<-4 zMK9csWFzn04{Fxzm=ScN)4XX00XKbn>!?sp$2m{Qj32D2`1z9{z-PR1Ny+;f7@00a zuPG$#_4S2Tn3^!ysTaG;13N*5kQ|LJ4`~Q5th0;(YQXs4pv=W!q`OSmi@`4dEp0R; z?lOqi^wE2|W4w5KNb5BPtKP45fl%2hmqoCos^9 z(eazOP(zmw4A{mXQ=ZC(5wBH5?z;u=iwP{*!@zWcD|M*iV{V z@{9L3f2dT>Oa&{dU01iP)6C-Uo15m32{ZTX&Dl#L@^B*lGwtzsE<9 zJN9wl9|w=S{_EiJ=YeC#;nDqgP=*lJC@nys(Cj&Ia$J@K=aW1kY^(EZH`E>^?V}Oa08gTD6=9$;MbX^C6*mb zDcPcuELWIRCaNMy2S+42vV3y-CLF4+7B=AZelhVpW=nI9Z7t=VSVlFm>ce*}FbFNIFIm z;vP;h()@wf`%GRJLC=9|0!i*4e|ddC!n9T&!f!oCRmRls7TRmjaId!JD#%kUw;~vy039#L^9pA*@qpk_bXN2B~fz6jy*!RrrAIw%k_&tB}Z1grncUYw583L+|9PmV1 z@9{y>nag~#K5LS=t0>4?K5&>^dBElfv|2*yV1d)Lk|Y_0uO(&jS1`26<|faz1VhQO zCPR9c1UuRGj67hArPTyMn4v#%gPlurCWr#j;F&mE% z2ox7XK}M~{BX)|_r?-PsuUKad3@;<@hI8*NL00CmTr*SfVftWsXV|)eFOFs|a-Nt~ z4KU%Vn?5A|OKBOgR@pw!mN3D(9e$gNF0Lba`DDz!_NY)!mnfMHnIct4YF~9#APY9$ zQbEC6`TlTtF5BPar63OD0*skTOnk@k>Biq{0`h=*0av3Fo6B-Swc2TuFofAFUlUP9 z4YwDu{ltW^-vKE6NR?zmY6Ex|Zeek2Tw^P+K}bAp0la-%*9*HtILF%SGWS0uD|rKS zEI=z9Dk?ncz6T)ab(e(@rF-@J^b0~uM?qLC%0cl6DXU6@rM+0;p!61E8rlr;n*0}Fjy z#lwV9r*hA#9{b>!K3G^}tS^rh83s31qaD@*nY3w0?Gn|tLe6S63u&?74}K3wt)Gpx zij?X1#Xvnkb;H^zR-3S;lq2WH8YWzL5^Xi%s+)Km>FAV^_GQqKy317*kF8S9k^yK6 z*Vn-Ja|)R`%p9=VyDS0P*z#{*e$iJbM+H|o-vrF?d;pnf$&?j_z1DWAYO9o%uoKmi zV!PeT;RTiR6+@$8jp&M4o{td_EDEWfP0K?(^ib3a2JcT#mPBh7o>XrB1Xq<#)k)&( zwyE%*;!}#JkjRVe0%RaO(W_*Kt~roEvv6I2N&fe88GgJEg+?W*5yM%yiHDcI^RMaUvDaDiwPQuF7)}Sh&Jy}G;?ND5b2-AtX-f-1{ zQ~W($7OtYPE`p@S*B5F>;NI|XSG{1V(E!?Y!8PGk^X^sBIRdoYjO_k_5f_=HK}6ki zwhn?VH+EZE>oRXO4;00>6%-!Q*Pe%r!!kRHwfS{!C5whKLn{0TYZaW9o9^%iIfd21 zb}d~;j6o|RdZAT@ltsUF_L~Nxm}?|dj&7o5a>Rx-$ieT#uE@F{qj8GRzz1IBRQTrs6g>tqd%^h@&K?huXhCvcOuX8B zyA-~8(z@81@q2$MkULiC6S32@Z5;(UycWn3)*MMuBmgKM0xf7>`GP^uYEco03ZV4X zlqJnbfil-;|8%zoN@J3MvHck}H-{xqp9%ttdZBwHvBM?E+_7#&~3hUt1# z6vO8j!AS;cL592hqCa_y0ZMvf3Rp@qsQpZVj`W}3RErv*sUV0P91v-aIKfo#JQ}?= zY%$((Y6@S%pTxXt5jwqMx`*yOa>Aa+g-HuG8Q43Co-6t*PM}L^(J$a-Awvg%I1!<` zZiPTOa4Gj$kvO3p5m$zZn54S6p}>=nwpf`rf#L6|{;jmfkcGHGp7X!EAFgezOBv>O7TKy0k27FPfv6GjeXMof*OUMC*f6bn1%vaEtJlMp#gxZpF5ONYWDLxMR69hJoY}eN89v0UkgqP=f zQAn<#_?KaNmQ|9lfVWQ5WP5EO56KW1)05JCz7}f}YhlzOY%*FjJX|c=JdMh6qmCtF znq+|C58`pzT%SjYKzsuQ*Fx;v#`-wCE)#?*?n4OQa7(4>lf)qNfMprGh>^{A5hA1^ z3wwgJ4;`pzfLY&0lHG^_zij@G>M$BVt9Fn!;5Lu2gF7zzqZXWWhJr7)$!tMAA?qN= z1o5=SiJ+cQeFX*wX{SXwr3M8VOLC1@ddAFaWWDCiLHw8?`KSj;;QqdEWen00xmRdn){4^!8S7)7^_I z$yj7_Xz*8Ca}^gW7X7*0cFqV}5ev@|7Aa)Ge+@dY;>iQafQ2^=&#dGF!l4R@8(;Lb z1|!h$Xc}Mq?3wnoYOGiCR~Ynv$3i5Rvyuo4?zqe`zLI|mB#2n{oE??QODX<8$M2@C zA?DHF6_t&)IUSCdeXjao#e0+3_0h#EzWyXs{vU%tA8~%4{Wj6SbFE!o+}>0bP`$%97O>= z5HXe=jm?UMZ`JAtNu&%asA!Bi&D~)xC&Hv=?~-y|?4d>8y~|S^w6!Vs_h$*;lz_@~ z>)$-Fe~ew&vKfv&WhNnyi|>#UMAlHxilF&dt@=QzJer#e?*@TM|6g*P+QN$P-h>D$ zq5*w2E!~_Js0N^mnqWzi&F)xRE<)uuF(~--JkTy`f*4Ca!2)f6#4nE9UkuL2nAoak z{_A=ipYmF=RWPgx(3MzwnO$aqg7-IN-}f$wsm&^q|DdAeMUZ%+x&y_9iljwMz1}2 z3T9)7qEj1FdGJ^XRvg$E;3BA7)HGlr4@+XdoGs|cvfYg8&Sa160$TodBn(7+@Enr3 zI^acx66k+-D3!In%W-y%7Xxj9dNxGm1h@$>6Jvge64n&0Vl8Hq zfRS4`PKz9dFe10oT7(BtHnP}e(rQaIa!{|Kx}>*sM~_(o`I1H(N75IjOFq>teNieC z#T8Y@i9n+=GYFM0ENCb>lg|)f%q&yZ_WR{yi{>vk=~bIvHkaUywsn9t1~?P5z`!g^ zbRzF*AV=53WIW`S>JV{@6*UAsbsvP2e>c)IF*|uXa{^BGpXm(2T>B!p&#Hr(T@rOR zHLb+*InL_&SQdLLo}m1igJ^>7L~B(zu!&=m*EG~zXyG3gPnw}3(;x^UgeI?o5uW(z zlQ$Y<|2ek!)D(kaZ{Vj3$U)vK4O)g6E^nJ)5-C!~iQOo-PlS*ZG#oxHj~dp=X>AsA zIU%#*W(%|Afc6cka-ZV`y(KFY+`#I7J~&bth3^(N7v`xcWP4HU6)|p4PLL0lpdx4b z;-S827K1lpz4zFPR$Nj8tr?$NbgBu22trXdp)aSE!-TX#yRlZj|(1<$LE&N>6M|%0U zm;3!6n}s)5SLfe6eE9C&yJezsmWghAIJz9`|4$fiN#8}fw$Q|JS&k+;qvmKPw&Gqk z_19rGEXcP(Ij00h(9QCgI@U(b2@<%mycWz!QCu^w3wg}xfH6`npnSnd3(GV=>W3+Y zeji`d988PA#~uIX`m$)n-gkX**AxL;!)z<^M^r}j;WfI8uCE@=RTwqD7BsETZAt#V z?`!gZMJAh;4f#cQ!v8fwI@N-Fo7u+wCYJjaC)7Z%9?g{)7}xgsOvMnae{lxeva-Dl zrSxwT-1#j<79!P8eQ=s;IcW^au3X%TQV}EX!EWSIk#xRXXbEVIPWBpvH zTHV4AfUbTvL8}e9*0~0NL}x@YDxW{uzuhj7gN8jgcW8_WStuGJLtLY;y7@U;ZA-0# zci`p=Q@q>I_wfbBs-QtzEhK5UXRRPVl-BSF+RJs)vVt1&H;Ktp^b(9hn@!S?7nwWCJXo4wsCz2~pTI)|dIVPF!>ZXk*6_(#Op+rmFIjtP0dLkT zTgrqqaC&`mH6rRCS%dhppmr2yD*jxz;QNg__&(a&gMq;%n&nk$9IIBZjPQ0&W?=}G z=nJ2F{m>1EnTyTsV}UB<8=r{9Pmyp!zE%LTnLMlzf(7fA#%su=E$3!Bod7R%3c+R) zB~qXsLeOOhKX?+i9DWxnnAJofr52}pIh0g;h21fxCZv3T@xlMW5~0(Q09M6T=wsLc z!&J7h0wEN%knFhBY{UQ%ETXc7fXQJ9O^_9)B-)w*Sucb>N|JAoJUmpbpUMJDAdpo!=tOfY%c$)nDV2NXIkOUm>h@^2KI zSDF_6K$i3tp~9yuE||JDRcRyT2MRj3ARN8ln9}v%)bNjrVTC^_Y@6dL+wthNAj6k5 z2pqsyY>V+fqg6)$3m2?b&n9Oo4*s8}pt)SwytqD~5&humnq)I8gJ-)&3j)qW>JrN$ zV~vdK(E{o9T$d#k+*w(gN?U-dk&v3U(B`5aez=vk)DPVhq(tM+K}YN|tCNJU0*1f` zMX8Ah#I+Sfpy;(UAcoJKO_hpOEgLUprt`CPt+FE4T*Wowz9riT)m}ZCD~&wA)?(zJ z5VLd7Rr9%YjF;AfDi9;Ikn*Fp(OT6U^49dY(c)VZrU)T@@k{sF)81UGH#K@?PH20qGcJ@1S!M;|BCKr|` zz>{bu7FpJano;l&Jk#A7e(9#iF-;LxgJ_Y*MubSfA+##$2X_2W2Phx?ZLUm`;G^h8 z3dYaOFB#hVCft^6BPOxrzP2Ubc|~CD823&2TS9 zSVoIgpK3rSh*#e=T_V@RipZ9u=K3gacso&?a z8(TJ0{mP?tayRT1To`(XFYFx(&Ylhj%H=ZOi&E@D7_R00jStMvBuDcG zIeL+IYrm&$7s8m=kNB+#`^DRQalvpFEkokO;8+qH@Jq)-`L??LHqw^S>aW73&yuZK z*>M~pVOh}vGCe{s!#te)TC&v_S0}`c8?>s%Ro7R_HMFf<9^(2%$dOLKHI5l1PDK`n z%g4)HZTlACS#XJ++T_tlIs}Pg7Rz%rI=sdiY84f%P6gk3B!lGkqKd_=H_Mv&^>%V7|sTe$?d%98%|dzcQSVQEQoDjoTo6FFCRSfs%bnK$qfrhX}_V4!!j8+!q!tO zT`B{zoObP*GMP(0puUs2Qmeqx!Ld(GNJba-H9Izy&R*?g%J>sLrq)dEWJ)p{-lkBT zL}bp6gBgon0C|ci7DWNS99ZLN3u#=bePu|QMo*G{bFl+F-kuT!SK~1IT*fJThx{>NhUElphV6S@FP5hxQxGgCk(nvno5H@nk7nLFgzX#f-)by1r4nESGWgJtASI}SRS{?k6A~#lWrOAhid$2(gj+p?p9%d=hXD6%ba~n=TAaj)NUCZtPLShief#_1 z;>iy`eEU%ULXNyN`VT({1VXqnJ16q*NQksCj|sRxWqbR)P5TZ~aiFs4z%_2*Fh;{Q z2V0j7i=F4kVom2L9fXqpp^ATDh(eODTizBT!ntJrNf7~C0%(l;Sl(TS_5`Chvs@w% znI)O~Jlt4L2^oP&SU|}IO-T4nndlYMRiK|bWP~H82k*rQPq5$XD0FaCB28XRKzSzu zkWW#FAx35|=ut8bA}R-gc00iKg5@r<695~;0zBA391g5Yp$g>XoW=ZY8em+GO;0u{ zFHT%hzR0^I@^MqOumHATTvEn4vVqK&E+#TX*t2ztguG$WVf&V5Ta<4~<4tbVmIJG* z>wi~`4HD8^hGO!dvFMJnk_Bs{L7~xtm?g0CBJ^?Wi{a_`_teccS$1rd5e264*1HBS zG_AOWxyo6WK6J#Ivn3*u54rf(`AGkn;zXAPPRZLOVau0(Fdm%@FEFuP4Ilo*5I<-H zdD`2B-=~i6I(DusT59R&|JXI! z_Lt6eCdKKEW`z9mk8ikIZ}HWmxw6yZOOqyqkIq)n{rWR^p=HSUD`|#s zho|($U3gM_kYL#AjM~S~BSPZ`R({gMNDWEyM5m7C6P&q!p5@J*V$9Ilw+}1DK_{Ik z&|$@G!mANHioh@Db0v1%EMr@lGAd zx#y@g&9v1U&7b*>xRbUZ9Tn!QI4j09Me(zhw27hx_wkln8RJ7e#jd{LL;zU=_@0$2 zS2ebjV^+G&wEY-6miP!8EF#ubJqD|gu}vA~qBVw7xkAOh~ZbE`wvl*yUEcMH`sH7b3wMKm3FeOd&QyK)saFR~HdV(RN#BBzWzia~uSJT2 z^tg~qkj*l`PoXi$tI=bU44)jGP#TF8VPY!YVDoBQjP8WS)0T|5Dg;WaNeGnL+C2D! z3(yD%`npooO(iaH&ay-hWNmZ8Nv?&dOuBucs-_8#g)cZ%vS-u02=$VHA1|WeG`AMz zm|~n>&Ga!;OxHHrGgTvUX@78BnnlSyq0Tq|6EZ1I;S51@ zD#rp*Y^ZP>_}g=CH4N5y(7^>PP@^b$blT<-(} zm;b6MZ2cwkk&y$s8);|jLQf*G;sK=o$pR587scsi`Q%z~eBKppHWXy@qgF2XrGoVOJcp5M?4gMMhGU?|^4~ z1L1U0BZETV(?Ug6M>lLTsGh04rbtd#=9Hf173tVcBqv5zS)!M zC9?%$oUQng>`-0|>B}$di{M%CmwuT3($d<>%1VCI%P85G{&p^Oj5)C$;+HeZbR6Lu zdKLZ(^XU|cayPTe+VZ2RFA*;g-&(iVXy2#Z|F8bfKQ+HT-QEA8dH&N<_ve1+-+kKw_I}W736K%{Wv?XafdX8}Y#;!rZUSRS)6gWRz?DiboRvOh#J~YF z&}#K1Wg>V+x}8-U)8FHUItr^7ap^@a@!SxMEhKEPjo(V_+bq$U71Rj@IghO%`c z7WiLjI?6djas^e5geISJ>GoN)q)`m6KR*4jkFk5HRg(RGu#UI-aryp*WE4AWqueRrm`}c*pgeo$@_6R=k zUNRr6re~U0=+Mo*h^Q^)z5ic?xc-+GO#_Jq{aC8tY|TB};SYOGkNv$_xVKP{SEicw zEDZyA>_g2D?i%fpW^!21m0plG@VUVJsr^dU$-XKKm`N-x9;xKZiHX;RUIbziXjcbbnX*B?4|Rz({^@U+Dc= zyA}G_W>{gPRpbC+-$Av^wUxECrN6vLdx*^dUcMQ&UoY`{Aw_Ijp_+x*AU%G%2I((2C6BlxX_uZZ_cJv&Owf0EZq)Hs%VSsau27Apd) z$E+h9RbVggyC|)b_Xyn~oUudR<1xAG&2#f4UpAdDODs|6RjcLaj&AjMch8m2zf6IlUOC5q+hZc%BqCDA1CN&K)!at(3M zKR;_0-Y+dE91KT-34k%)L-Jn$z8UJKN+&+1qzhutFNbt(L#mPe6i?qfnN~U2Xp{H+ zVKZ|u9f0Ge8$4RukVm;95ykHaZFN(Tr~vyiF~4Cn}^T~#9OEy80ug6t%%NH)uGAJ~{ zFTXIA5HG0jCh*&0Xuk=7zmOdBJ#t+@27e_t*dpklRMA}HAM<*J%4+LlW9loJ>0O6ssvok8;&T?^xI5lL1FcUf$Xf9V|UT)g~Ju_t725+`hp3r~ur8ZyBwlEEZhB*_{f(Z|r!Cyf?{jAaIN zN8;>v52FN{3<0RXgCzo}3iZOEoYRpCUr|*Rh%f#=N)?5$T}+X1;F4g>8-l)u7oK)5 zT4t9l#@{)m3IR)L40U16a-4!xQ0(1|3PZQ8$O0>paiVg(Zj0eO;s}w;STFnAca;<$ zOTmc)l(!NnNY}KW97RWn5ag*_`dT{fkU!6QrP)jED$vr$3fk7RIcGWdKYL!ioR0}? z-`%s_;2ZSp`PPpv3h?V(i5m0yu7RhpFq#%O+~o1Xt*>CuKDzsr0zE|;`q8GP6G=wW zob$fVbqwvzhXlGoQ%6BxBEUNM59pU;8D$pi zhE(I>qC`D9RfCdL+`!->%`W_5N|pyGJUlX^!tGG%#H3Eoj=LRNp|D?JFEklGY+#Uv zGLo5rkl~oD46u-?ULCP-sL<;H@opYDr=zSeF|By%8G`prEtJ)Wnw8ag3hE{>J%)hC zsKOb5PV2?e!MuO95!e@cGs8Lp(MxvUZ z2UzGr@boirVxs;-2QsxfEF_iilA^k?!R5L{&KCK}!Zh9NYawCuIw@5?LvllgG^h|6{gH{IAg*X@ z8oofkJ6>|%YQVnY!yE`$kwhy+bQ?LGyx<8+Kr9-agCJ~by>?iLi^&8v5w>0kj~Jso zYCV)jLSaRkq5!HQSxgx53!f8hialevG}aexk-LK9;4`nO`-}5{FP>6P;;G=LUy9eG zyOwV1wQDxN`?Ad6o08==G1sk$bsnM*6axV_sIq@=-bWmK?xdIc3pI0 zHt^kotd1lli2+|-*nf|ji}5>X2|up!_V~pAru|*Eg6Bt8i$^x_ZdRk_oYC>s)%Y}G zMkCytHdsAhYp~pMBEFjQFh4M@S?;H=nQgs@!98)@s{#b&{&?0cMDY0Uy~)iSuk@qi znt9}hn*Yt1sTU{t;C@d>M@J_(HE-Q-1S^5irEG}uQ7 z+b5?iu6r+j@v$S`0TBOqESZp$qH}EK75islO9&s0V9&>T)`Xuv_5eQrk$(OW|GSao z7Fqg#2!?!qn1B8ep{JwQw17=?lusXvwI4~F|M4WG5)to!9HX;;+!a5#Ij7uZ1bo*% zDZTvTSk31h?enAAe>btHm5M^ zRk#gyl56uT1UFu)b5M__V$1nZRYh&zW=kc*+_$5+W#JAk9gi*sy+YJ|FU1h5=1UpT zE$4$`KdbO?ePO%ThRdGqG7T0*0&q7F1eRuzgZBEo9NkfUfxayI#jG((upZGhJ%*9y zjYUi2^(7gWiWFFG7PV2&n2oU@oqa1MtyYZtu{j>@is5@K4&H({1dW6T^Fwxg4%yS- z_R1akf3?-s)4C{@;QA=lG|Y@WWB2{dV~ z;d=fG*^uj_<@!%Bsy!_@GRy$0Jvh+phL~MRUXH?uXmbJ85>+5Y%2c<10FR-c4lCXf zC*lE23c@!8%E%XpOF3ksAmIhUW!r?LrzxmGa%cBnLnf-Jd>_h{rDj1=wcC*}}MhDa;NXDw}j@c~{#m5+Cj1-wcV z_^j6${+RU87p#m_-4zR@QC5IZ_q0jg zGfyXAh{Rq*t@M&dlWK4rmDDB*WYdeE8O-YHCsSr7)i`qn;2)A#mjYSJ>jKAk-KESZ z|3#piA7G zfvdP)hCPlT<_l$~tv#88r}RAr0Ho6beajALVIxs@Ft~`~mj20~b3NEbQDEg#MXf&| zvIbT)hcsdCVZv#T--zZX3DqNp3ihi^cp>auA}Y}J(e#bSny zn$4BzM~=@Ej9Hv^aDpg^4_iSv3Z6t57K0;>`lOMdTHF{>iFczD)sSMfq4l;GkO@Fv z7Uf}e2!1EZlCtbMCMl9)XB4Yt;JKQK%wN)k_8rrjg#7J!b8q2qRMEJX8hwPaw!Z?+ zABHan@YsbPkc;)l*MEEdn2sQ~M?;-ILFQI{Y!!p!{{YV&cZrDkkv&tw+KH2axLh;~ zYmZ1B!Na7g6ColdU#Rcy4VFy2505|oz{au+d>idxyKWH;gkwuVpk@`V1YM!GhYq(w zqwYlX!?hOK$oJlG*5WB79cUuE^%HS5aOb3WPX=vRC}Rxc2muv{r{*Q)CP-ii7attI zO?-cWGH?Y2@_3y|FLmVfkc*aIelbyiDuvbXHWiH|*n43kdwbEMI&bb5 z<(yQc>>}mFM$)MJ3|-WnzZ@URGEq*;A(}K=(ngNjzk3q@(u@-V%dHwyvb+X#c|;Vl zEW5Jh76=321A9&~XQKE)qDa9r6_a%~WK4L1BO50ZGj;n$5m90?Id|7r2OtC-h1ZFK zk7C;72QCBxtto}k{uy?-oj7TjXtiz?xSlMOkSp z&|d^PU5`Mk3dQ1bnOv`PLz{ho5m3ImEh4?CnMSz3)|bQ=(@Mb37YJ{{Z4kVV3nH6} z)7nAC_Hy0$HA@SnKE=yxOu5CLqg6N9p(U?tt(4ilF(wko*R;)g__$+L1z+hpAfQ>C zN*(E`8^lWXA1M4n;S#`x4-_+{85~gQ(4-^iL$H<$8NVf$ZxSE)(uFk8JhB}>B8A%9 z#ytlw*eGoO<**65)x)|g$#tM29IeMN0x4hM7HBn&X>>&U_RnpBpx9=}%APgOWvFqPn| z_lG-=y<+KQ^F5SDM!ln?ON5?qJZ^z$B+jZTOZZ!b?Jp>BIpyht@%XLgcs6KW49WBS zhH5U<{=I5?0}hj84U>0v9yhBy8;_c=S=HlZ?|A9M&!d0=))|~zg0Fz?cRIQ`v@C=l z6%U}y%RdG7vmrn=@td&wf2EoZ)||&REiC^@v0fq=DNbtAa$`hnM`y+mF-2P%f)yUQ}lAyv@ zR69Y^2$lIk3MT`6b$)%pi)2%gI*u3Ma4K&wl(Wo2Zi1lUay1Tq#Oic2Mnt$D1&6*l z7n1AF)yK%KgKbTWGQl3<0^`~PiXt0zLSvY{nBZ}BO1ZUC+%@pA`f+_b{pbar!}2Bh z`bh@xiV*}zneOI5w;AHswj<75HN;86F?N}jZ>GU4VTl*WC70|YfjK6CXp>xPZkZLh zG>0u;8EJN?Cc_Nm0pkL0*yQ-$<14%Is(d#%7$M$Ne!cM9!V?}0w4r!XI?Mytd2Nl6 zB&AgQFmr2(4-gsOn%@QhXsPh_kfN6A0i2SA>ahuNV{Pmev1_9^qJk-BDyyS72ZM3M zl>`QmWkR39g(V+UAx9zUOMC;bOG9m@)0{iQ^M-DMl?cd3xR|pafr7}S9LK0PrIYRxh*`-Z&nWmXSk{U7i351&Q}nocDe84>Y=T*i3dQp z)+mhB9z}${m}+O@0iAd8zy=T8Dj>~pie(NRVmqP*R6(c4mmkTpZ3fWT-^e~pWzF%) zBIF9P7Obajg&9U0pEqFwD&{S3ICCm}VvjSV|`90-NunzIujCR~~MJ%=oe!+T}>!xyQ-CbSH@2CaaQ*vch- zpbn=Iwxc7+0z5dgn?eGLV5{<2!}(#BVfmt{Q=;4?+@zPC>c+#wCIfj=41@q6;0NNC zpy81Ps_>$ci$?raa|E^@{jnH?u*`+GAyfvZ+O(`Ux24}xcbp+oFlU}3+8Hf}bF z$PrJ5vT+HdySZ$f?#GM@%1VVg7$4nOPSk~QX{Eg>a^-{)6d_kC+Z8dr#2GY!U(8qA zbM^EtUkpM{&USk8v4!LAnNI!ur~S_U)1RKV>cqYLgr4b(wZSP|^C@kq+fC`sU8Lt; zgug_*6iArdMnEyHY+BE6BE9q66w#tAM}*yTZUZ%y)Hct5NEETrPgY(j0<{&7eAjc1 ztB+?qS1a-O(q}y1&)5ReHJTald1#g&43KReHAC2jTm$xj}SWaU?kD3*jws$w*aihwvN$Oxk=YFMKG zY8ACseEy^&Q`iMe^i2deld0er6PG$;D5IZZX=b9$q;ZiTStu1|K8q(S*Bjh;CVPci zUeK>e;{&8O(rtDc+x=mjl6>W=e?iI3l6vDRV(9S$3BHl=IzEZ*b|%+pnu&|!04qce z&+iy6ytx7@i+in@13|_(va8)?K`a$b0s_dJ&LITI78g+S>IS+H$UNH>r?aq+K7I#lhV@?Q%!_ZYk*Qv|%Y0 zjh=&8Y%~9kwa3|%vn2PQ!xtquYp)fanbj%d7KefEf@}?ETfXm(g6!Ig7mb@ZAr#7L zg9;hU!A_humbVcIW3+kSFlL&$ru9f68Zsh?HbqR?XCqu=e)mvzM@O^An9UV{xI%I< z<~X<$iaB7{_mg}pAFk0B<0Mx0$nL;>qXEXh+!Xt$K7!BylA%y?Ac?!_2b z{~FsJ;)#~k+JuBVkN^{f3nTLIn*388++!^2(!^7pJ+~mZ^x7U#g)*pzAISx@#bP%V zeiRLpSOrsmMG)0{SVClngHv;%P)0$tBwe={z+0mgr0!_jEC>%wFn3p1zp3LYH{+z%WsMZ17L}hO0GzJsv!bC*?Nn0YU%Y; zQT9|(L|z30#3z!*7F7^VH@}3!`gU_E78c+em z;sWame7C^yWw1|B7ZH~HcEJTq2Q_euGs*x;K`Z*ojn5vijH%$a`Wr!3fv005b)_vv zZ4~@70>8k%1Zw+N;gHfI-g{9HJBzD;kbu~Z-$3&Eu!3`05S$9Am0h2iRLj!uo0NZT$nQ% zC$-1RYf7t=L?PDSx^-0)ML^K#UDpZ-oRl<`z#!_i^%r6;WV&A^y2t8X) zM1x>w)vM)eT!C5Z+3kRs z7Dg8s3K9;OxYl-t7X!dV&6m#hzkP`N+y|5ikr|w!UJhu_WhBESYgs9@rBXDsZOFV? zUQ70Atz<%EV&|{VEee?0dT=a@B1=6IQLkdbkV13@)Yl4ED>Tg*6Bb7ZpUd~Pw!)(A zR#pU=2;&@X1u^2jE3pJ=8@vt+6?>KZPG5JcdnV>+%L||L3S%wnVtOupzsTZrTRqYe zT?bEyCHrpS24)+y$r7qUzar&oDq_b4LWK*&PUr^v%$E}mAX-(N2lpb!i1M>vi^=Dl z5p(hKV4bMV+_l8pq}9r}d`g;;LS+FuMSg;z&6N23h>?C#h4k3WvH0t}!!8QPuH{uK zo`vv{iwjg%q>};|w+3KN%IJgF!=)Hi7sRdQHF~31+*l>*X0d3Wb;Wl~p)94CN+ji; zaW+Vn9-T$Pp{eeVQ(Bj?1VxWP3PxD(eYixHgvEi4LT~UiAf7HsIvgIEoTA?&*&@d% z$~nu;Gwh(cPaY z=MdYXov}_4As&xC#hgRac!>-qm{ov_aTaBQP%l)5Z3DqGia;71SC%d=729id+pGZw zz`~laO5_Np0YffD|Cx%et+P5+wWZ3eMUl2amf7ht!~mZJ{uQy}oDf)u5RSWbvpjFz z=jcilOdvmfztX$LN~%f~6^WDsk0r;qUU(jwzg!S1Cw`ggL5qi~jVOb9Dn|0<7eB`@ z>sMVEmagcn%w5`5S!VrY|C@}k&>$Qui?z)};K0%?Rm?Q>Qi zF7$ekqfYCDE?v-)iFwd?^ZH_ZYD6UvRwj}Q>@fF6Bmq_QORhEWOD946v&I0dlF zq6bVWj=14jZs+p-1LpUm)9X_PJsE{K0ru%zwTJ*q;)vanQV*q;Tn~yAm=pbB5~F4- zsG;D#(^&|WtPGIa$WI_A{?mV5TN5MTZ+(Rl#;MC$wob)4BaM4KI8@MsVj}e#>vNlr z*2GW~at;7)*XPu`ZeWuH97z?RnL;>Y$d`sUJ~U1HnPv%*={kij&a#962-Djk5-7}; zEM%X93B)46B)OA-hPr0)^Hxq-oCMx?ivkxKH7Fm>qR09otTKUi2ZwL3FEL9ij0HrY zb$!44eDA)L2%8(aPfme_n+ah=JmSn|4+eNjEV#TmJs6)zBhn*gMLTudp;vh{`^R?+ zpE+pLXKlXu<~xNRNv($F-|FDTg#Xf(>gLjSWM2abPcwnRkIuj)5)$z9jZ`ZCWXR7T zj4=h%u1w`|qhZ*zi!k(OP%<~|>iKR2rdSe7F*;631{I5;#jDML0OLud zA;#TT=V9+j)BV-+Y;^>Ax4KXI#=;e8P%fL2ik*g45hZA{IcOtasOU<5*JlBj%@mIR zoP)+t!9ymXaIR+OKUH7(6qA7sO9wlvP`0ZHU%h`0vGWF0RaIEGLHk4T5#g@9t?vh1 zK4NRlY&JYrgmG)Hw!~D=JcS#Abo79E__W{iPI_EtX>fRB#&pYLyhZic*&xk@6*;pA zIqyd5L7bG|d4^3jWa=My0|w7n!HOR!i}4x|B^1-};PIT+5;AVS{Nj({1;H8N%Cw_^F%gk3F>V$h;#(}i3?IKn)Su|dMNt2h@59LFvJ~)g^E0@fQKl7por(^D$a5$X<>~3fgjY=tm zwuZkUTigLDKiVLX2~Z0}^V89L&Jr9l66=AYA-FWFebWnChZFkWYAMz}pa1<2I3Q&) zggVQ?`eIZ`+hGIovP5^%2i830fVAe%e?r3kbV~5&NKyhboqa&a7D?z2;%9YecFjKa z)E1lwN8`$@`;#(v{PDudVOOlQ%lj8pc!9)21a!Z|$S{mWp}}~Dj;zd+pAX6I`TX~$ zQ~Y$@|1rxA(T2tr_TY z$-O#ZyLN5;wsqrIm@p7V5Xz&xRq))~*=LCQ&d-M-al}@9)F+&HOccF{sUE=1Nb?h6 z38sj&Z50Shj%z2cysOZxxk!KK&$pkpuP?y}p_{8F)T1kyxl7>6^)vHL+e8?*;1RK` z9y1)B^F#kBz8}msuKTb&*t=V~rfIsgXY4h& zT#Nx4L$~Negbp+Q`E!e%rhA_!7kQL}v}fns?%(gYVRlmQ(;{O<=G69i&PkU(G^rH0 zKy-qg3%UZ^3Sf!gP*844+_e*GEpEQ`{M+rQCMwIe?fP0gzFKxmEg=@vKpnAg8TYOV zp!rFD+=>4i$JmVvntk5?X2;$Ax4WJ8pKjl%B+5B=s^a{4s~|RH%v({-kKBLKR^yUp zZ}m^U)jv(&YNWwG!CH(4*e#+do|KLt9HZ(J@NOAP@?YIfApJ-m{>l1Gs-LMHaf=#2 zoWC>FgU-z6p9Hy@#~{r#%rH11^}0+Ee0_EP&BKTMcX@OLWk zDX!Dv^#tLNn{{?%7~gb$Y$z6e_2$CroblNU*xZnLD~Z zg&bh>D1VEn|DgFRXe)ePTVCP3Rm8220ICmz9e?(*Wu*}fjhf6#ZRP- zPzqj#Y{RInIglkRY`t3ida-)xzaP2$^-oy7fga54K5te)*wv-2)vfjQrOmV@W8;B+ z;}K~^7S}~{&aO{FMoVXTt#M)U&YZOKX6!)XkB7gUoKR7;025z6fG)pQH2hzeA2dJX zg@e^0;JJVKFL?GZbWqEUzZ=;>50e4$R~Hz@Utl{w6rGZ>U|+OM86upeH1Pc~9jOAP zS&sP7#seMu_7<^(=j3|I(kuS3y`6wI(Q+}#30;W*7ne)Vh5?U@F9PQ-cH0!cK4|u} zUy5>!VD}J4U+=T!22?c*dE6?vCJ~jzU(&c*wOb?86W%WV^7GUC-*}rS=z z>-F{N$>ZZ=&3m@=?y{I7WZYE^fVYiai^#Yex-m=QJ_IGr|MukSDy`>R$)KimL+h zPeWnf=CxzYMV3b1a)r4hRE)oB*0&!tKfS#gAE3i>SgQ(#2@X_Rt|P}C5@C4ObaaKz z-VX7D{LHX zvjO(b_WND|-A z&jb50;fuQ`Xu;ur91pGtC;?l!F?Z0pH0%d;uK3Wrgc4nDZSB!cK%1zV>Z^|dfL(P1 z0RsSn{k}41ewh z7v}mfT*5GBP6{f+V0b5APb`{$?_ReQWh(XZQSJ^uJ6nPNPp-q zfPA!0sI*>)bf{V5t4~8P2KYv%BW|8qMnZ@WLi3B?x1|3`;&q<>%g@j5CFB#K`T?K0 zWQQ)iaBx9XCJg^UQ;FojZU%C4uNJ^-?o7sc-)Yz? z$`CHN9KKQ$F37%HDqjGz@$1*f`$#2;ge|)T;4T=(%yR(AADS0PP};5aNa&%ud?=yw z2_`iS^^G*dU`&%`ELkLz8`{M;AcqtP!tqC9^k|kX+t=N#EOTn8Kw*6~nPLHhK7B4Y zXso4GK(X_v07bUW-U172dvYP~jRam4wb6yrwy;Sc_h9a+%vU}2NI@`@F(}^HY&Wc7 z_d(+*f{BSD0O12>Y+$#&on!@Ir%wg!#LGVgjWnF_XiFy9+WLoXt}08YC^D{EvZml< zo&;LpSj+ND{S?h#u~9VXQ3NCYXdSqcvH`iGX^yO*j&^&NpYe}4TGjQ4`5eU9&Ky{>YohXeD(Rj8$uHqQ-*KSZ9ZW5{()Rhc>+g+hx?;b z5_Sj`#Sdd;g08JUPtW3RWBRl);H+fEMTw@@pYEp19X~YO?8A#sCS}ud;jI*g` z@of5t7r`SCESqzj&AZ`&v^YPSU;ABQ&gVN6yF!NZ*y?(0yHtH^M?^9K?OLe^3t%x~ zkxfDg!BH>5L45heJTRK(H2JXPEx5Q1ZW(9yLtWpC;qMR6R#w(_mfu{_vA)pzKJ+kb zOy0CPR5t0;#zXu}6nOqzETuRVkdZc1O0?WnVbUK6o`PIRdJ)y=oFVnR16Vi4sdewk zVh3bK4Q#A)%A!Mav2-we0432nBtGDIEh@T`Ywd7w0KPqlMTltYj!77K`^ zLI-d+7gc2&{!7XR*h?YZGA<_Ps@cPoyF|5?9Atq&Z~lwR;GTo^q&HxEitx7KNn`Z( zXn8;^yB;G+1QGTni^^7bb*^De+QrE^CdZ*HamBC*8d};jQ;amR0>UnA!EqS`_Kj6mY&19|rXQp0{bqM`>^(fdJFrIy4=R`-XCQZ{ ztap!fz#~&B#?_1NaFF^l*5-OKh;y1_v1k=JSjft+1A2x>um*je5h1eoHPPta~^=i zms&Feu*u!e2NFWnS5`JwiB?}(+3ft*{M`q#uebc`N3E}WEB5s6R{iwuwtc;~MU&KW zl)do%UiJ9uijA_mzFL31zM)Z8*ZcL?zQWaw?RxOljgCG3sP*`^58hj^N9pa_(|arR z*Lyza-ezm?JsW&&^-=Zs+NKS@*4eJV?shf!S}$*ikL%l86*Shj4I1m)8K(ABuLS)1 zc3(rSck)m=kM+*FeckKV>s;Tp1+5ncqNjH|aZLM^X0T8F^0)phs;$^rts_^$dXE%>t2mdUJE9e0kF^$L4l@3pRZPoBe#g znrd^;wzs*rRzqWJ&F0!#uOHuXbl=)&ef`Lv?$!vo)ipfb>bCG=%P;B{_)@NX%a^#d zn~#?jZtZSX5A`<%_^o~c1Z`wz#lGKJ&oI-s>$~O4JDY~&J6rV`?`-?K+pSaE+1}G! zJ3IBY#3g87x9-_a$9B8Z%}?v)E;>a)b#}U2_Hs9`)0V&6v#)n|E0FB$`U?831^cP( z?5))x0vz*;%!Hbk=Gd?yT89I_oQ~!Po8fb=LFo*eL4;yv}-! zZ=H1mb*DHFdwO$6`|Omr&pxMnZ%;k)sUFpruH)y_+0M5}?{4o_5bpS?b#`i)b^O#i ztyAmxsdb7|vlVoG1>N-)5W2qG;`p?LZbCDC>aUbY*Y!i`LY}I1cKuYktyAgxsdQVX z()Cm6aw^r}ekffIr5=3U(5zd4pOtm%>(Jdar0Z_hNY~vobm?+S>MvpafW)j=jYUGv5}s$ksilXL8G^7`|GvNspsd^Yu%Eb-;!Q&9yaHek>6e$ z)IA4v4~kTtcF#fGYk|7wpzgI$yXUCgYvXXwaky9PU)$fUu3)e4D{E1nT?f)`8<}?< zO1mv&-gO}Dwt%$jK-z6VW7nav+lEHp8GL^&^9vT-U$3rN-)};{1^E7k?W4cZlA-$B z#ZhyZ3zc>DBl!YjPcV|!di+$ z=Og7Di=lR^?R2)wHFQX)N($KOsu5QHsmSx{su5QHDU>0trkezQTTEA6XLYq#pF%&) zpda@VFzr))ynR#AR{J~E1z7DnzV&OdnV}2~_f~5(+*=jXSlwIQt#EvGZ_OWvV3b?l z+wjLXDqgv|XF3=ERQR~JnHQ`%Hp{*3ZI*Y99}NZe9<>HGO>uRvTVFxG5M$?`jN0}w zot;2T^oBiDD=mEO@3yp))jhwFdxhX_qwM#3*H?{$ zt*@@t;#j9=XtMhVN-QgJY1p}&{iLwPL=(!^JSLRlz1IAnW<}ZX>$0=g$rvrTtdvY; zeZ>?${(*+KroY?viAr-Dp_{N|j(WBIXdVDmZ?@L(X&_i(x8TCcdVQ%^)?MIOFUT(l zwX(5oiqS?}h2Ak(R=~C%tAy3)YZg<)2=8LU|yL$!VuWa^wOo(9`Q&06O zIb>_Y%nDnZ8S!~&>ro<-ns3XlEC1Ap%@-yYZ0%OW8V%Y&vemC{4PTqcv(?Wcj}6{; zRlZ+SOVn;#|4zZCS;|gksd{;5-Q@6{jgka#U2DB<#|<+(oArWrwoK06*~+}j#&r2@ zXFGqbnYz6aj62;#7VXCh^gAx(?9@bZWydW$J6LwoP&T2Pc6NHzaqjGvw`gU@EmS-G zYE_+;Yy)5&oeUlORIahJUahg?0#|1vAH3Gm*|0e}ITYAJ4|<6!FaDw&V34O2S>lr%`!(WDESB%tc+3<-P6@ua(vp=V$F zZoBNZygFUqY?sZZG41h9F`Sid!IJIkZNsB(v0MAvx7RJC5&PO$K(AgKn`u z%~fMT&v)A^E``SI`SyAnHHTd3ZPl}A|pS!Jn?)pA=%X0); zyPN4|@u|PQTJE;**7$zQJlS_l>2I`Ep!(&mYiKQc-FJH3FC;7Nz27tALLZP+dw2P9 zWp6cK0G{4kFNAeIm3!D*PqMl_nS~qsSY7-*x2W#njw<)PM*xN%E-YYLxP|@9=nu6dixM(|Wq=Pj}mccb&s`TaWkr@m_22o)6w@J-zEs@3sf; zyAP_rnQw(w-gnb}f4BbHb*+`XPZl~uNDMAhmp6Cj!HW3O=R*v3y3$Q|j!$d5Nkp?x z>9+7`eY<>et28vM^f$71jz{~wq%G+Jtt1))Y}Z%JEyO=rytl7+B=&Ln3R-C2?@9ee zEy6_5+Ig$@QR?mE)eV!&KXyz=aSVhW5Od&lv2f&*8BBqD3Y>aDz8Ky>!etH{uy!p6 zIJ%1}KA{_6S!Y*KC!wNWnm%;twBEW_U?i4>y&4{mEmrUuSw7E*k$*$O_Pss6=6{eq`DSUJVVuE>wXjKP%NmA*}}v=r)+JX(Q~221W^wNI;Qn{9pmsoFR#$=bxy>IT>1-kd2{A)o-eqES#Dn5jeq*(`YZz&A7 zJaxFdh58P9QQ-N)(Nxhy5q|{kSvs9FzMdt`@Er+=1Y|ag>cH{b6O5?An8MRU0{|^y z(nJFRU6OgJyG99G>xiMtsMkZuSTzyyvhaHCD5l(o;=!@sv{lv71y^*KO&*`Lyj=#5 zt#9;}A8{UkyRj;p}7OgDh z94SR)p3)j$*}qf~YIk<4)7x{{U)k?mDIV5X-l4opis}4q)@L5VKJ|&LXtU~izt6#P zR_i=8$R(@vgt`?ZiaFH;RF@#NrIxZ`7Dm+NlHqP5X-lashuUl$j*nG17r-gT)3%|= zc>?+<%j{L^oiAjNF{%Q~+3b{h=mg}7{iue9@T4EATifVm+W&z`V&S5kKM5VD)pjKW zCmKbJE%_`9!sGltwkYwPn*j$9(+wCNa{i(5hnCX|4xn#bP|6K*3j(_{Vr{bfxKabIYF7sc)PFQ!hYe3He~Esk15=TEpN zUQgKW0|x}Rw|P!aT@Um_r`gG86NSJ?*+h;5&8QH-sz@%SFkY_^{H9a|5^-&EAM9~M8;>^>)i4%eZ+wWT*oaI{ZN5O^ z(;b?kTyqKSa5wy*(^VGz$t(mDCd-VMn1(r){B#8hgropnRfrBs5jQcilt91UzfVL< zdxeF#u%UNChXCjI;+AeMa268Y|C%$1BP>(x&g;HTX}!hh(sQ9ONdJhmXf~Ro+S9B> z{_#hgLObdC42?ym5YwY!$R}<#4A7Zbn0b>(=oY)HCWF|N!`W>jK#zia1C$A z`?xff@~27*s}gP(r86U%6q3gj9XA4nJmZPO3~Q+oEVC)YO}#Z`8;GrwZ!b#GgRF&L zO2oS_v~adc{^foft->2MF!M^p>(tSnQg@QNp~CtEpN_BvQ_-KB#W0R;`+U$X*JYK& zf~*P4HN7D2#mPxL6YG|6MswSPA@XFzNlr>7OC8K6%P+#p3fdM2o2CfIQ=`Pi z#Hs0vS|XmlNW+*6!w9#M5^?3cVUoC*DHZHG+L2tI5zP!M%_0qFsa0GKPhQh|&)P#v zB3abQT%UOZ6_S(&C--9=Wv?a2drqsdnFL~803_0COTd&O6R-DOlzO%Mk^()tBuP-J zYM2eZ;VhOBS`hCa9SX-~NDyS~lsk5&JG97Pyc};~-`$6XG`A^Q^+BT^(N@ zCw4&XwqxXORTw`q$g2sRxXdMceA({Db#tnPZYI8E&Pz?yLsVGV>A51YS( zz6s70gw4H&TGI+rhjf*>fNwYyWTE0inpjLF!L33%-2t-}2Q^;B*w5}=HP-o%wF`=E z5Xur4K=-&ALa&jxd`XA#Sivd(h7xX$Cs}McJRF{@wTZ}+3<^IYjcGtNtvb2+hE-*vBBtUevKL>hT07J^mYNMY>&2iVUgC)W4_QSe`{S z9)m6^bGH#QR-Lz^d#FZQ$(0fCJob%3xI(V9;%(;SU>iKH{!jzTc5qjt0)Odc74asN zg+9FbR2~HSg5YR9ErajGi@?W40uC!#=U2vR=bNKPzOE$B7mKAe1nIr1DVK$4I2G2N zK+r{z&5mui$?tG=Y@#LX+31r=D1P83I#E!Sk)d8u>P_g=N#_a~lY!qC3@&jk>>Si& zAb6j=oBFL35~Sw}qadKXFr#Ci7%>-$-?A-V+@++nPGKJ#otav#P({U7OY~7S%2!h# zoo*pcD($THaL1Gq6|Hf$z1MVDy%y=d6>z#2U4HP{(C&%c?HT5IQ3lcGSXn1XX=b2n zUSFTZg8VWci#P#{Gc4zZ1~QQwHoFWc8@_e;5k>X`%DP1w-_Bo`7ms7t&3z0!!5%mq zQFK;U&q)U^yo(e#G+BW*g!oX-u1>IaT4%0tW=*cH14~Q3KvwfEw4P2AaodFC z!cf=J5ufoHD_qkSt&k2*auLABeHbrU=1lLi?LbX>GnDP+Ht3~ zNVR~i2>uKqpC|tm=%$bKcvY2FL4@iN9rF2wgiaCcOK-o&My*h-*&s);`lgNDCA4=?vs3mV7=OoJyZUn3rk>@O_5Bu_)s#bK~{s5&;G2X;u$ajp+A9 z_#$f0C&4ewm}J2yBLD4DDumW0119A8mU1-GEO z9h^Vu20ykPXoiWm0gZc<*B6(3++-$F#bympb+b4=VDMa83t+So}U-TAhAbioQ@V6&%BQG zJY@~Tb(=wf*C1*$R(bn?tQ2_-rj4j@X2xs?Xnwt5TNTkXf+cgGJmE$qg`sbTC+8a9 zL~c{71TXbNe1M4&JHS2AF{$73a_fKVy|9@D1M|H<>#3+lr-%09Lz!rFZ?1sYPj>HB z`Uy&rJG=Vnl=sSLXHfl#C+Y8{AzgOme4VO&ON-(sUw!rML;VZQ2`yyU818;blUJDXar5{0nr^r0M)DB{xeZHIAvZpk zJmGd>`8IjPBX|3!QUT9`Tv@6AO+-1ay4>Byf-Dn>o)}RONA!rp#O5SVK+(u!K0q83 zmROHK*MU1PdrzNcp%815IJv84{+l^YkP+b_2rz_ph@c|WX7q8Qb06PGb*jyx?ma~5&4Z|8wo{K(M7Z;>=rl&h%2wmeT{`lnkMJwAF)*N+GqqqwrxQ`{o7s7^u1>p z0eM*s#w}R!V`wD#B~Fr7R$BM&Zdb?$vC%$toVcXVGtT-QZ`CTrScG~7Qc$0gqK;c6 zXXseg5@P042vS+Ix>aMQP^}oqP{nYhjjUm)No;KvCD2Das`T8MXrxFMoJt=_i5>i8 zSxsO>d1H|nsvB#R_?bbjzB-=||Jv2++mY6*LMSh+~^PZL2r4o|r@Gfh#A7mV^0% z?pO+uY2Q6Jp~tbRE9QzbA}LHlLqM+?X9Rz}iHCYOGD&%aEsS4}njI>%&MW4SaJBoi zXQ9f55cpGmA76OGb@S~_TRTn?G{>}T>FI>=L<^T#=lMYGtf z_vSZz&QIuD2Z~%Y`@vJZFm8Vt9MJ7PoW{}EjKE<(hGwn~XACc4C51SN3=Xdd0VqbW zsBSE;FRx?&QFvMySO&Rp$>a0ou)qncdob_%t5o8if~V&6vV+!=k3FE7>4 zg~iD5CPbRHQ8QX409KEgZ0HB%?Xu~hWm-Om9Knd?9rEH9u%K&)h;(HE$s7Tp!4S)) zvyo8Y#f*fue|=04fxLCWQ0!JDNP&*Lq>%`lUw&ah6}H3#7If0^bV?S$tn8$dDg4MQ zU=&6^M;Doee;*x*Jf99vX$)jm8#8LX9-NL4iCKJ{2GQ1>QY_jih?}cCd&$^QKoMtD zHj=(}2bhg%Y{@PjV-$=YxY&Fgsh&>@gWsJ$YSqz5zB%nc92H7TNNnMd8KP28r&HS$xOb_R(~#~?2x%X9T^8XXR~xnhejVres=K4$FbO-XBD zlX?ihgdmwhkc;FjBT0-U7|zB@YbURr(%(pYkUxzf$`DPhZ!{z$)xzfSYpC7iLhuf9 ziPjE_q-94NBqaY7Qf{NYR8>W5m(MZ=R0&`Yf(%IlGn$V7LeL z(rm8O3o^=RlmxPTaWWoAalF11hvX_4RxFuxBhxNXGbR$e#sn`!GE(VMfe~hT7Lpb2 zC!`jKX)>hNXKpAw0VQb;oTOPoWp-7UF4z>gQx_wA&%_p5ygDzVs@(AO?;)lD4gk@R zdwO(+6&AxAc0+TF1}Cn_n_Wg;lkX6yruB&pa-Cu)uINciF0*`^K*h#S`+PRSNrDh6 z_Y8qU2?_kbT*jJ-RYu4~W8Ya%J$L@99`-xV(vXNAWA0FLCUTGHhXgaZgPp586_uGl zHht^FCq$e{74VBGI#{v+mArM-Y1B++0|X(hjOqc7hb#-p3%KESH4)n8zym;&okJs^ znlm{8buALAmu>*NtPEn{@?-D~)|{mfH~8~*kB4?Y;q#!p2K^lsWuQzua7)X3VN496 zntjjb|Jdg(Bn(R`N}O8=g-w#J%Pq;J*4WuOBEHvljU*11-9I8?wlWuSX-Y}_>*dw( zl(&JNeM_5YLy&mi@t*yv2~R_ZU7k{BM+mH;rv+|!W)Bo0dCUwKQDq!PDlf;LT;l>*b`vQmU$~}`(iGb$;S1BiqSG%Cd!|ZCFSDR z=!ngdKo}+poxh_Wu`sFVLRnwjfG+L(ECqU^APGAz-DIvwe9tC#8ly#BNXe3dl@Q}E zh*#b5<=`#NDzDF)#cxL{jXy$N9X+|PlSzrfMdwC)%p2dr7wKI&Kn>LM#pAg_#6FkD zdpg((pX2&Gq>m}xnwU(d*2D8`9U>+M`O6TLeK8oHj2<+p^TQnP^=RUT_y?0lWItZ| z@$mn-CPMi^^UJa*_|p>r_kY=Y*Y2o}Y+ZAI_89&T>2#MRT$Y5u25dks48o<_fH@%B zwU1BjJ%p5i3f)QyGuf9A0dsZi#(JI6#PMfc|CO@RBu ztrvK}W!pcx`}9upBS+qGn1^l$@$&Oi5XFBx+al57Th6>4J-q|{&wkw}n}e>b)Mki4d> z8J6@y)}x+_BzmZAs}MOJY9X`qWT!w4O|) ze-BfJ_vUv%^9iIH=?y338_go`!*jQ>hT1&PsUMFflN~Gr(N?<&0-un_OcE>y;2v(v z1|uQABN-xYmvhCqz=!&vR<^MOUdcmRIpYD|EjBQsmxn&O`=t5l3Go_kV4%>d&f1z3NhsiD(-8x4I3En(V{PY80i#{vlo^Ee7Ke^q(!RSXC6F{vg z$~RZu$HM*u7`d~%ck>W*QO{PrLzWIYfHUGL1brcs!t1q;xP?>?lKpYL#($FpnOSIX&)STm~X6JOKL24Hbs_9&wt@Rp#GK_Po5 z&7%sqo71TB@FX_T3MsQURxV!?J{AaC`uWQ0^3LhbSjt~tkY%<>1c-`@#c=B=)$<*n ziD;C6Lyz{kSfv2kJ66vI)s;j*5Sk?8pEn;qDCM8nV`@AEOplb|jGu$2;j-BE7L6zx zQb4gZZjUcC*X3VMIM^&Fu0J%C&v*!6{sK?g*?s<86pF{Rf5mg8jW7DFa?sV4Nj$Cp zx5@w4rw=yqN}RqS!S&??oCZu1r5@kAUkH^*xh0cy$t#$)c^-$)g;_~=52ae72Eg6NI+tj_7!4Q(U{_lS#0oO!$`59F|Ly<( zzY9~~_0{dG%W&m|Z}=S-(uDo`^)E*M#sS>duYKP8cQm@T$|2I%_+|C_^{ZDo3g*ga zw0h<8<<);1UH%>Kdgoh!C0oa%(ZBtjund5A?)`83eeR9_d;f5&SNcAt^nF|{^xI=h zn^z-uPoPW5qPVse{|27 z?Eg6EU8u$w&p>&j@yFYjcfQ=d-1~KZXX^y&FD>=HMfuM94ysIF$S?+9zT})p`thhU z&d0=2TT%10|f06h>?k+hwn{Z;JzW6z_jH)+}{Gw0fjN4^fvXFxaB<$BP}sf z`7>{iaWlqPmX->BWB_5D_MlQ9IMSkyBbIHadTgxjYzPInn@MRSe|ZV#GC-M%hb0B4 z@T=^MByjBOHO*knqeI3l9+lvcgC}vN?|Mj>YNw)?Xg(5A%v&xIUbDwIx8hnYbb~xd zuk08nmm{A1Vxx~0d{Gn{c=S*~2z(~>N9b3*>3RYbRSo3XiAVDU(^BK9jbT11JL4uP z_@}X+tcAm@S}XO5(POdzPld9jS&zJg8 zdPfU^pLXxfO*^JWJP+w42^hWQxQEz|p2vmH97iv8&CaR`nZS6oY??r%_QlG03IK?7 z-sHeSri>P!Z5_Me*x@tvA(Aghq#{7PwTFj-O?`5S480-*;Ph8!Of7Xj0+~n!9*bam z@z&nqHc@1?9q{jG`^;XVvfI8lBWfZ`?)Lcqq#EtF?UyfpV;lX?X8YUfV(;ZTjQoIK zfLJVHO-wq34RNmQg&E7rDunImuyYXO4Lo=pw@2ufRUki_U?xi{+89HrwjSO?)`vh` zR|n2WHIjdaKuS4=e6)yURr;G&RyX7b*8q^5_}=qDaasA53#~xhBGMaYtr(x5^gtAS zXoeS-dqery`TnGYx%{ZSC4c8~bf^H?+ASd4+9wsg@oar@%Sf+$??3n@zPu%zR({fF ze(_g4b)`Hu-oQ$)dAY-i|D?m8H#f_P*kWIB$&0M?0`FPJs5t3%ro-k-{Kih ziCKZ46owK!GPQ0Ik0wq!ScPdg0r0Pw|GZyg82qC5YvuU=h!+w+@Cf9FfCUA=tKmvHGj zzJy8KxP1L>&yTC#`O(n-0*}^J@o3ZrUbwnFU?S2tU#c8cIzgYo6`==LmGFB&3hY{m z2j5!ppSK9t6d1LpNac(OG0wr}A|5RRWiJJfKAs+ab^46B<)38H!~>Q=S>v@?->Nj2 zd(;fcBRprUUD$cnXq%|X+sF5J(ACk_pZ0e#N0p`(rx^Osb{)SKz)8vw zVDQBu_>XTvzZ0dHWVejX+^DQ6w$n~SnN zZe8e?C|AY&(vUbzjY8~(B;8;FVlXq}e zpXpAoz8B88tCy-a1DDAbR=zS>%i~Elx_Xx!Pl;PCADOwbel_O)2$N61!>^euDHHZv z6w_5+p3Z?QQoua0$UZi-GmexpeS*j*7oXM_dxvji_9%{~H2%o}t{J(I^xBdNixHKH zg2M%u{i!SAI7fDJC|hV{&1Cl;N!zmu@l6jaMADk|Q(9?x@-p+NsINDH_Y!fDsr~eluH| zCK9X=f?HUzAoyuu#tvOkr0w+RtCf}4uU{{BG1*Y$%eVnv9xHdU6}x`slo^|9cjnU+ z*&VvNePU5BhA>ERPvS&A0^>x~)(L|+Mw`K;`zR!YuAr^59v~D+~)v!$BL^KJC zlsLN+!0xzYOf2Wyl^vo=9|Dx;p=uH8V?SgLx$@UawUr>-8izMa4QK_6%S{5Ozw_Xe zDtKt)^zr~@yH_``n$vZQs;zncSTSj44RTXC641DAs9aKc0K$NI4=Z|2nY8QIls>C> z#pds?8+;!tr$X(~_fIxTQmuA3BguwmM7NK7-6&jwHUp{ib&QCZ)#<^y7`o`f*4e0}ah1leLSB zSJEfs4^L#-4k%1{w_`GV&o&y+T$dvI!Agol!w2vRRz5 zPiHM$Y(X81QFWtnUZfn2#bs4<7~ak(V(J-z^0D*pEvg2g@e7Y_PAt-D@i1qX9n02TNWoC zvuead8WoI!FJjcwNd;V4FdDU|5Nr_ptL;FO$6^S-P5g@8&GoFZ!Pp zDSw9yE8yPA%3~)ln-^jt<29vc>9-hozfBgP%Q~U%6?Q;RL0DbSwmSeKjg??H75q)a z0nU#~LvNJ`6=!yI>0hPc!R?6m>gh%tYB!K&-RMx|`Eg#w>FyaPV@bFoLbGY7gIDaL zz8KB5OpBzfWAD#C%dBY9#-0t{*>qiH&$S zfuikGdd6b}$iZIk?%yJJhl{p!zyENF^Um!tVf(dI(U3%!@ypazlV=CG1a0WZUH1*4 z-LsRWM_V+vy3bpYibAlIIFP-AD<0on`>6OsHDh{k@%5GIoQRF}10D8g>m^DotzKEW ze0B8I=v(o1%r>&dRPsiPfdqucB<4}qW?;2NL8!PI?RH3z)`C=cve;R4 z`~#gor>hqTH9ewLN;%trg@zscF(ZSh&7gx+CD9dW4OEmK1Pss6{%1vZ23MnzWndmT zmZkSI={CJqhh*$KuBhGz#06C=W{X5s3!+-1x|82k^|`Y%T0ku<=!z0dv=-3|$Uvre zH={cW+5wjmyS?0X53_Z(02N3t&>T&PDXa7WtRjE~&(fxMavs*Itj8zCq$1!&I(GEG zXl_yXsl$=fPpF3$CJR(k=ZFk*awkWbw6D^AHKNN{yE`p4xmqFVxPtVwRhA(_k)yx< z^{@V)&9;IP7wQ-m40s(9g$+~+U|Kl%Xc9}&GCbe@o%y_*qv-Ld+KAovL3>p>2@D}c zKs_NL>A6EUKW>|Pl-j0{dB^%9^BNk3zInKxcsp^WV$JD1RL^M$~ z=O+uSut(Lqi~hmM|Ga^xB%Ay)TEy=u^L+Qk%hM!;ms`Z<{ab4DsC0bv70!p3F!(W1 zMzMEk#n7E}N7A~M>Ctzay~;TzZc$H+R5hhX6T1Ln*~|5j>4DUUO2OVU7a35j=7@>D zfia0Lt8z)U_3`oHu_~H`$1@I(EZyBFL*JAmaX_ocl$-h>s$R+!NFlojoX%f;c_ei+ ziF@BYS|9z#`sPun#3y%9tg{V$|Y^)(LN=hHdp4aqD1~8O(GfR zy-L*TZ2us8#xO{pq=%mWh^jWCQzhvQ!OkJUE$$ja0R~lC5=fyFoxvmF3u-jYSE8Z! zRZ}Kjr^-$xyk?G*#G(_|bt*JoK=7+%c^`>E^I0XsMSJZ2mfE8zog#MOA_f_77{S1{ zJ5pJil0dlE+wkeVdy&1ep6^1PQm7L%%haROYD8r|4)No-oxMc|6E>)rYVkEV=EX-G zp1|yFcf<+eSc10d+xaisTgiRwQL`kEPW-Rd-pQcHuTnS5TF?YCDa*>A% zhQMmFA0tG$Udw>?I1r$ot0qfKf{WiYkKxF29U3~W&?QC{X-;S9lQJrMd7e~nfeTbA zq)#@bg_hd|{?Go-=%901P5YcBgGW-E5cs9rzZlD)>enCUl~xnvZN;QquNq4kSuIwI zINpN40^3k;*-ELnv%xD~SqftH)@Yr~PFHhP?d7F-ZYU#b&~j2~w-*3UaX###@=2s= z6Hdza@DVa^&KADBbooZXlO>(&4>|;cHNl$)PrhJjI8+Q@K@I?mn3JTW6Oa$6bwK${ zP|2euU<_67qC%Hod@jUNvtBqmx|!)ZbX+(pDPc?Q$qvg)knJ6wo@|kLq?VlaWkiyX z^PXnE^zxTWSH7q(4W5Oh1W4EcnuC|k7)zXwvy_0)9&9bwBIhbcm$+RLzcu`$k)SqHTleC*L*e1#sEQl$7P6ordgp zyvvkBgd+zuT$Aa$i|5TWGsI3Te}dGTW64Ly-6J+aL$j@lD}0idgFMgb zYG&wDXBzSYP`3W?F~^g+<5gTqmmzf7k6^~=ek6fc^aQK0jqhB2b(LHdt}f|Tkbw=0 zWq^0M^)xW=Uu&(w|uw1_ZP~Iu7meXq)=X01zO$ zAg84BCds%owe#_zCcF;EHKQ9EL0RVaYJiZK3-5Tz>EP-LI<0lI^NK>rFbf{O@2JrF z?kjL1p-EA!8>V9&*2l0`QC%SnKnXOPVC;7M*p4WNl2{!eX2E))1`O`p6ocwM3=Umr zGj<2T3DAZb32W?!94gCs`|;Y+l`o5EqoK6(Ipt%TgU(n!UtgtD(FnO#Ur_=W*>j^q zyixZz9^H8`TEDlksqLY77(z*m<%Px}KvRqi?`fMTk20EnIP~X0#UDiP1RUZ!=Dc^v z()@XHa155;d`U2vI&&8B=nR$OR;);0{Ky^F@Rk7VvJLf`u&5s@McPf@oSqnJ-j;?$ zdz`~o_QvAZbCsPaB4t-8@L9gF5E~yNd4fz z=ARA#pQ)8TU%D&@SY1Q55@c?)kAWZC0$TEeRk6g6Y?57OwMU-B=s*Un_^`OFAfCAz z+%2g_OB*RSyZ}CB1~u0l(T5Lo0(6)hFR(*U;>x9e`R0hyEA|I2kkD2Z#JnO~zrjBQ zP0pE=pi-SwI~@Z)^fc1(;b&wLSQjFl_MXq7(AA}@#$Cl|{fg}+51ys>JYjKlAS}G= zy>SVN#J(LZLWfY4mGiL-bfp#Fu}qk6ktNT!jyW&NPgsBQjkK=)V-)6s8yQ8k(?M2% z>Qyj6# z50gwBMgbU8(Q;`E9m{2-Md&8KpUaJqttU}zlCg27eWsm1^t0IOo3JKQk#SEDhK~-_ z+=KJ$c3;tR4 zVdU1H?2K{iXe^3p70-@f1|mgmvgHD_9Qbc4LD&iFlPTUgJ^jIlT1$lBj57WBe%(+i zeu9_;_`Qc3wpoI-_ zoff55pY{}6b~gj$)Y6RwMOXlEXr>tJNLgakD4gMxxN+n9mwGG+#*HZ@0QEfQ^HeOM zC(2Ow=|X;$;NLV-Ie7s~ZYef20!Y8+{5FFIsuUl8` zQ2{1;-ev1aeOg@(->EV^tjOi{5`4A4dcn}xf5qVvKKVv3Rxd!ikD>+zV(er4oZ4xo z{GIZYc_K@%W>U89C0! zk5k<9Te3e5u3~ir~mS-INq?4juX+c>sygG0~XskR8U{ zOMFPHjt}g|A9>3`@z~y>CE1=MHD6if%P(a)YUa@9ZF#Qo^~!)O-8zY_h*1=>+g|Qm zH&u;~ez|j5BXKpM$_J@cVX{`)xXt;(x<85yRaO`p2jEH>JE3!|axtsCgMhuGmjUka z$%4a?mv3fL>e|xkDsiv=HZX{cgYm+#Y&aXJ%P|H!19;b??;n1%{$%}LZ1rF>#0r{R zm!V{_nIz~4;e7w#!TRP?aWH?C`@kFp>xKw<#J-k?!T*jH*&voB;HJBg`9x6sF4LI#rD{jiX!~tDaMXye9T`Fr&BtKWrF(} zwmmcx9!1vFPye~~Z2NCAXYiWgp)vR(TZ=Bn(Wg*`@ZaU+{OR)2mw%Hez~G^l3v~D3 zFpHY)Ftw+smQ3l$s8iuzE-yNXo%Yo`F%hu@e1_~XURj$nu z^|K6SFYM5+)YE`0-KR#B47s>dvBGh@C>)oA7b?`sozddcNqi7DY_F4>X-cu7B9Sz< zKR<6X+GEw<-LtKg_*nMGp+kKPHT3~8KLJ!p>(Pe2Fj0>dg>l;d2{aT*v$3tEcG78i zgw{Q!wN=CZ=xpf`|JO}D>=WeS_fH=1o;!-#lwFK8CwPDyN;d0^&)7!!R+t-e0pzBC zln@{e?g--) z*!WTG^<#sxAW6Umq_KUZ8EcdvE)i*GiS>AAPgG97lX5%U_t>9K8ZJ9XfW=hjja7rwgIU*1kHj9Zp?%GrPt}pbcKgm@h#S2Vz!MTodqx_Py^g#k_6F7EY+wLN2)z##KsG@Y=c{l$}j}cldvR#@+gAFSH6Pp;9%Ad@^_C)feh{cydpl0yXc-a-BxVB!6Juvg&fCbSg>X6WT>V zl5brAcQiF-$B>uj!Dt*r2Xlh6-6Q ztijkz1BF#Qj0lZ15NC-Z1SGSZe2sd2PQ;?o5a$r#^2%yYo)-rpxnfddiqoJhC>Xb- z3Pg8!{DQuA5vNjwJa83;;uCV>Ri7OLC^m3-rpX{wFApI`^~*51koli!Np1a#(X2g7 zBK2}4e@KL!Tp``|)Fz!7gcjyy zX5j`+mN69h#tIS;2wDy{9k-COT$)`70inm;q7Tzl4MXj4!Vxyv+E=`}x!CF*w3&f0 zA{J3H5ajTHtApbCXaNKkzb}}5+Z!35=CRcmqcERhUQ22`!HKI3{0%TJb-dG19?;rN zN!<*O5TMS9sl=t}aJYW8pR$ypj0QppdIpjYF|kB%_^l;|=!68cr84-Y-Fjey%lv=Y zOQw{ZY1RzAC12EpB<;VRX>9vJm*pj|w&M_F7!m|)vQ;w)1QxhDTD)5vovO~AGs@w^ zm(F}JIQ3S?Z!q7{;le~n0~&=oyBPMsN>O!|*l%BL7qRh3^U+Lla8^a~bC)U1)v znkSQ2lZiw{^YE3LP`XmkW=S<&>3ScOc5+JMBk>^GOk0e(*^Mq-+ej5CnZ`-C`if-J zqsW7u*)aj8eMX@JcCnl#KsQF4wiqH)AxLT)B)~Qd-Fo*1by5;n@VtK>v2VPvBi`P^BwnuVawvAT<4{Fo&_v zvPC?wMak6;j;1NhMd}$Yj{%a-Q5HrkwX8=gVvcU(g(ZD+G+kNs;Rg>gwYXo#L6w#K ziK^5fqARJu^P^X#8?tt@w&#=I^;*`OxB~C(eumLqJ9y*{)0I}f%NzyHV6Y{#MKVd1 zIJlN)&^-{&c&(`Jj^ZectjxmF0XP)Dit!4n;kZj-(-JS^eRe*ZVJf6;hEi5r%NEPc z<_!ltwY249v)UOjU=}e1FF<)p8h;4Sdl|!jiR^D(5{cqK@OW__uokj)aHN zD#V!s@=GV5M{Y4=6ho9{kQqd@Nhw8pTW^$9EYYt1sH^~%f6=1Bfv{Wj&fn^kqNE!J zh;lfCM+Dl~NdbuZ4-cK5Pv^y{qB{;T%^WRj4L0~na2(-=gi zb^_Y?44iWO{8XmEb}gd^l$GA~^X?Jh5VK8Fec9q$Zew)_BhrWVUd%E*mbN?Zq+L|t%8a<^OST)Vb3%o3*u@anqjio z%-`};iC}Wb%ZXQr8*FaY(nD)Fg~^ml3&DfXkpU7yu5btH;7U~o5o}&k+4V`M6~1%Z zC%@-g{nBa%jaXrq9pKI$!C#wbIg=lutf-tV!b>EZP`VEBCdtNP*z%MkpK5&e3EBiB z=bJ&`17@c0t459)p+ni?_qka;(S0o1Oka?kbos zJl9OSbDMX1hELBIU8URM*3UiW598l0;%X6Do63{aIDMU*NzmRUQA?%2(? z6plNh6~_F>Ferfp)>6apT1&&uxewl{a{;+KZu6RxVqD`g3t(vWRrr<7HZ>d(HdBOQ zr{X@r$fp^Zj4aks4`N-}!HoEo4}(uniMo1$ag@h(u)D8PG3Wuk-;9(%F=ryZaTOOk zPkwnu9F5ay2-!f6ute&?Uh<U1fh0nIq zU|>)=Zst5=IH_d1vQSZ3Kj+>}C3w&Ztr9zQzi7^QOj>0D$V?-Z#ks95W35!ak2C-J zW#00E;L?#cF^8d-qGP<&9P?r;kOPs9K2H>YxU#WMKBm2aVLDc9q-L>*74y@?9_m2pZRmh#ICc zQ4$!Nno0FkPC4hrTf^eoGxUo70F5OwKVh@$gFC`H#u#oPLl$qxK4ypMQyrqwynZ%E ztYGs|S|km{jMLM(YO||V)T}>YE7mroH<#7m$6vKAMsJoEsQ>mA$)28Wlq#m7j2uOs zYJ_>pm#tRRo@lPGu&BU**f7PuF&@d%aW=eWaA63b_kJzz5)i})z=0o=S^L-59B1-w1v;~JVtTU~FiBGa2ei(20*zGFtzxNxq8 ztWAkTptZXt0@(sgF>ui(!ht#_N|e$_Qkkv9=NLAl>yICJb)gAxU7dkGzuOcl{IgKE zqq`mS%}CODEk`vg5w&Y9kW(87--)kr2|W6ayvzv7h)X`=TDYD6e=X4K1-6|61c~xP zwhAO9or4iC$TXL4ju$VmzJ4~&`X%a5M{JLGqnq!(2;qETq1?I>A!Tl~BQ)OP63UXurzg zw(9_1`t=fx`{`K=p`yWYgUXoa!>dkO3(6XY&4mU6ehTtkWn zjn|stU1T*JQ*7=Tj3l({)C)s(tu<=l_tVoc*o$zjFD!dL{#aZ5PQGnvDflXCB2T`$ zG}OR zGUG&+Sbq%i4n!eWP8n?2Yj;L~`Th0Ay77e?^xXI&`}5qv(b|Y`TZiEemeXJ9#@kVOjIm16i=qg9xuue7x6%kzk^yz`K;XDqA1fS~?hbken(r8lULO73Hdqvr3xE`u-#t;HLm3qJd-BLPb@+6#3igkvl)iOJL+QEPt51ufP5V4)FnbF?a-$48jxpWLIj`b`2|Mgk&*nkpR##k`vXf zpjE;U8BWqDbS{1Q=EMn4=#WTwBC?&|k*O05S|}=7ODI^q!)d=a)-EF)T@eYu(xFo{ z;Wvc*c#Z$)I8n`zk3QiiLYBUn+&iQ8a*mDytlnn!!KP*@gV#cp&j2OPcd;(rD(A~k z4vA@@<|yiAhQOH`6nFF5@mhUK%%B^dGIC8^TTQ1eNFxP`BGV~rg3M_=iK1)&gcB2%m+lj!lJDRI9GHu zdpd*u&0{P)R2N5!`B5W3QhYR{NZ}F9vCA?dN(cfDwp$I0gnyXe$kr<`4swII-AlA= zSkDQ~#1$Q=0($|*nBFZgm2I6zn-YAMkR7AZF#WVPahwXR5wQy?QGylCG^UK58a18a zZ!bcA7)rD_fv+_7sZA02G2R!MDpS_F+ZQFZ2JumuSqGw_Ay2^|R_^sL852WpZG^X6 z2^X{p5skA0SWZh0T~lan6bz_zH$>E!Q^M|5O;R(&z|y|4w@zHY_wuC51pp3z-8$BP znssH_*<9nb_}6RNTlbU#Bqs&3tykofdF4aWYR7B(uYdgX){0)j>p8>W&?#&yo}0cp zT6-v-lR@Z(Yeq7M!&=5vO@PHKm}G9?8_tsvV6do~N*2=ji&IQ8oLb+Oj@Iyn>p&Er zjvg(4vmC~gazK>utU5JZOm9YXZE+6?&#DH3+EnSS*RK?MPe=HUs-D#0e1m>IoTG>J z(YgP49)@yU!LfE2CZE#D?c`ILDAM%K5#If^5U~hHQzxQS^7f?VRjx|YETWADbt*5V zWlG!6%EM5bpHvv#I5g5|oS}{ID#m;KlFU-NAh5?>m7HT1$CFDb7477XnNrA z$)^GNis19IHz-aTwLrKhd>*X_V~^=KjYboJoun=i0QV%5fH|hpkU}?0@N(obQ!Wf# z@4*UJWr2$`t5=UuJ{D5Sk|~}{^ObD6uxUGSB^|1+0f{tX&<)Ni&Ik5obBM?L>JMx) zZ?%|ts6Q~M^g5fEs6`A$9@#na7;^v_!+$wLn~%+`Ah()8NEqalW1_t$2X2`n7R@_& zYBpEuo@rc@1a6+GSJ;h{!<8V8PQ(-*u<&j_L-T(;Oo|Wa@;WgIyQmhh?=vRit(?5|cR+2*^oJ5u9JjlMl+ zwz91Uca?1L97gm@W%P+d%OogJ+FRpbdj31aU7l>Pz6EBxsJ`v7Z>wBe8Qs1m%aZot z)f?0hU%$swvaFwVSJ-5FnWlK0mB0cupLkJbJ{A8}eS_EEDqZeI3|ml-R?}o&3*rX1 ztclmUp@(ej{^mycGU7fj$KbY@QL_mPSP_g&yED@J1e~cNZcNU*1xlX_90dyt52z`E zT-3}!o=hVehaf`N#V~b4j#73aj+l-`d9WlM@73^gB^TWJ+~4uFB- zrL!bSXx2KYV1x#juZO)lRf0^AaT>jzh!^?kJqo-3w^bC zp}`bKs0Wn8Iody+QZLE^Afl>dRHkdmz*`H0?xlS2d5GXpNejHVmm3G*~?v9c{n9a(n#wSRX#WJzl-O{DnV% z!Dr5Jkcrh;gRzup3C4<6EAZoAz8w8Vhp#*S?>+we)5pJDxxDo8FZZtBzyG)Rdz}*k z_~!oo`}gu2&fD={j-?=IObp=u__fVu;WHf}$XKeRbNVO1*i` zdPp%n5;dY{_3TH@$@3>Os<957C?I>%mE+2Do9QfC;-z%?aUIfEDJC zbsI^{ZzB09l`2=oC;W53T={vrEf(IU++mWt-*4Vux?w4-Ppj8-VpTLkw!E?SacI38 zY~)2o^|gi@=7C`xqJ%JN;2riFTAclhPV?MaX;=7wDf#wp^b&@zej2`h&x&Wm(eJA_x;_cOP|NW^23>qrxx-G z!*6!}C@e(-t|cmDJ6GOae>Zxn1HeeP+*S@-crJUGY*UiG$OJGAikH6?Lp^!3|Ll+h zv{J$F0`ePI$J6oJ({Q9LvheH#_Y=NkNKay?souISqTTSKwkOBs zRC=;?3NC=2f(3k2aRHxU8qh-|<3plO2>GqEWzdfv&{K7P_~+#h6Xm@UIn1|89#6zl zCnS($4|3?zAFD9wJV$eO{@ucJfstJ3Na8iipq+w=P^x{UeyD$CA3%>CHQfoicn`I- z972*f1oJ)F&sTj^+0}AVna=g{n0hNNsJnOc66dRUk!hT~p%vdVJ}Y0L*=GF2wXj>9 zbHB?OZIvg)aB0=vKSOkxUo+EqxR>iqw%BYnnvlRK3@_F<7Ie^+&-laoerOdGMv zT`yq&kv>(79RrXbj00kt9zNqf*J-VKX@6xyVL|-w}V-@&T7jL6>0%&Nk9D(!_LN zB|=*Bhzt}-wn&92km;5*(i-?$ijGeeDp~SLqUQCX6kgjq=G=u*+`729@YjXk7JgZ{ zG+JDE$ftdOdcdc>g-aL|eyq$tmQh(VE;!Q0D4@5Ls9RXE2K@B=Of@g1bAAeb3BER~ zzlbwTcb~0ul=$quZk8`^2)}=Mqs8wgtz?o}Rr>gea(cQzcF)F=?Q)>S5R!ydshkp@ z$>C}KLcRNEZ9f^$S}b!wx5{u~MxJqgBGav=)6^%5-GVxUSE&57z11n2VkB~Z2-g2v59_@QWjo0!+ zY8s6swi-owJQqHr1GN_&!JuCoF*48@%ElSSAs(6AWqWRF|F$Z#L7=)yBBLsT@yJ|u;l;#m;d{f{@Xqpbx#%qYaQe1 z{n@;z3v8_C$q)4;sxXMAwNwkaHNW0un~5&A!^ZW^^h;kh_(zz z6HX^-U){<&RYg2}K|DNY(R+IbSh%GlP|Q{Tz1Uz1VWq=7C7g(v5m z@uroL%HT^E8%Z0xN<(|#O#)fcLYJ+MTsK6u3a~uaxj@FLdJ)Ll?iwdEsd=+)9|hWa z>#^?Ch#v&Fb%vYTx%!V-`|y|>j^i~&qBISsdXKr2ZE67o`KdfWvIOT|w6%)#P#~Sc zCA|kceM4)|{hS?0y8SaAg~tU271Y3g+amAw;#v111Ho3xZVE4}polsM&>|?5x@efG z3F++(SUpQbmKEp&=@P1KNLU3%^IgNP3u8{O`sqLa;0~Z;8RG4n9FX2E^3%R{pa{^ zUb9Q1T&G2?(@+1oz}I}b)UD-AVsdWh?oQoBDU#bSn-`ia%{KC7p+@URuc8Mep3_iz zyS82jkLu03+qaCCJX(+W{N=CrRRYy-nz71ChU;)+`Mkpkrt5U5K-@?5KdF_&fYFIw+$w<6@MY#9+mBhVw z_9AQ^!O?aPH$q}s5V!dfglG?6NARn&14B61Og9vOnr}%TNNB62A(dA(4=YU$@qOAA z#lx`|>LvRebE=uG4qf=s`lHPaodWY_?{qZQH>AaeQ9>PaCNby`(HHnjwszEXIO23a zAMVN_E`Q6BoQLWlo{h;ttH(uND5hskH;H7u*J^_t>2_o{^~*c(#i`D53N_F068Ri> zvz8NU#Gv7fy+;k1-Fpum`D&zjgl-vF4XHU`x28~Cn7I75CT%Vn+o-aVRk3Z`3Ml1c zhU5ednuwf(Yt7T7CQ}1V#o6NfF+_COI0g*$^(Uz}3fvXb`;?5Z;0;uoTPsb)2Qe4h zuszUG&(V}v?)=ydy%&z3d2OYVPaYz4uj7*11Bhnc)x04SFmv52L(IIbzHFO?vuN~0nJwSy8GDoGs<4r6c*dJv;mTi>Xjk17Qrw5nl?)Lwo>M!xwu%%B z9dO^C9^s5G+a$O+VrIRqp#-DMxy%F%c5r3$+>Lcxw!A&#^8T*SAj8Ygc53eg*w486 z?R$E8a5DQjA@_GLuvxV;Y_)NQk&7XN&Y1-f@o$n-r$ma@;ouarY`j*MBst%vS~vd! z3;IT&k^`&kxrsa@jYb|ptp@+QJ)TaZPVcjeFuF6Wy5kdOy+95~n%q!q2RdLMlFFyL zjg2hB(|pxu0+=w_JDI>da>~J#QL8F*ZEE6f>WV0he|Or6#sLR;M8VXJ+5%P=KKcyr zd$%X}dP-|Rx9Ff0W;X>L5g1Q6#MZj0{|$V--d59Bb||Yjg{7)Q@HqMs=H+{;}?|e~_U3q7U)+hs>rjH~3<24z=Y{V}5-;vnG!by15DGFR( z zfrzImFi`WPpbqtX!J9`k_4L#7-FjlpDr{ydpb;QHt&TS`aNtXF5AUK_&q!#F+EJ!+P=b=I|%d?Fk4c)JB= zE=Nb7P4e^N{LIn{y|f(%<%?=6F>;kqS|(CRJBMfLljYYaOd~)17T$G?3Xc79?92#z z$yaoABwjLeo1lbxW47+o1Bd%Iw}$v5#xVT`4DXqb(@=(wD`c4OLb_Ww!#(TuSvmi@ zM!#0GxO~xA#XtS0l(pxDdtgcKoc4=_SS5KCny(2Z%}?c@VL_2GRe*A1M^)%t$3B(u zS`>Q0Lj`tw>(Sk(1>f{lX#fX6%<4e<8*gaVMpP^p7;S0?$JUr`G^z#@JSQvTii)YP zD_Om<{813ook5cbUC`6r+Lb&i&PEJphC@jTP-rRo3Q6eTX>?{o`&5bnrUtDoD>>0K zWb{QRc-7RaA6gQzBH~mUHF1nJmY)e!FcVy&>PpyRt=iwEU~?`G}Mucxy2%6OXfqAc|o$nZBp>cYSa-09fWR6qG#)? zneC?e9Z8Vat~$T)YWis&v+h2WU1>i)gf z`=IWmCeoI>q!R*hbt3~7w6#B4B1pE_bInOyGzou52Lfq$6_J)p9H4Z8YcE~G@dspE zI8R_WP}w`IC|9nO@=RW=d4!+v{UL`7b;vW@$T57+JWrF*#qO&;(jD73F}Qeyd~$DJ z_jgC)SDZF@rR4WM!WEbpLL8l28wr#OI+a>kXTb>}3sydm;S%IZhBvBH>b|6fE-FMq zD~IU>3zc*C8vj<&9upe3b%uN{^5~mCem45XIY8gUyMi{e#kNc&2xsa(j$~2|WTUf` za6DA5o08w$o}#C<2Ui=2Ca`ZOLWvK%Y70|L!6gC_<}M+fHZ{JqqS zWX>d;t@U`L?yAQd!wvN{L6gRB@pxl%>zJmKxViIFc)5GKs0I|z-(6PJ?ZXE<_0!QO zG~>P5d;jo~OP4Nv9hXh-&7~!8(}M|73~~=1(t2$Rk#M%M?IBjM<4vL_a@u(51_bRpFv@L~G1i4r;>EjY5!( z!}`R6kgL-#dZ_8*SkM1_kCXSK@7Zg!+_IA4;W0^X;cWS=10=U>admP!OuDCGg`F#& zJ0Dg7*!E86twlKZOvFwmYcT?ERje%}+BqAI$9+Zi53BhGFnD+YGS%BYCIiVd?4)Dr z!^$BMk?>~Z?k1deX;k=B{e>j`_D3AM_57?mE61OF-%o>m16pTS$m$nDOT*fY7<;DS zOmlX+0Dr#bnY(O_Fur?FYg3g@{Mjg;RuJPgz6W|PY4O|uVUKSlWztMF=eo=~> z+Q1o|UE-uo71tKHJ8ehw?s2_Cc+2{)WMC?_SE#!n@8a={y+3vN%Dn4SZMH(GD7)NO z7YgOZQG4aiYDdrWP7Lj&++mVwb)bJH{C$XrQ^zuVq+hx@#AYBXR|OkB6~YD0In1-O&sylPE}S0wSYd-D6vlltIhaLUY*!!n!YZgu=N`rF?+i=5gpLr&(W{EnILdFvB<58Om*7INny z4!sS`I0tg5q;Dg{3@ruDvlo%Ws)m_rTE`DgLMe|2Z2IpW&gIBTm!-odPBC%iN#}wz zyyrER?vAzff%}Z%MRTp{K>#CZM2tp$1_*cVaMVgehC|pA?uY1MM{j=Qfe(Xm5uJyo z@{TAFE6<&i^PeYV@GFmYiYfb|b?{Hm1TVSx&VyFCPv&v)ixi3no6}2PMXsW_P{Emc zHWRuS6eAcnvKJ&TX{Hs8FoZJm9<^(TvkrB$?8e>gGcqP}=t7w%^q}15*wm-A;=*q; zc!A|P)`>gt681Tx|E)*Q8sMv>2boCA+tg5zM9Vfmvn;?@pm^GWO>xt6^L+tu&X6%F7=fKIQTXbJ-z+1dO}O zUQ$I~9*Grtv^1m%%BiYEo1Yb`=Wle$!R-;ZA$Ik+d_r?p7dD=RVJ<t}fItdC2P!oKqPll#PxM@qOugxl{CYK2i1k=_2T72d))nJJ~8*`w= z@i44#jscE7*tq(-2la=&vj+aaW^y0g*usclo^Kbu}L4>FG;2eIM`oKghfNhX=}nDb=~XJD>mPY`;>8&p{+ziz8Ht z_KnZ@oI9M7euB^hd>e!pNjOh+@c95^)&BlpSP2!MEiQQ zaJxUh?x*u#)6${8SSDb7?P8b9{a?#psUATAtGCnz#a7G5Mz^~9z;+&YBW{L6qy*4Z z57e)sP(VPD0Dc)3Sk$>rPt^(KGujvo)k1FcVlUmBXLGq_nbl%w2kvu^Mij~*Ycv#{ z6Opqq9!Ils1|u2{9X2I|L-mK$iO<@KH4iL(gM>NheX8b6u!;|0*s5VL#Jm7rJ%2Vw z=Bi40F-o`$R8>zS?67lDBB*1RE<1`9bL5choJ@Ge=g(QLgP5eD2f(S67R6E6I5wiV z70)u**vUSH5-E8w;&qeStJa$%#@itWSM8S~(!@+Ht-!+2LSsR#I+>*bSM7!A@{bk* zTwe3!Vz{CKRti4=;*<{)^Vgx_LF_^aJNbRx5N8o`KR`vMhXMwT{lJ@2;HMtE zF8TDWhO%>>(T(9C9}!F2WA*l4v-H#qz|%1RK17M$%vU$mb{id_Vsl+!e6V!#NT4k@ zYE?25z{kNbAv8Wujp6Y+Y2Sl11Q1<2cu-;i?6vQ16H0bVR-tr5(*DsVY*#hsf4ywK zML*I~-81h+F@3JYiq+3Eu_BKwYMe98`8ilmQ@N-lq%D>1z;tYq+19iu=wPCpTQf1@ zP|_Cp^G~Bj_GW=@vk*!1bE90tjehUDyPN;Sqr7goZFO*1SORUGjT~%*$MdQYCd~3R!TO6t!q_5NTtc+^(7jNyq-)B zO#YL?kBXW8e@mH`g_0uOSIxd?*)Gk+9r&l3jQXP?O6m%zLba%LgUIaSTcwxYp05=&E+yiJ;;3j4)%g4DymCJDEDHjVB?hm1}h$iaCroC!Z8)CR5iX&^QsZu z+ECV0N<7_bMTzI~12s8bXivd$9WdYo((sJ(O{d7^Un+M7uX5vC)qg-mNg=3<1urKd z{Gs95%h~g58&1kg3#6wb{f8gWNmf3XUD#>R5#Ze2N4%mjanId*nN{tKluJx29gBAM zwA77cxGf!(SkmFh6|+-l2KLsK787x`bYAFkW>zFuta#{q|dYT$${4y;;(u zUF6h6ycuSI^r#W2V-5x{{@1B_R@Y%ASMGPL5}cbD4muN3ZvDc0q--1Vre*Z=Hs~Ht zH$KJ|Ypg}WWZOo3v7~P*l@7!{6-F#7^Cg*Ey z`{z!C6#ia`Nl7#Gr#Db9ImyR9SfdTxt~_3+v#CqR8-To4d3q_AYD!*RM9^ijHzpC> z=o3BMV4GF4n|g-q6%XKxU`gecSJpe8?hyKpe$b!)n9is*-AIO&D2k7w_pOTHryqY| zv`b{*+2PL<_gG3*64spFH5BY=x=&Ha@UF4xl9bB+s68_zljVXTw-IkZJyvh(o#p7d zJ-TOAjWUA?fDV<(=fH@<3hS=FDi~uFAfsdY4Hs7_rI9hJpNvFR?6@w7-Ds~TD$cIb zsjy9*YGcQS10=H1oH`S=AJfa_#F0o_izJIQoo8H$yjk2-c(sNom89uI$hskfpDSK0 zAyiyRb<|a+Mm=Dpl-A2)^2^lI#IX`dYwf+3?CaHD>FFNlUB53xhMi(7yP7M!)T{L2 z2g7xg@ET0Pn~ESSMI>r7+{vX5T$ra({T;658S!Yl(aKw}Fe?|%p~rc)O7h~rj4Y4W zKm6dEFqB%M2Q$WUglwPM(1>Ktuj*@{Myp#B^}(w!{FD)V4tZNtRVf>`NAFLA-&qts zT6?fd0g&WT-J!vt5lI~^xr2s^tJm-A^s?eb!NY&0TWH9!bz2haWHVlXob0W-yJYpC)N9&3y0-b84@?90SPK{s=1#Tj?m5`7) zl*Q@lpQmZ4`;0oFh*}BMrxeiIVJ6Wj9E!@0L|5?tC`)vG@cuX8;zm!%>L z5&jk>>t%jLGS)hq-xs3h9nCA`ORC?@@uF;UK}eAD$(-5u|J8@glQq@XR2>ac3I@$M zU;4MFerv3MYkgKS8Guq2K} z!eC#(zt^D7bvH1%UK*I7Jd{oZ=H428|74?6Si;7sqpg-pU{hKN|DFTe-NK48(x3pX zP-qH_+FOA9kIvO(OCX)q6F<8}AhWZu@`4MCnxUr|?#A0=xA-5Az7}GiUhAJe{9rV? z_0jr+^@p@R>v&-C~>mCq+seulVdMN z*3=}Xy1%Ctt44UyCqk(UX6$Lx6k0YGSiBs3Q#OeYX&@0!*rU{ zBW#sAIK8V$U0Xt3a*$%CIm!cf*luc;O+1cms?$@!EH?`~vzz2Atkji!-a6aEDQ86x zMd|Hs33nN7L(1&SSljs_c_YJJ>8HGwgX^@FB0C+D;oMC~tPwYlCo~HJld~A>9Bu$Y zlDQx$u0I$j|MSUV%{`^|E8kPukx7f6ri35^W&mxYv|zJN2=+zVIhgg`E5WZhPJOM! zscrG2B&Xpi+elUc%TY2unb`iTBTc+y_pBG)og8+r<3A^Y`Uy4tM75DVeXF5e%$a;p zxi6K$S`w6}u8e)}nznLON2PQ^2LQiKs=;L5j27#az9axg$QyzQ{HZJfg=UIJT}jPb za>>d#a#{gg%kScH^LB_W@x`%@_(;*Dqcv@t?5n3*l8^MXewAd0sMW52Zb1Ly8+-t@ zy90K<><-wO&IXN)#hM!ze>T{)2VHAWcC^rKy?&g#pIGS3!o`I*lM|O!9~{z@!Hr$j znL#aq`#`W%Yc-<$6hZVdKj5geW6-0Z-*t7iCn8{k@bFDiZzww=sxGE&(8RmPh-JwYz5 zD@oSKZ%bj0ChHOL;BW>Fxf5mEk&qpjCDN9oWoOP34rJUtQs|;Q;cW{WNE=nlO(9yFo4&k5b#8q(s$DBoRLRECI_32qX+<@*}cIu>3f-~!2rp0XZnR*-j@PMEs zc~TxWL0L}wiu0cDb0K1p5#DLh6yKfHG8ElSwB?OiMDhHKkdZUIfyO-b8n?@IALF^EupsV zIU5ykTF*l|gL~|1Q!lf2&w!wcEpn!86xj$gY$P4~XsyDztjs8)OiKRtn6Q7MJU~#u zy$<_i2Q_w^LST71ua+hjn#r)4n&*qq72`2P&9*{9S;-VcL9MpSg9E;%Nc3SDK2zzc z0^+x(PKK|Wc(Exc`(%HMBd7W4)Mf_-uf=jjduxj`*h~lhq1i)=`Af@B8P+j{$Jx#V!bHwu4du}tnV~mx``b|i>=erpg7!d-VDQ4Q; zRd!akmFhcgK0(5$nB}|YJpAUk5$m^&s=ltJtFs$&m;y_tKxa?>*qxm=`ZGF5=aaWa zr@TYiHW4jw4zp~4LZs^Zi`Y^;b6_!MF;#skt-y_mt*K~Mv=1>HL|_Y7=>loCtA=bNr!Vej%BGk z^?JKtbavrq9&l|9rrHMj^C7lztekN0D83P?>dr@Y=XbdoqFit9_0}5-UV#r%@pRWO z9bYW5IzR`_K&Gu-7VzRLzeXT5+dX!YaxKr+(Wc3kT_{F(E zhr@m4eQjd^RFH6=-#Abn<`m3An52}}HnGb^Uz)$=ABwdxU$aJet<_N{yl3#DVww#t zF(43IUX_Xo424z3Q}vn7og}6#q3>;x`2q0wECL7u3AV`7Hfd7tj8{Sw%A2`7`cZ9v z%XOBLUV9!JK3-|V8C*W09{@)m0Z_>}sGkuh&drigLW`?`gAhYsM7~#J^adyL{+L5@dfgs?0W8%(wnlUT|r&P(CiK;hX;OgAYF_{RQ7w=EZ^J$pwR%*VKLLY5^aj2#Y-6?tn;hlZOLmxM1+?A@Pl+*Nqg@j zXv2LKNA^`q#=+s-YGDcY2T=xJS5ERKJtdZ6v3pajHe;%~La7J7ve9z#BxpO9osu#@ z%CGPD51=pM!9#MGfd^ClCEl}{E`3xaOhDM!nyIa2ZJ!;)A;IX%WA(X9WQHz~YAo=t zK5dq4>o9RAJ~w&ul-y7@q7 zTPb(}xiVW;tX4d7eL{7C2`_#Xi6y_V`-0`DD-^LuyQSo+kEaw0%!Zsg*_Y%v4x*Qt z7^wqfFdRN#dZ4xivjC@1<`f|TQwUe=DJR`2EKy9dMj2T~8IG!lK_V1F-bYl}5j5Uq z-9yVJEJ;`k|CEz=!J?9L$1iBo@GCEY-iHB7Bf}Tbs~N>MLNccV;bbD$vm(UkA)m6$ zTRFGUA&&yaI)$!`KzNT+*U6vP5y&G>7rqzAR+wPVIX6Jx-SH6}qFm_?Yc#wm)k(LV z-RIGV>vZenm2KN8vcVbX?T$Sf4`H$K*ap%^)yWf-4)UTHGv5^L%dvdr+mH|DU$dkV zCXqF9`kWVr6SgtSa2*i8#1`ir+L`4v2Ye<=l^meuf1KG@mw*&nF!h2(*@!C3?OZ(d zDO@MpVVYUq8kfpMT|q^m5)G{(lK;-e?cFFD(()*hE}u=)8msXdfNG4mX7cb_S>ejJ)0Eg=Q0X8H)CDD-^YUX8G`}Z0TTD|Om5dWm2 z-<0p}pXVwQuTl;9o*rgq%Tl#e&NQ?sp1tVmL26_mZWk?|ghmb0R1X8YXh?N*#(>qE zgg{CY+9Lz1WDb_TG__U=pJ}OBJ9gsn*1-v3CH^L?gf{WO=CHa>deiC)AxR17Z|ijA zt*nx0u4RGp;Rm`<^kdiiEF1k@N&FW1i$i(ICt?-x`+#rG5Iv&}iBlVX0^S3YzfQda z5PgFmtVCyk#x>~SQ+ERM2@x@}^E|2c~4hsbP zm;#Z;1kvll0x9tZgpeJ-?6azDz5N0dgbXdC)X^V2t`9Ac_iZLXC_YPEtzE$z+yjfZ zP7;*H#vD7OgAo5BGvr@rB3!>J6XAMkCcmjfEUN^Q#L~1Bwp*VT+7?#P`Ka2vAXem| z1!eCj>?il{u3o=(jnYo#ZxyHH=GF_uNnn;C_e#e;NIsK}EBDB|7~zyNj;U61g7&aR zxUs%@pX_nF-i4tybv~5#Qm?VCB1&yqyEO58>uE9|W0Js$Xt&254RDgZ2*AOfiG)sQ z7AbJ((L8Xz(pdhKmb%EIH=`bTH=cfX4M&)$JABIa;TbM#DewM6B;Weu;~`g9bB(mw zm62D+kRrkB3`iwNfD5|mQM;k!VL~T(R(#MoztJq^nb_ivPc2t5|G$udzi@un7bU-| z&MfM!Wlk&JcYC~X=h2;|2X`KQd-uD0>rYi`T==7OuA|cw9%ROE;Vs4@x*42i1m7ck zv!?csgDzd(s89_lcypE#j_-tMXa`$l2f`cMN-g5`-TLIn%NkepAXaXX$H*$vP!fbgBQG|n^Ku^okaPf_l95mhQI!PRBgk#Tw&~ z#x9k9w&1s44C}vXwTrNqe+>-tO9`dKDJi?thXSD?d-=^9!%J$Fo!6na9)&pVgfU$4 zorm0)(m7*S8|!Q-TbH&_Iu70qGi*+ts#pkkdUU+|Gj0l9UM7^j>DoY8Pn$KVUxp*L zUEZ72Ux%0Im50YWGXBX@IXF>Q&7vo3Ur#{IbFxFh0(LJ2PhduISJ?c=J9foD!r?MY zJ@?t93(FRxx$GTY-Runf>!1y7kakPe-CMVQ*xRWNzJajk$=Occ(5*|>NY6rgpQuXiyyR2ZY4?h@tdpW+Q z*PXMw;;FE}@;DQFvaC2~U>8OiBW~hYC17$p*xoqH)C(BzSQLK<&K#muaNw-vDh^5b z;6ma|37YU`#{V1U9LHX(Y&jOv&o3(@j!9T-95sMc4v6km~6J`e#rr;;WbYIc&&|Fp*3F#hvVL4)fKU zvo#zQ6lWRqR-Q^G+X)f)pPM-8eP%lqIa%w%B}a}vXfn6Q6Dq|v88rE-+pq9G9lkz* zZGIX3{cQgTi%kx|nE9{TlrIWm>_1}Z+i%8eKPqs#O*2srGQ)7+J*8o1GM=y6LpP{$QRew1<4i&H~0`H6-lC<#aE^du_g}&uI5=UE#xI+@PMcozjDo@!V zB>j?sA$rbsFkjf5F#gNQ?*B#5>6^*^|NU~|0Q%_qTPr6=3NMxL4Ydc^J3(7HC1AlK9s6*RpLaRhy;|5M?i1{F zNpIlF)*_b=3{j`F!g^cosOPdfe7ebERDC3u=3%xuqX;_;0s%EJqPeEWWzJl);dJ|U zIaG4m1*eo*Unp*a^Qs(iBnN7jvomX02&1ix@6?yzm1e(M9pyL*BY2!&~SOTtcZ$i%J=NlZA(Z+1HrXQPKpqCT`Pk|@)7n2<$(eh?v7aRt0=L^mEh2tN$3|&oJEWz zDwNTD%DBH$q?MYYzTg;BH)M@68NHgkAz4xZE4jJ35rBnwD~S870zYd=LMLEl2S(W8f(-~IdgqtVA3 z4XnZ%L-^$gdlR4sf^rswP{rOJP6|}AgPQl@55%A1 z=#d7wj#GK}v?j#O#S&N$gZP56mfft+}>K`{*P(=3Eq04M;N&PmbZnB3zLKXg?8i3yRebrHHN7Vt9}<1 zWuL5vI0gU)59mkG`0cTZv$D;4>D{QC9Oit0YSAWp>Yk23U;L7_NP6lp!W2ujz$V^E zdqZ^I>;M-Yde>bcTQV)WiG)g`Irx?-8wPdWZl!B!Mgs8xiZ7o5Ja@YJyRi`iEkh*Jh)sYOvGlVZ`nUO zeM4r9{^=%KRV7S|3v5;}kkVF-&NnU<>Cac)LzYLPa($GA>6UvYu%R+Y5;K=^LZuj) z->17e)lM0D{36Agme7mI0diy;fl_WHqd1f57QfvfPgn$zC1Sx!NQYEGN%<;o%5Hb; zSSjS9HVOsCGBeD%GDkf{H(jn_lkCfXyGh5&(1v2Tn7RR(=TlbBnj4CC3@@Gb8v^^; z0ol!k<8zxdY~DD&3WMuvm-h#UgQalF-O}-jttGauW!CdHwRY`aeQ%ZS>vy?+>O1t` zaFVr8O+D|v3Bp^`SFFeT%H7prMXwi0-`nGFHy&=CZoiZ(xHR(2nL#s3oCnoPCYGkG z&MO^?h+0U%HavuEGCm}Hm>T?KvQIAN?lwG)p>DH&@!8}Jy&h=O!M^^5B-1dHVI1w= z96@RG!F82Mct&6{y`kl z4fjqRihzmlV2EB(D9cx5taf#ku|!(4c8d8rU5AJZ+USp*aF@j2-ZY>~JQ+?LvQo$@ zYmn@`(qTlG))X~?Bhwo< zrB-xf>eBLy7jy0an(mI<7?RZWtBp#23V9#_&8WzRD0wL`YZ+5W0SZ|HX8X>3Kw+## z{Q-U`)K=IsjIz$VU_DVXhC~CLB|iNpk1mn_B*j5a-uZib%Gwb)J3JDe170*4b)&_a z6d-WOjuyM<;Y~Np9{bd+b|gyr3;(5IH)r2}QL^uY$OSc2ZYVA9Se-&r<;K{XN!mm6 zHNp+iQXQR|X*R}dE~?MX-M-xTWd=tZ+8{*=LpETLPC0C;Tcvdh%Zt#}TZr;Hr#F!; zBw-IgS6O5LPV>LHr4XKvJiQU*%st9b#fz->Pnh91Qr-0)QFFiHugEipeb+B-JlK3&;Sz0f; z&X9V(?ZBf6SW6<_9@l4v%dgY)P(bap-bfZK84Sl^f2lO2Z&V2pI5%No2kB$^C2jQ@ zq0zKr=cBTRDm@a{1K)Z?>-Eh7im>uf3UeEU` z@+1h8eWT5GoW$TJ+wsx+=hCOMC1%^wy`n>o%=!n=OAvm51lt(!or3i}QO%@&GY1Y2 zXVhx*ru=?tN+6CikUx+2Cr>CaZAGvCL1;>lwoc^H;d&NLKbbr~iMiAt@9xn-MMLJ^ z{H;cf&+GWqj(Rww)OD~jHl#&Nu@3E;?|~DLn>X^B_IBaS_zKRH>1EKc#R$u`s05~1 zXA`M#&QO-x-wNdhG*nQH7*IlBYCDT(H^v#G)BAn6M|I$YNtYu|uV*zoyRDT?tC)+W z2Vg^wlAxTL&M4?;NFZs)*gKw{LF0+s2^u?iiZv zEfT;oTvU@}-pfg@pI=kn+~?Oatf**RlYa@$)>DzlI!RF02<<;L<)GrCuIqf~5F?F}f0@9FQh&ZWd(lQAV# z(t1h5mJNM(F8LI~qS zhpNSgC^vMWLXGq!zSKUGDCgZ#fGA<&)a|W_+YY-g z|GQDgCRC|a*_n)@4l_hSnLxTk5Cdt+Rl`c%X)ki3fa4-2Pqyw$8C+|+n~8(uwRiac zLbQy{^>YlJE1(jw_p;k>Ty-H`$&!uE0p9PFEVO6Q&N+cxN4)30HHW67$ZHEgvlFHv z_N-Ow6{JhDbUU%w?zrIfkc4$b;SdHcvRof)kaPO9Eqhl}>;qy)n{wDm#iG9@%mQ>4 zlxblX5&aUq>xNVlAsQf>)g4qs+E-k3XY#o0+byRNr5q)+C7+tKM!piT;0=uyi1N!K zmx>sL5EsU@Oi&){e;(0>pD_`w{%C-o5$($PkhaAr7146WQ9WjeN52)5$pQ5X=h+j= z5^wNVOmDy%4S7mb%A2-P|Ebyw%iN7{cJRT}29M3H{|4oeMgaa|;A`nR6(?gTMA%z+8drX!}kuI~Wc)sgMs}0kx=3UJ0dp5w!C!_YS zM~gi(Cb_#h{e00d*R!gP@ml@p3rhppOdY*@FM7?!zcU{-bplYxYb>X`vyPrbb~T>1 zG82xc%-O}9nq*n(<=$Ujy(^fnmkMSnqkR$otsA1R4Wg=gAQ%m}3ZL6dM>x(rpoVKba;?K+)=0BdXkk1aO<+Mq}^8$A8=FbDax;T2{n zl2RKi!+njO80~U;%8sJRDAA@WsgW1>B!!LgaRrSQWS9L>haw}wa;Z2xj`qg3i=f6> zelx_)DLSeyFS#RTxIz_Nf0vDq*v$aA$?*O*zxhIsvob7Fz6og!rTzKBzPItKwC~$` z6=uDI|6-+{Ne8-1v#~V?2C*!;FlHPC>&ce&(W2J_r3hX{V5mu9$eil-JiuX~WF0GeMF6@OaKU6&Ea9gT?ENQkcqIf3>6)Vb8 zAmP#4dgysVaUimG)1M5K{l>#@DVfA}yglBVJU`_O!qLg`_U-Y@)6=7`R#slWe!cwq z>M{k=D;q4{$;yvE{h^!VBqlx{iDQH&pB)=iuJCX9LbWpiA8L|C^^a;MVKYfg zN0E5(eaaZS*M-4a)ohqKweU%nMl&XBtbr?Zyr!~{1Mif3llO77kdd=g5-V!0V8<-D zNa@XtoSEh~+zIM%?SJg&>F&Z?basrFU-lwPDY`k>%kBuq&LJmIW$~+-GIcKO35iIZ zUWhwl)&0`$;X@Ey=2|yM?!6`$Ug*@YO|C*?5{C~90-}bxx8=5OCnB_l!R25FPNVeM zwOe#LWPpv^kxh%#AU(=MPqjZJBpm$GKIvz8X?c(vCpa%Sjc@BXLfmS{G6<#0(q$ER zKy|+bDYdkb{NgOW+ErQ72>GW2quHNI%5q?vZynJ=^&KA{9(RN@wHtt(M+O%=sb%hF z^;)eLmxP~zHLD{D`f!GI%>;Z`wYT2Bz1dpxt(q^go$TlM=Wi5G{d8m5Tzw6;dbXSI zhDxur9`7r^o_?z!bLh(K`8j9blKgF7ybj6`Y|L6dg7yhtw9Qw`E z>%^Py+KF2(yTFK0mHur9G8McV0{RlD9)_%Ip2}Mx?!__!WGVLqNbHX2l^o32_2+D| zA1bZ-p2W9s9L_9!UdbuR@mZ@;-qlnk$hlZHTH{w6=(u@;YoP8550Vu>^3tTfs^1^c zh9mmn|AyXtNi=A9#UJDdT}GB|-5mWPS3Un6UaDX5%;Du_@vA;W4PLEb!lZ(Z7!<>r zoq7uKZv4<})|t74<^F-hDPONOI;3cZP77jxzM8+0$PQkt_o5zO1Y&@?%9B^v@0rPqq34c|A(5pr9< zQ+5RWCgq4gpCWcrbV=Qn_JB*e?l!eKUcQK9kJX*p`XZ`Gq|~@AK!9?=)pLUVhEuN> ze5h@`-V5mb!l3hu0y<+UbMz2I!_SqhFcS@1a-Cmr3@2{6(F>J}Sg&dpq(AyZDLcbc zUlYXdf(rRu>PT7`HF;1uvmSFDVrnAT?-z=Uq}mA`m(5Wi?5R)YU~k@b&5&N1S;2cnF{s7 zB3b4~#8V8Gy*+v;nzw+q7+9mD?blask6X2St*G5qoEw1THlEyRg)}wG7mecS6UUR5 z64B&CA@(&96DFZ|@(=$;a?s+!*H&7e`oDi%xJ2QPaxZydRXM#OLH;rrR-t;jdcznh z_5KiIaJ@ke)nR{>w3dIzK?WI`;?FwfBupLm!r~m1(m!sh;kuEJfJ&K)o2&yFBl0MF zYQOoZMFZ`_F8#4@?l-qATp|$a{Mq;EL;K>gTv1$kKHI64P?i3?_|p}F^8w1A(z`vN+RWHvkeYZmbL6W2R>hNma6qf-c4J{@jj zq_o7nZN{*(6BHCFENquCV=%>x6!i0;0A?VGTvcxs@$2|rE6r^S_EPPpZFY?-pH$ri zRkC;J19RLn(&0=lZ0$gFqdsy;62bG&0Dkz^uY&W+g$0wu>3NdhnI(kMG`0RGxra-QR0S{nWAshlgB9-x#`2XSkMO^*5Mdt&wE;-MG$WVG748rIieG zdsp#FQQzf4I3*{O>&Zgk*w9I^?p+ogTiUhs!vI7SBb3bI!n!O~?1k`N5^kVG#+#+k zSl$*@z9jHAgBLm{YviDz?hEV$wgO93F0nZ2Uzb^;)B74F#T?T08-MIh`+dX5tXIf) z6Ev!?%QuzMD*3X~YUr7wkZgPVjP$Q;7=N(=+a>LEndT^#S-B)T6PXhvB%kanodm2U zg!0kSlGu-=4jNC0N90uFe`LM&ua9aqF1Kv*^&sc&A`ounZ?7$VljEMortfLtwCf+o zx$4%;(tKH@Y&VH2pfDaj$lH$26?I58Q%ttLqn>MFH;>NDA} zk^(5gCT2PV{*o8iTp-v}A-b)U_SYG%Bz9Lgz%@{9HMt^cr#mf2d)T_Y+%t6Z@DN{2 zb0^_+`Ud}~O54lQO}**dv)xPEj%xc(pV#|47SnYHtEKK>nitz(5o-b8T58RZ8Ooty z;rh(Y-i!3Cq8AJT-p6`&fN^R;wS=*rYRst0=D`AH1S#nch@p$vWJ@PVqbZdI9!D-z zeIJthplR0{l^FCejk1av_n_{>Nk8xH#5+`gd~iVe8P7@8{9CGK>F?0_v~oygokH$y zuDxnShu|vKccwE{|DD9+Xp!!82Wqf-xI_5n8IAtF*6EgtRMiVz)4M2F+h?Q2BOJ5F zDZk0TKe_ZNXEc$_|LKgoTvtsKktIERwDk7kF#)64SXz6+ByvJCZ?)31fp@n#~DVhJ*sY_sNB8K5ND5YgyW-o+#H`l zjpBe}OrE=0>Nhi{bH3p#B-{>VF!RVSw24#MF85tn7QJL?4Bz6VBYC0c_N_VR??v_5 zkF#3*IGd_#Y;!1*527BE{Bs9>sM$qWXK(Xp5z5>JMN8r;o0CihVg9yAjLn#NttD|a zg?S%N&RU2ri}>DI92PrqUUzYiMtLtanP^`7ZUbL9`WmL#bo#lR*Y20R-%3|xUk#=f zL_Js8_|78#+B@sMA1a(CV6^qy+1_u@e%mE-gaR1oo9;Skisy)UtBl|Tf3)?twdc=@ z@!p&fe0o;HQJGP+JmxjiiK=HXU=+^;hR@roM#?Qy{V~mxl>tyP|4p1?8WU}5d`eNc z8SQ=W-){IXgC_R1qkZO!y*+VbsBh|Te z`qrDeiFjm6qLRJW&38HvB&M91K?VKPL&<6#w0jYO^F6p>uL*J*XmN!ZU~8V$$`t7W z7bM5(78y1s+$3bnXrFrmOK$RoitI83(W3R z4Y#o~K;4BeqIRKsN7D~*Dssn7+6)CXRo4 zumU*(eoUYM*g2&vKq$TA45Y*9SLV`fp0syoEe|WN9puPlcn9VKk^b=8z1~M6Mx{{` zsd-8!RE)th%6Bc&FqF=-Z|3;M&(odytuu9a$32~z82>s1_4Xo6#MI14RCyS`D;Q9o zSpx&DMz8Sa;$xkC3vbzEM|cYQ5Fxg?uj;1Gyv6T9^5zdepkk*w(WX^wB*QiSfdM=z z?$>q@46`hzhg$yd+Yno4R0hoT=GT0zbRCPk=cQxGxl!`5)S{}eI6grFz_E>Lu4wha zS*S?EW8O~ol-jB7Mgmsg^qCo0PIhkRcrh{z`K>>B?c?IesbepK@rX``*h4M+=;Oq5 zsoW-(MRWUEyF7G@3@aruX((#x7pGIE>?COrQgVXWi*g1T8ltxHl*zxNZu{YrWl`&G z2+&Xtnq-}w-_chGk(m=i3C}<*NW?^VBnRH?NR!CV8x35W3u38BEY-n074pLQ@85T4 zZ_n>^_TY!8Kt|_Ms3raX!`_=N$8ltNzTauacgTXQM6#OL7XqY&vdl!Hgl>xBC81K+ z$TVXKAPHg!AkkPvQr0gtk8{9EJq+GpqNGo$~$y64Q7=DzZSy|_B40DlnfSZ?i``qFYQspi}VT?gE3Td?$Skj zZ^fCM>+p(f{7HA0?_i*s|Fnk^%7<|NU1roYl+B<3yjo$KflLsK-4&o&qiUL98;k|& z-gaJ?-i#ruHRp3s50)nI>0r;wj+3677xC(nvn#E}WNQ^EOwXC1;Ah1wYn12gz2jV6 zJp}=fu1r#6XkZ;nhh0zKVOz#7?FSS=)U&2yJ3$}P3qTelm;uBJ6DmAEc4i7Uj1j3r zmoU$iO)m)(w2azcv3qGkr<6XHXUpQ&L-DFo8vKtm)LSQq*vKLHGA#&Od9$TNDnnkf zcE^Y!i$79(EPfgj((I%#?MAF}sn|D5ACCQ;bi48s5VlSZeArr+X9y8L6XVmV3EJ=- zB(xTmlm;S&;jZO)ppJqW{V0EAsmH~TZn4MEN;k}HzfPZBW~pOx_9LqK&DGZ+G}I4_ zQX@-$qW zf~(G-9L=Ldk^kZmD*48F_^s(KmPQlIU3)vPM!W4n(VPA8^0wEXzHjtme{cUE&HIoM zB(YQ@+MRo&)|g%QhBBJXVb9O5r|+sk!;P&$i#xNZs!eSLl@x{{WlwnS#~$@hE#?)& z!{22wFcy3bep=d(epdA`b1<1d+2j|5zAQ&Kr9Qpfz3Ygrcd;KSWk`&*EXu7TA6q9H zah7b%tf&jWFj;uq+uI51sZUL*R5xVJsUV4bWT9DKvI%+^Ub&m7c*BIwGcYPs*QBl%#lG7&5Kb7Il#MnMj(E~@U(B)V1agN*q?W@f3CTv z*C3rznju!fjH11mv$d84zA6@%3*u_WVd{4A)#(5DaqpGN7#$o9`bW>MBW)SjXPoVd}GZ)F4_xLS= z;5Lqdz1pPr8s3VTwa%1L_5{CWuqqLpSS4X{IAsYEVKuW}WTTc_d;5+=iWBx^WZq+| zV{_}yIH_yHPiwdZKGoKAY)hnz?SY$v{?UzzN+d)jS{a4yR;uXc+zsk;+woH<)seN4 zDAQ*drZoz|Dk4X7+h{BK!P|GkNp;u@Azsh8;jLQ{$bK7St-TP(P=Uo4{=xZurp3=9 zy%aT#7Qw~KocxlQ&-CFPmaFOvXpr=gAv|9tY(fptFQxkZanp~Ponqp1@L z-seB{2{9c-qfFA2T~mmr>DfYWFL5?O z`;b5RqWBi5X}9=F_#IVf2)*pF1i-O88rtu!jEwAsI zCE6Felb~H(D}jBtSpGm#5^V_x^7|m_ls=QJdQ=B%hWcytFnb+E22*f&1mOfUbf9M0 z+76e3O4OY|B2vUBBccT<3zDeXL3PA-*^U{m$4{A!V5Uk!S`ZyzGog2S^zx}dHSu}4 zxXzCjk=8D5a1R}#%9;GDc9574r>N4o%c>cXXgH;|xh9ZA`GosYqENX4W`2E58M`c> z7Gko^o3rZjUPPlzDkDj$H4D|K7XPUPvp^q>r^SAvA@uC4I53>2fK+TcYWf8Rl#tk! zj1C1MeP#yS_^dzBL2qLaU50%JTPv17`*;P2beAgm{D?X$R1)iy@YehHB1n3i>rZMU z&h6#1?NCNq_*?k*?y8*JLPEYUBsR~GD6woxhD7R_+_*w0I(vi9qM{K zRE3_7lhcKcy9tUWJf?h#;`DS(vLi$xn5P}4En9V_Y({k~uB2wAox*DfsuXZNf1y;Q zbJaruN~r}h$|f1i1tJ%^LS#xJc5Gx?RcC~tMu`eHWCvPrFwYWGP`D$7b7s(?YcP;N zIk%nXRe-htOOOM#0r(%#R^bfV!7A+|4MLqYZ|^7|C>oV+WAfFRQ5c^e5PRHvJAEIG zDvTc6nw$IRw(!;N-oK7^9`s&6suK#Cyvg?Fp0!UV z7tk$Ik0w#;Re-kES_@o9*i*QOZ11K^D*d=DQWtS7q_15COXY#GkRchoWx%^oZgWMJ z3S=T+9a$uS!i5nf#dtuR9X!p>s^kXdJ3gg9kgV;|zXl4~D5*7)j58K-Y9%_{Tx7aS z=}X0kA$3pZRPGI((m(x)GokrZBNfvD^+cG`j5OYqt~YjyEetLbP@}$)#338$!K&Mo zlRZB=K5SMIbvs@$=I`@@kxn7t!ATJ5(p}|A3d3tdS2JhHL6y^55m~$8+(WLLZ(U7-+ z#$oyLYb)Q!7L$&?>#||$PL`m01~7L?*`(~;IAX=HMuL*mtjdFutNk>-7!XW?hfg%T z!Y4z^EGIw>yo%GZQdNfW^=^{@q~4uKxW=Ue+GJc(jJ_ah(ys|U>^Tnnn`45fs!7sq z(mJq`tcGHjDm%vs==7R})F1)*lgiIE!v8EesE`ILst~w#yp~1cER?WvO(nb_iU`TQ zuiqk!+X&%X-&l(QT+)8RZg+zdybw5_8%Y`2M46yiq!ZY3|3-4(ZIaf^6R`^mZXtW` z`Q&&^se>r8?Aww(vP5expc_ONLI7=D(ao6;XPSN5oiufDtK{jvEn?fn24zwkCTkb+ z*0hc2LQb$R9$*jE7r!iLohFlzXU0(hDT&w{tFAEgEKsF1cpIy7r znARS)LT5wVm{BKIqK<#@e^fZM|6yLX|0P~h-ei_lel`yveh@W0p8RIVy)e9O4<;(l8 zSJZ(M@A2s|k?PlP&yX0XMPhky+~jrD%rjwab_tBK?Xkq)#2I`&7dCl@YtLQ;8}eYv zfzfNYw!|R%P#&q@oUJy_AkwDP3M2-&_91ttiovDT`1l=hM9J~hj46P_pPXN)y4bY$ z)rWp%Nj8sUZArHFgA7cP<cku@KIM}RMo~p9$ui>vxx@<%dPGSC&Y}$l8H|G}kjUbWQYrx*%iJe~ zwcOL{Y**9ovZDKSsRC>P#v~f+1hgSABcitB3=uqm+TO$FC_~MZ(9)`Hfkh z40vS8kt9dhu7ffq!AC=OE5=$YAjd6bLZs=T`&gQ8*pe`5UYTxnl%tzlp1lEc#s=7% z(BWI)A;A3R@`~IIGGy(15Gpje=&dFxHQ_))^H#!M+%y~fwh72AM?nA8g)+wTS{{D& zVP)wtXuG3-`ub-}?G7CTL@Vk3GGoLZi0z77_;lM0km@NGn%|w?qOHv{3J1CqxL{yH z#T+D_FQr3_g40wBTvYMWf5vkP*{6e|y}7y6bj`6MmDkXrm{bE_MOThat`6zLB&4+9 zEV2^)Gc=Yqfe;hOpKm~bVp??w|6&l|VLca<^i5ou@fb0+;7-h#>Aoo*V%2K{1B{NI zICO41fp*4B!0$XJhLXewcVdK z!c;l5pnf6;mz@959-EB=Wz~f}*ZioKTE7}cnLi6ro1 zfhA=MCLbAvw2;mRJKzy9{nG%cMN@nsFy3^!JST;h(gDXT2Pt>MTBjN-hCipuyI3Hu z`EJU;9Sx;Fv)w1;bLddjNqb*3Uwz9;wQa>xZOK%Y0twkG%yKy*-ZXI|MWzbpbGl3G z=jnT+9msH@+P{&31eb(RCiyM&{stRyLqlSHl`7a0-#s#;chsA+Miv~bXGvNPzm3yx z{nl?s`Ny4SVFcY3Mdk}@9+amWR7rQXC+o0^6?1q`Z_dtL670R7-q1jhT`UhwUfxM4PUD`k{_$2xL(;#^5r z!`pRfQRDFt2ZZ01>&_p&g0G}Nj=!b!;r)9bKK@1zCcbm4?&Ckp>i^LD*!wMH_&0tM zZG!;wPwOV}XnRNX9z&5M$CRUIefv80uj6L}|8@MVa~l2o_v}rBPfexoGxD4B=kl@6 z$=$yf&p?>qS0CaOzxog+sSiF){TL8_#)LUbn(_`Yi&6*l-++I2^r@UohP40K^#CD3 zwigEfIQzx0m5YW}!ibW$>pWOP&DbUcOgT9>8Yrdty7zu^dODAKNQVIU6Cry#7M-Vq zgU383oP2kLSQbYl#4_GipRX=-%GJs(Q#pbs0eFr z#B_B-sj8woidZFFOR+YEPua90nmem_D0TKPr)R+q9Xit2<<7O2a@Zdr5F}G_H0SyF zv8)%jDjKz6Se)%)DWDUfUlenpGEHxg!RjHOxZHy&{i}KiBed{lo)d&@LzHM zRDB6y;%*T+C4eC;Nrwhh_md> zNf}}z@peJJp}qOKQc3f7m8gw}?cdXk+ii%-G1XvAZd5Tt%`<&yc{vN4{o(AR5RCX= zD#oU9C|_Dls{8lA0q234*L9X)+mow+DvPe4&$%GxAPSPe9Q-0&|2bWfJ{&GO!HK_U zBrD~V-VQ|&)06xNRHP5e${Tz*E&CE;0?*H`orMtxb~mvmbMz!X zWIxNr_Vbf>Zbi1>4>e1dL=J7Gf;C^n4V}}r7Uu$ic%l zcMPfRk!}*$kBM8^U}dvNS;Z0(x`vSC!OjJnvI!FADAoOW25xofdOl9Uhe5>+>shwu#OS)#XFZ z^V3gVPrSrRQhpdF@u+j3*(VT}d?MiMuk|QPPx`%oPc)_TLBrE=feS5^^KdCQt9i9U za`}&C#ciL0=wryJ<-?WeV>Jeaf3`LmuTZx6ilx7MP@p9F+4o?fMiwi%)axePe;ix( z{Ztt;SkMN>%v*556$&e`BwMiEDLNwZm<$)JM(L`4A*qk$+zyTDe z1Ct;Gx~P{t1x-zF6Ss9#Fz6LPi-$W?{63x^Z{rnRQK|NnrGfVT+&j7iD+)~miN-j7 zFvI;Bzn3`QgS{YS^Lx1C=Re!C&wh?iJRg0}Qy7}2?|Kg-U!xA9g+j+tidQ!{V<*P) ztfHbz3~5Mv+l(RaFix}pCR@Ot=I4>s2#;KmNclh-iJZ6d*Hab5eIWVG3{-N$ATD`) zVQ*hnTr*um4WCbsEmfTyVs8ajXsTYwgTq-nadK!6#Lv!&dcayhkD|}5JE3O;k6Tu8AS6#>&ddl2u&mtGx^jXG;5@mTjfxuNk^)VEqa;>Jv6&+Dq ze6M=%&ZqCO@mj+TNWD#mb6LI#fx0kuCu9!ZYpuChPku*bagXK=nI}MWL8g3#BHb#J zdgCq1zN}tT!8pBB!zKf5VdYPKu|@Y&01_nvn(!+8e-=U&@y)Zc^le$F1@`s69?3mQ z`oyzPC|N(6eq+e^_T!bXiz;5iE>qwPAzy|cdI{}rHBUiF2$6_}DSPEdLgyOU$U!)j z(_MU?&}x_q@~NbL)$bc0(@&6sFO(;)x!o7_Rai^=*wKEU3duSP06etn^(a1D5KwcY z5^1gM8Qw;a#X{lyra(i(J;rkwh|#$1VPIVmjk@e>yo5SoAuc!&Y#$7-8)fOvQBH&_ zUC+uYNy)jkYLExOz!}q6MD}Cd<6t2aHinrewgl;x*oED3yEh_ZrOLH6r<20reY53d zBnZi6n64S#zPixi2BvXI=PqCt0$0@JTHN?zidvBlVQYM~J=Ecw+pb7oUOXehx(jv4 z4l~X{d=U}knNy9zA&E6sLrGyt$FeDhof0s-27 zSKNXJ5w9W}cJaY{t)VHr+-0!o*21Mn%Aq;8U1*e`377KXg|5Kn0`?jvts_xmFN!z4NX(Zu>tAG z>4+`(VT(|UqMK^Cz}r(r%A>T*U=s%WSihA-d`L~>{h2CKD5oy|>YY7jM zGel-!RV_inxS1$W8i?Xjk)rXOD%J?3aTb}uGGZ$@?Xim~3Jq5%if|vcgAqoWv^#Lq z7Q-GQIKrWU7#HLzq25bBqBa>z5|qkx{wrDBChCgH#h#@;-xSaQM~tsmQnm?c3K>XEzF8{3w`~gsykgnb#)f@UHoDlk z$-L=t?WmThz(He#XZCwSUpsihgRF3xDKkG~hv|c)sa+4~Qcj6kdc2j*g~GPt3rsjw zyb{Nv&@&^)^0!5F37g?o57CEQSY097AVjYrbN!YSK( z{lntkXxLj`+F0AjKg)Umd(p9sZfJ>3VpT_Em|6Rs@u74*>t*A4I!V*=aN48EuQa;r zpm?`9HPiIcY714d@M;Jw+>*RC;d|ktdEid|2fd${=C^+Nr{06)#6znH6`#I-dw4nJ zx^5ZIzv<8K`i;STmD0A}znQoE1wn6S7C3hU9`TuP=c)&Shr&kaX`+T1VO^-3>5_ud z*fT^JV?7>Dt#>*q*^laZlhnajB?l=o6k4(!t0f7%u6jfC!H6C(Qg;4_c}v+!ww#Wa zBBBt`8zS_+2^GgJwjo4Q1eZmuMjqVaM<>UU0d)*8-ShIR-pz&Uz8o^U?dPt;$kYRvtA=x3P}y&kAX4-T4AI? zgf9pz0;abdlUZAGQ9{70uBgv8k-vkCHUdYQH%QOtCupqZsx+sD-= zQ-ej=z(;p(*VS4?TkNjz!R~3hctxU&G(wKjcxf`}N--HH;B~~t3j`^uc<`@J1)`p5 zb60n(Z{z^6tot?UpJBLHSTh4`IQ zz&%OQTGwIuLMW6sH;L>ejD)Kwia@i}fUiLn0xPY4bE88c+JRgV< zB*I~RXDNAG(M(US7kaN@fE?RXQvDL77`)mOXY{qU?r3q#&@md)+P-jLg%h&L<*aDV zxzHpT4q7RwS)zGSCbC56n*!=UVAmK-vPs98;Fox{g&u68=+G<=hec&Xgp6n$oPzTw zZ#>b!%uwtAMU;+}h=8zKVzEUENJx?CHY66&rmEOIv`^_xmh~7Z;#Clbt-60SRQY*v z&|?71YnS1$z-WRH%-I<%O&FT2zA2WR=2}c2J)@>;n;ynnI|ctD)T+UAK8q7rsUrc+iqBc-~RK!fMwq;WBrUg!UfI^yZ#u6rRod2*lu`-n^#q_q5gb3ei2)R|hT!W6>}J0AtZwB1l7chlO?()QWG zP99QMqDUqg)7t}hZ1c^6RRq)QD>p>}GX$P(pTg?qIX~yRu&67&hi;`?9X_+PM!q;Mk!+GiZw0Z~6!wH;V-^H{C3}#9;c#v9$|0R%w=~ng-2m6Tp`5|T`BNeW zPoDvFVtmXTHuBJ)22pImvsleHfBM-HxPG?Cg@B`WsF;**M?{u6j8o6Y*DG@i3$X+A zpHj?F_^l{*@%kUH_ebCN=H6akUp!r0eEh8EgrhLo|Yl{ zqE`@pl2O2*v9aanMCRL2Bd|949Hw9Zse1ek)2W-u_oi+Ki^6iimtjEv zTbo%lupUlF-f;{VN)bNOXpEc80j2#S&XeDj6A-7H@&i1;-|@1xU%ZY5Q>$qOsR^lO~|`<~4OJ*s15t z^=dY()&960s~Rx9hxDaloqu4D3jhazq$@M9*MI-JDqKhVpM43_)m&m0N5Iy9uTw=v zK>^s?;raAAtu{gE#SV@7{G_ChcGq7+xjiP#+Y*XBko_=>2NWNq!6VN2yn>LFFkVBH zGS+*EXm71dhRY&9?Ox$&xST}ssyrfLzY1K!5}Ht{#L?TqR-fm%iVg@R*rxg0hgOEn z#i#qv#po%?lrii{%-#$-&dq%ms0T7i+gjOM#^9GmnRMX$S*VgDi`y zX@sCkVr{m~BJjqh(&SD^ey7(bgyj;dOXO#i+i(=GnyFQad9;I??j?F`%xI55CA0b& z?ij^ROv*zjj&cgE55Wu~|E!<>h>V*1(eFMM`_6gXl9k$0b2n_utSIR`&s2?-OV=m%+u@*IUhKJQ^iU2--E&4yd%w*6zZ zpS-Uaw3L(E;}?5VNXli5{L7nz`PcY#Z%B67SBMY)$M$_WVHxV6=jxr%IGQ-TqQXM1 zSRs+MF7jsf>A))LGZ#s3k~D9XSQ#Q8xcX6|BmI0uR5I*VwOl{ckQJ{$nGOd#>c`z- z9}WkFuL4<^d@BOMoRgC&4RSDZo|IorNhLq?}BeERNVHygBSZevYh4sESv_QydLpk-}vvdIwX|v!2D9zCVEkgp*X8 zTq0pBEZEi3lKCb?jO>JSmdzxvrD}T{aI4xAhykUOdLFH0ZQ{)of-)OFWuhHtTvz~! z@#u)-4NT1B2Y3!m9;hZA!WJ9{dwWs5BfKZ9BO}14@suqp0^pq@5kg}-waxozDxs(e z{eZRPG7u{yGpni#&*+%-7pk1O=`AK>h5sL4?S-%vnU5ew!I)Z2z!D0(^9euSWW3=y z7BR_!%p8j_G3&VmgKrK_s4N+}3thfBTv^@P;2@EfcOh$&Meerld9u2~J!m#z{&?wt z*B3EyKe72fiSt#3ZN)^JZB$|LAq>Y2uLox8^)Nm8s)W~75`yFu1TRmi(a%xf13e1K z!sL-c-inxkrTOxbaW(c>#gdNCKq4?Xq7vY26o>?y%w_*oaNTE#%*=jRc#%RZ=Z0`b z?{|`i)0v9tvOk{7h*;1KXs`O*d)i>Gs;?YYWenKjuMY9Xm<_oVV9Zn}3&#tr!=fk& zWfr8$lq$(=a;hkC)I9drCxRO3KS7B{s%$ySx5hoKsn1jzUyu0wYXhvNlx(PbUPWSI&j1E-<%3+1b zs1^i%`|h`g(g>Pd94Y%E+tZA6`U2bh8`kzklPjJWd3(}L-cln6c84=zt|(L#`reK& zg648KxwIpk)E2gDAK($POh(i6y)?A^8}!N(UzOD*=Xn41aBnZ4FqKX?Wq%DJVr`@R zIK^3sgqHq%9!m}NRupe6D`UEto z7U0_$Lma=MG4=L$-+i;FzbHOlyYsv6v~D3*5yuM-)&aa)oY#?jG-HWEZfZ^wf(!vp zTdlDJJ2L94_dx4pi%~Xw3Z0Gx2jiLUEIDh02Qn4~F8XL?feTHcZmLCi=pJUYBQGl9 zkZy}iGGnB0m4g3Sl3qHMLE_?t^*f@#<$swP3Ksf%&8cygs^)XpZhCrMp5Vsvvuw(s z1x0(NysQyQ$`G9VZKZstMOOcucjCvuRUfVu6D-6zliA7o_Lq68u_@>QSCqwV}b=qX>G_$ zfE?sdWIxLadpkLjeG=M@JjOm-5=~K`Mno>}XoY2paEFxTYMksbrZxt_RM{wbdlhQl zy6H^`Pr@lP!-ew4k{iGYf6jf~CK8-Yf6a~IYCmw|v~59SR!vhRvGaK>c83rp>p!4> zI=r8 zCkn<-x0;oJ^uIT~%B5@ev9@nl3m%D7V0|{5gc+b;7EvUXS-LW6>sjg4`MG^CTUY6$ zQG|hJoAxN0s!FS(?CCR(XuFiq=PPr|OV&G&KOsif!>gj!aR1)6m%}%m#G}!IMRZ#` zSc=D)CV#^HdJpGmW*O9-f!~~4VTFhq-xmoBDnbRH>J{JO8u>31h_F{TvHB5tMpb!($OryD$bwLRyp}fT8IAGR44TI5T3o$HOBwHXX-VMC(`0{iz0G=-w zT52k0`WEI82ps3`%PW-2OBPYipFrXIr>E$&1u~bxMbfP_qoRjRs57mTG|WQ6u_yu?eM{FX_Knve8}1M zy|HNjVosw!G8ra(d!5-dG31!-FJ-`NVT6YQRg~IxRm8jY3*O~`_~A_Im)ZA3KU6>H zQ|I;eZQ`&Z>P8n@SB(FsaW|~V2qJA7Wi*z#qQG|xs)wc;<1mcghKl8~v`N{FUk{HkVhpySASAd?{XPqkh&~0xGP&)TjQqCN&UB;YP?I6D1 zt6_@GCxFRz%Mq&V1R-f0Q7dZ7mdRp(GW5r#;q3lDgg>pDdtyDhCsar>n4HWCrh_JX z*z!iQd781y<&QYC(&W}nCVz+U^~Bq1N6o;6uS+Ubgakt~)5WZI1u`LWcvhiLIR2!4 zGp}bXw%`_c>=Ba8WY9OuO4=k@oE=F*{)^?NL47VM|Bq;cwap8ddNt8}GkEVe7|Fq& zNMW8U&-;>u>}Ex7^69lWIr@1(NC&msBmCwlj_Yt(wu6Y_K*qTr)cT_L{rK`7XAQP- zav&2kGQ)qM*0y0gvk`ZuPm$3N_wVht?9K!*Tlorezu8^A?`GaKzp}KnWOouDB0=*0 zy)Ik!kLM(sA5H8=ZF#sEyZ4jU(Ba-yZcZXIR&K%Wzl>h}(EDj`XaBq2iy!6(uSWfU z>Aim0ACB^2RnM;7(e7?+h9t9(*is-MGMN(?L)8<{(?Do7xjy6#=p|?D3MC2Ha(zn( z*251u)?{_HWK|3L248jBJf$RqB^DN>!L4S@Bo78=CE9bM$gP9P_4^6saSY%5Tm&St zxr0`bHKuSlH3tgqR|ADha}fKD?W={wMw~1`p`=h+$&B~OE@oyw8|ZL)e17s9b1;_f zHLD^>UG!y%nD#GK6x9jQ`|0H=O_H?PrgT`k3b?L@+UCe?kkdHHG(Ww{D*-Ugv(L~N zKi{6QW+!7Uo{T+^SfH_9-7e`Q-ja7Pt{*~$oe?QbiA4`BO%Y$i_j})uu+dX${~S9Nl@GeA1b*_EKnLMuK1WB{8Wn@kX6rPgc>20+SSX?9BvABIxYIR~pfQve zXDALuNKb@Z9n~}XHv*^s&GZ&%$%DDZujhvzr^2iB!S3tnu7B^%J(#O3LE#p~ap!v} zI?Qcd#rQdr} zs$3K0T#pXLKlSqb?QOS^T6 z>zy-;3y^0P>B`n{r2R|^?&>4z1Pj#Dq-9ac=Y42J>IMtSI85v$}80IvfjVW zFRd(X&M$9mJ!$gtFB%H6ZoGUw@p5E{Ai_w;R@{=wFl|0Af$3O*8DheUPV*9mff|Gw zhF6b@cJ_M~mJFrnlmB?Eh@}@6mvPnW-yoMM2eQF`Z&~T@P_@=A>SaAfy;Q*?qGqK4 z&B-hz31)}e+6s1gEQ%ToOBAqjX%ZN{S{jH*F@^#zqOn&r#%zj!A-`UNuHE7_#^vzz zEN`@7N5&6iI{7CBx8ik!$~PpzFqV65+xrw=ATh~0gOUhBjI7)dzv6$O^Rv;cFh~&V z&HBarte~ktLm+(GYvUX2w@zIoe8rkM7S}yv4v;`c9_KWCQB`({w>*7w?ghyoCK=oW zW9%w#V9o@9QD+g&RmPW`yP(?hbX~J5(#>hbsa#nMN98#nkl0yvA)%I1C{suwdJ@{A zVZ;czy^uSurq=U74%!PypJo=M*ZT~`@wgkbA94|qOqS{J0`hDW>S zrxRPs^~Fq0dXYmA?bMYq0v%9?Jlak85rPJV;&!u)h(tMz33~CHk}?;}HVU{&z$wWCK=Bhu6$JML*X_L+aURUSN#wW5&28HspIULqt$Z(UYaCJ$Dw?OD z=$1IDF7@U98rOBYlB0_PtxN(zw3U9*9VY+~LGMZQmI+BsH;G)-fu1!-2^6m^Jk z%v~%*BOBCY9FWoTbaKpu8OE;05mNrZ>6OA0Z(Ef&%2J6q#_L=LP=KIX;G=o`1c&@; zJLK5_A$)4^rI5L1f@^)MzUY>&5Z3YYXeCx3C&mc}f`>U*Ehz#Zf2vmlXg*QD38h7QFm|xg8~)={ zq*=Cg0}EZQqZ!?}D>5@sbn-L zN^Eys6oLxCm%Y(K0ko?Y$Vy9UOc$22bxq3!SPeTed(Z8)=)RwRZ|tRTC*W3x+UV)W z&qddwSg2TD44`lt_%Y`7K@fnI=jtfdu1;f51lABfV_Ad=w__w=tb%t{hk6DAhNu8D zm}`W9guYPwa(6teKxJjATC^{yQD0OrZghqnEA^@%uWg!8TAPd?8;p4km$z?WJl=Ow{w{GzPUb_9%164 zn&>%6WE`Lg$-3WMU)o?tL^r>O4Dr(n&rVPM^Rnt1Tpw|p8kdHPYjcfJ#6WCRqA9MY zXR%NZUI^%4dcJ%?V-~|0ui`f^GASn3c#D(DVzI#6W|7uTo-^ zJ+Qy(fe??P)ASiQ_^HP_&*NPXcgFiZeU!fxYGxo97-87U?ahidm~ky08i^M5}HV zlF)!uO4yp>1%(A^WYENIRjMzPt}z%lKc&=}cZWx1sY3VFjKeO3rAYMeYlz<}Ih zywEC8%V6?$dUH(1F-XCWF%C-Df}1in_YFRqR9i$p=FwDoIb#OU)!JmEv$_;T%@{^S z8wPDYnm(tRq9_&RIHUc-7pplPBQk z440uB5M~mbLK|!np5ERYSp!d^QkeZJKWT<6yK-@6sMIb1kiv|FLJ)_O(Fe0!fLPP^ zQIUlK<<|bN{Crl7d`HsY94P=pffaUuTIVj3$VdQ-(M0^3NUaL6j_#R&Mmw5E5DGcc zTchW1T-C#fGuM1QaA(G(+iQnHiAGf6qq4#&62M~d1w5~-qlx$=F^zVu1GqPu($oYb zSt{{KeSvltMnJYA0ZAZAXBdVU+fbEDXwIAkhB6u+tYe$5OctF5MUOj0;-*X~-BNi( zGZR0aV|Y*oPvp8v{}tQ1p$r9o5Ac~;!G7Ht#!4^nx87jk;-!ZUbEURx>n&=?@=)~T z>%ZZn6#vs{2uPEv*ihr>k|vUuCxja-tl#3_!1)-`bVgNHigsa~HdDdaBjV1F0fM`f z&*{5#b>#y9geYEEtcypX6rfQq0y88g(rbYU;)geuC3zr=9JZZ>-awxSL#P_9N;>0A z%lB(R3UY{uu0<;vV*;#Xs84j}aqw{-zMU#oj+F)YfP2557~cw1Ot1@7Dk812K~%XB zu|V*P;)M1eK^t@W6+#3>h7_yK(?^ptZi}1Qrf7w59!c45woGN}wa=#IK3tOc(vyC| z5hLlrdoAZ(i=339ADNT@7=?NzH~CHUp~hlKLKJs3NhCI(9_x+VLL{=o8yDr%-1Q`G zKz_4)RAcQ)@+*nR0X5+Iw?4$TG|l4OxfT0GCu3U-A+ZbF|L5zZj(GnpiKioQzt zy{)FRs7)Z)I^`lF>NXz_Y=IjlzMH$VQ+hYDnTSr59PcE(R}v>{4E2IU2&UAif{|K8 zY=Oj%s6g8$Yh=!(d=}FdC&dfNKRPvOzNk}Yr04w!nX6(QlG@cASh0LWPdEyA`z$&F zpHXXFjD;8Q|01tDCihH^ODCY!|4|!(FJvrRlkqRs>j)_qx${fv;_{CdT1|-c!)zci0v#c%69-ZDhnKba_f)T}hqq^7LC^TWU5Cz3}jp!mkpK7$|MB z>J0ftoeMOP7L}oNm*irv;FT9+7~Hy>(=9qA#hx3Fa{`ic zMn;AI-rYyU<*cZ(g-x}Yo8Za0_Svyo0YCv%LXwk|Puj-YV2g~aNlb*tWRlEl`e#)} z0btAuv9PcVKYB-tXXwy%z}+9Ts;4k*4M-I zWqN!irVx3Xt0|Jl&o6L7|E@ypzi4)r)vP6?ya=Esf?tLg#jY(}`kXlod2Qz{$jkZ|9!77~wTJ zLWET)oi2GJcJqkBy^rf>#O$nksmVT61VGr8YIdk%foy9?4T(q7r&bZQGE&drk-{J6 zgRpa2$?2oph$q{ths?riyns&-RBv&wjWj_sEW5c!<&uj{H(O#@En~?poPutm0P5?4 zo4(=o)!_vu?gb?X%oJ>9a_{pt|XYqyf|1un?<(b{6_*`p{3gr!mmC~$+hXlDzK1}3#F_+|PrKA>=?&oE5ScHHv_ z3%rP|g4#lFw_P3s-`lLwmaS~WYbvk`P9|HGLnqJ%))sGuk|i}qbx_iMB`CRR7N{VQ zZ>f4sF_g!%Hz(~ykR;|+(fEd0=W#oRGU|&Q9@U z3~9dIF)LVWOy{*@K&`du%m^j`6JYgxRI}kk!gfF=n@iia5an=N73`d{J?a@IGpHP? zA93DtAgQj!)ek1)P~2RG(9ly{vExt?l|77nRRwIIyj~efw#I8#7}VqL7)WO4FbG0| z@USfssCyDD4f2`#Me9c6?Jv{y&f->)9YxouJEEdzG<@?aM4tQFdD`(93xZ;?&Z8FsAE!u5LE(RIxv+lN0wR_#E9G zC@Vyu62=X}Rh{y9*-YI`TlMQ{I$;N@URalOFsry(0-7Mf@iNk-?c_DK`SvE4p;en$ z1~$$qnlsvgWDw{82;#xoYu%VDzld{;)lD@*IU%p)E=8_ba&7}}(SUK7RFyE?E)I%eqW02G!oELR3DInW99-!9?=CABH2>fTQKqF6viSo6^q$ zuPss5jx!+_i8Yz(47QxfcI6EO$ED$uJJt9g_03Gk?|XS1_mbFq-$`g!*h)kO7UGx1 zFF(D*5wR`ZrlrOyj$+mhvw~<(&S0(J)q5zYW<)5hAchD)B|TaKH@B|fOqpNthLUma zw1Z0Fx&^n+w1bZ*;;_qG0;iPRQmM+FA|0GH2y_rakxkQ)qXc)He%)B9e*r;?=Xkby zZp=1FnEU#x)vfv$?L_!xjbBR3a4>CK*a@at*@#1Qj&I@}C~txwmWm))`=K=hoae*3 z?BqixT98C4@uiAOWZDq#L0gi+LQaE^@i<7UKbydJg$dyVm_fkSU|1`ANU}tTZx;6b zC_=R*769RM&$f$rsjNj`rKL7S6#J)$%Ey^4wGc$#2S)6Aq zQIafLY(=(4lUyCKIOJFsu zi7R)%IkJb0E&+3dl?fm;YQ@Z3;(_n zQ(iL<)}c5|S(f|5(-TH1H%8~cym4CI+sP@yh516_#*s*TVWnEZqTmn|+mNsqZA`AA zMrgFRW_E$Qh{xrgln0}z0T*xGv20XS0M&jgFBTQnyL@*@w+XGS3uqN8&*b?3@EUs^{U-4{Pux*9Qoov$exuBS$s6b_@f zL0qp*H&9<$kQYsP5<}Jp{WYaW33Tbz0c7z)39c{Lx3t}cAr<#7dSl+cn$RrsOF2nb zRwQJutOOy`-)$|K?U?rj%;1wDUHu1-EwRcG{(Et5ISeVqm*e7yUu|LokrJEm#oO*| zS0^)6J!Q|F#AK2gyh0+R=OQ~IDIL_{p=xVM^+t?8kF?n5jGCy2Om=eI^qn%xGn9FH8e2YvvjJ1jWD5Z;2+&O>r}AS)W|mrg z8Z+v>!QOyc2_$S@u11&Dn-madMnCji(ZB`=5}{L@QT8p+F=^1E@2S5aWC2VG(tn`;!R-#GW_-WIJkRPog2n$^RZrYNt~73i{xT&{uP6Z6JocN zxzkrVC3k4860pIbAsTWX5{Zh~Adz-i_@TO=Uk|5VrdN3zA;Ltz8-EeHU}!WklixGG6c1%w(fIb5 zj^DgOCdh5#HD=4hHy^@^?Z~{KJ7jU`Wi0qo3J2TaEfPoKmUHghY%*iVgTO3`Ld73I zD4K+9iboGi0g~m_(=t3m)UytljnKN-E~eMMEt429*|yqz5a<&sTStkPQx`EoZAsZp zgu{Sa=o;1|0ywjyxNeZ*f}+GFxAESR#ILis-9XYT;v1}!{ih8@oGS?1Himx;9!Ze3 z7$j$!?eXw$H~8$051KRc*;N~%!g9j zVZNv*eZ|r$S7HtDX)fP2|hUly%;l*(&shlpLDC*Z@3ABcYvH7@(A`}w|?^4xx zamZxC0v2AL*g|78R2NXIQxU!F8Ka3`u}BtCmdJQ`31`nOh+33a&~VkSaaMyB7tELV zW9^7Roq3Kz)ViS~bg23p2 zFBK*-v>)S0WI7gYn{9$*;eL%!1Ym9h@?q47TpUqTesgk(8f?<2g?MwhEEZ#IPys(7 zNR;?^QbEkwivSb~$@2KvOqsmIdF|Dh9&I*lBe7D$^|C@T43;IC0_(LZ)1*;Jq5t<6 z78FWJFh#H_K9rBZES6yqC8*+yIUuc3VjxuVeHR1R>!X8APZ#@f~ z{n@nnZJSOWLT4eQO-Kc^5S3>1dNXjr5nnj1r3%aOrGZD#P@!`;9EEc>#&Hlkad~-S zO;Stjj+C=)pu8Htnasziuq=W7Xu(lZA`&aEq^*5s5uXyaR&lJg?!h8d(b1!k$XK)N z<7dj^&=Hj(9gd+W3FyZOmT@00(tIN?fvbu}HK?oo90?wQ?n#zf=sib95CkL1^l@te zE=y)MA*-hvAN0^O4+VaDwWa4B*j~@4n;#lDj4t-Ahx#DVc(db`EtW82m;AT$H zps*3~oEkmPD3a5dE>vv^cz-xKyE$X%(~}S!AcQlK!U6`Qc5%eLSyF1D)DY;U=VYT% zbo5B1nngrYh=J2ksE>>hP|?ULg59Jf$=2&9;jRK;+V6~n>&^I3kq&a#>oMl%b|KA) z!x;pgY`+A|&65Puy2VjFjw&lkVOsa^jYJq_03D1E-`!lzzZl5*q$?JQNF#b_0&+r{=%vkbb72k&&HKOxN(BUotpWo+ zVD>vsfakTlq8aeS9KwL*YeuDC7uUb~@FGa)U8QT?4DENu`-H*rDSjAp=!iBoYun^l z!Lf~{+6+0SwiziTK$u4tm&Lu&uxEXIIL1=qaKcRKi^q5m)!1i^G!KF-bzW}(3Ns0G z541#4|QMo?SnzG$@6mZ_LqQN^qr zaYKS^=C<{(`C3zGO*piUEpZ!Z)O0+-k(Rf=_s}HUD*~@lZO&#zPAmFccwu8@>@VfG zbOk!NE#saX97e#G8^V6%EwL6?tzxv)kno1cwkfC~5PM5WXh5n4(D-Z$GU~V{l!LlV zEV@$(fH0HrS>eHkd;Ic@ z(M?^UxNbCe83zz{>U>Ail>B#h0b2*c2y7L`KnXs9FICdof3ia?QrCs2ErH%5b2_`~S4=?rHw9h%-4}1m>Zjmq&`TFv+mLyI7gNbD`XC3M< ztBx34*5zN%mVnCPF+r>-#Vr@ELJGwNW<&Tl^ii&VMJFgfrj(ibVdX|aoz@THC2V2% zc~As{tY1jnvxT;YK1db{!)Q4F+edxIy^w7+D2E7RME+|C|tcMR9>av;t6bDJ9rxp^X zhnPVBczLB23!Yzz!Sk{}qPQtJfQGmx)_PkVtUTCQ@#11=H3R0Dq)Z~=<;Rp{O5e(` zqTYbOwP9~3Yy%sKAZzhE61Fj0`h5@ky}={ye8{b!laU<(`EyBCpD`k+J_Ft?t_HKk zREr5HP3Nd8V;xDgqKVly|9f!89o;c|4Dce0bt5h#6k0cMkM?v!@jW6lTw*$JN?fhql^j`$&2!h#={)=)cf5&WNV%zo~@e{=ZMMvl5g7a?kTU9zJ?^fM7jVzX5% zB!p@e;TJOHM3-U?O4Fz^7sv@xyrB@XqL)c>L#nvqMkkr>1Nc5`y}#aJxzV{+tl;z4 z>FQ;S`U$lg^V$GIi2wV2i>td~Bm?ZCx?F63bASmnMX;$HUkqLQlS0}CCkLk|(|D<} z*&f(Wm_sk#Qgtx~r@cx=07^DU zWzA=UtRNk$_ipm>+c|4+I{dEx>h);+F2pb9?>dG0*S&L`PY+cg=)on5q4!N_?HQ}h_6pDDIm@Eumy}ysZ^j2DO%H8 zbBevWzIeL0$iE9GS0@XTqnpK(^Ld#2}Bw$$6Ilo#=CJEZ8DE#5~Kg+3? zRJM)8wy5pU)~9WGwL4sXvau1H9uzSUCzI9&PyDyxN^8}0A*c)Dkg zZ}0JyYD*E&SWb@78A#d1!m>ZUdM9iBA?Bt6$noK)+;HqbmYTI^3{n1G(N7EiejW_f z71(=%`gb^3h|5NTL2_Rp?|o|>lF?cvme!5ct@-IGiT-YOS6)|G2)(rV@yj9Ks?5mkBUt6#UT~hs81tfL)W_C}Pw>CF%b||ZED1$J>74mEu zo~}Rna46#nrZqV!n)j{xN5t0N&wnEk``7U=Ph*uCL3w<7|K6vj^%&@GZq9)2>hi|M zZ1C#W|HSaM$u9%qttV_2Fs9`n8sA-DY>_aR;*HWA*$`S|6Fmn3<%G*<`1RS*`06cD z(fIVQd&^H5nv$aghe(Jm@p;z9w9w`Ly#~H$V`!k>jrQhS$e5n}4RjL#wZeg&g_^B* zHE}bKr}iDIhM^#qSid7AF&A&or>E27j~NtXsbKNH;wXGiXx9lIzWE=l$*I78)T=11 zxj`PqXL}Kf{Y#8Cisv9o)U8p2@fGC(Y`{~9j%tX%}V?0Jdra#q0 zWrGG;KaVz^1XH`5PK{mv^wGa<*Pm*>WJKpZSI}Q`(|f;v&-2kKH{H(LAKCLyuljM% zme#SKEJnvSY+w9Fz2%A4sh`v|l#uT0lbbVGp2c|O;o%zxst;cDmN<0F^Bc<>tE=;+Y_RAeYDZa? z^P5w8f_jd~{d;>m7zEkSIlC-Fk()o1afJWP*> zc~pM9hXPzqxTY2&2#UeC3-iDd^O4;7w@Tf{)qX=C*K<0fhuSHKiD11(sw_6C$A}bz z_mn-tQigvI>ns$QO0v%EIFLHdYbG7U?B2kTK@xlY-!S?%jm^{@ZCd z6nt9z@${4$*7WCiORcK@=^Lf#{e9ta?-h|klA_<95M~A2{AXecFNN2<*q@Jw0F0VYZN^a&eqO@k(^4tLY5_a*}>^ zxF7!bYVXlgZ}taod;O8H*bhG3YgGrQ4r8@qmjikaf0|ytlUEWo`dIYg8E8ht#09y4 zpb!{rLjUTflI2>wsz3ObNG!6NHn{ws$H#i*`TW0}Qh4Lb33Hym8kke_HNdV|T2I6o5t<@Zc~LJg%+|-(H@)VTX6es@O!lAhPlIV@{bQlS7V@ zmq%a+B#bDhgd27*C-gDCg!D?BAw6=o1{WFLyM>$clX(!>9$Yw@EGF%L@nS40y?AhW zGZFk2-)e>n7qvR~M+Qo|-Hff=`RUV}X~*ZM=@e5j66tDoB)m34a8)BDSV zv7<;$ic`z>aLDoSh@~z?P(&l{;s14A?BHIgYu|hGKQFU<^6s&YD>WXjd#js|dq2Fp zo*tmov)#*z{q;CA4%vZp%0uWieHC?@&)-dm7yEvp_s<->-zMjed*5AN9dp)S@ArqV z_52E-Atg^%=a)Cv(EaI!fN%S99u9vx79HTdh^BU{;<-(D?Y)rtueXTKQG_4+!Xb4G zAO-7Bp73IrQO^(*8NmeEA^@0#?#cX*=VSbH$2agHk-XV}(n|_4&9>cN>Z1Dc+EVY~ z3Iu0mWov$AeU%ySPC#A4Y&GXj)o(}~*}FjU{qy!eoBnBEq)5SrE+xF^bd}8`vTLVI@4xul@Q0nz>v&8|DtBv5 z@sdxQYm&0PICuYEHf`ncXPm9i7qkATZ3`6PP_d_MNIXwwfhm|LTR+RD)xk0XJBc7P z(@GnJEprkKsBKQ-_+UQy-PWYl&8HHw{*2X`MkG_A9%OT0y$FI(P?l^9A$-VJ)A2O{*X$8= zvty62VXmw^*@}BKaj!h#M@s*cC*Lhm0FN|ujrh{}te@L3E&Kwf=I7bi;Z8e8DmjmdN)-T{irlEHt>I@UD>cYb z<@_@thu+_&qP>+>)k~K+_eIcC=39(NvH1G$MNcNxL7t<*A4%CLbX(^su(VCB3%d2V zSCDSx!rPGa$tvC8KMP5Dq7&2_=|3H)nbcFukl?^W0%E8>_KcUmZpfe{#n1&$c;bT! zfu+8FM?|KC+yOG&_>vSWB-0M?Zgm-*gvn1f;Srw)-pwv{(X)}h9C)9YIrK?1hXN1z zvF_rx%8ZD$Ih9?;ihxBXhpE@3)t+z><0TNCzIlTN{kWi~3o|W@5NP?NN(j18>FyxM zw=YqY^Z%>Ps&3H!|Bk9`ZU0~FsM_3=XqJ%NzTU+}m2Af|yr=i?X(;(mH_WwHYQPxC zRC)z9Jvo^6$y7AW*g^p4cleR&?{vdRe!OU^cC(ySP&mS_WDS~=I`vSRu3SM$)4+nD z0ue|I^0(&UG_QGj-Y+9=&v4DQJb=w5yFs&4=7o+dFFjdX%a9G8CSAJ=c(bcw27L2) z@&K;f%>(!yGLwRXPR5J-Cud~rlT}9a#PuaXQ#TefkU{y~N8KV9Q~}KdDhRk&E^#;Z zonW=K!eB<6qrFE)!jOvTCdUu*8`>>@Xjc%NF%5^C@ggcc9hyhGT9nS47Xb6K7+vQfsP6^l5YY*+cT^qGl}N zcWDLELLCJZkFQ2srv7Y2^@meMcT@_@P{>Fv5xygKA4iJBJo!`l=ZJ{!f$~QaT12=o zp#bYd(rkWeNM+_rUR!Jf!nY_XUU&@am`W$_l?gll5*HO7Du2WGCx>JkPGzIVV?p%P zPf7v(X>w3FPi<35A+zPSOW|=T)=lvPm>(tH=5VzfR1kj3(QZmISK5NufShnC^2%(_ z3jVkL3^JeUXG|*Q;k^n`JCE}N=r3cXo!ZMOj^bT+i{E<>J(q`AZnPas6;;kBwa&(2 z^zrEhPJ#(W@1iEUh4bj$&#~%|f*$Uv^>wRM{lyIUo3IC}*T?>gq@I$;Oa7M9md6}P z_dSKM_(vpCA5CyJ5n70~J^?Ve<{)1rkO;i_mjDU8G6@vKni}LHJV-#9sbZr9CFHz$En!vFQbc5bxIcP@U7ACAOg)W*b2_fZpv{EM=xwPL7(uhg1fpE@ zE)L(XemjTHpAT!_&dFfTg9tmRa_)0WOSj7^HSAxpmGk+VOZ`)%P!eP@(fj7M<0nfx zCAzx23nf18KXOI>5)xE3D05iUB$|_n!=~xP3&n&;%w1YqTP8PmX=%OxOYh&mK0mm+ zcxIP3{PL5=<>8Xuy|eMl{d?u@JDYZWccUI)Yigqx$lu4?bM2FAsO@?%k!v;Jg0P-SztRT^oF5`AKzoW!(l}>DN

    >TF`=PAB$8t?t&(e37k%yo!%Mk+*&K(k)DzoNqRO3f~sHCH^ui zbSAM*jRo<;pYf2b9@jNi`D22^$!)R`>5BnQ4_T)f10Zlqrmm5F>FpLF>L$Wivl|&D z<@c!-LYJXWL{y&|p8B(S{m!?{%Y}TXmfE0r`YcTwV1$Vu`_g-V17;_eRN!odM$YV# zQxvq&2El)t`N{Y|&%x$>ud`(i0Vb=yXW{4P_mGi4z)CnX;B;rVz|iSyZ~ynTIIRDj zWayUq@0MEm@c*U*^z2~T==XMl?e8BkwsAL~YNvQS)yd!AN-J{IoPQqU@t)ahcR;sL ze?jgB_Ala}6cvNt9s035^4O?MuG>>^3_*9>LGCKEtwXJwts98iL4!Z_!Gu?kNcC&L zD}VMlL8t3zSN_1NxBYMg3UBz9<1eBl5r`Ru2WElow2mqB+NzPR#%Pgz#dZuUs>EMSs@#Qtrz#631VmX2_W8Yr4bQcV(A zq{U4q`&hpEAU^TePcY@)8mq8<|4R2|%e#|o7nhc=#=&f zqXwzVr=@DVFAduIuZe*PH3iSEyS@j4h$9hjT?79fl!wxVoloe`?vuB`4dpM}PZQsZ z2EwV)Q^I~oYU~b@)Y!0H;F4BZKvj^94+2^O5;Io|{cU!nje zSLwpzduvH2^4=f8@c*6Z|4H&`dB`lk;;FT`UU3zrB>^8_HU3x}!~Ppsr`?cg*JAIN zvgCa*UVPBsFmw7sE=ENC>gMN%h8dyImHFo|`JrKw&+RCan-knnPVQLxyYqKpUw)ZvT0=mo3p!{Z+Bx%Pr~@rJrR5CSATT0av|1Gm0Bvy7}H zIXl(2Ff8%LO92k?8SP#Ke2=fqrJ0$EH9o4uUCY3Gp#;8=tk6Mfp>B$_OEJAOxo)c? z0=m-Q>~z=v87#Aci?6MDNzd%Ed$Ad8#uX@Ah>#j|(nh5D7Y>ywlnlOZ{^&+es- zm2X+kpNQ`dJek@z6_-sV38R%aNWE#-q3*V&v|NcXfqhqUM$r?3I4HIBc>e7K@B+J@?a~&UwQwe-1kii zeh|@ldbBXC-FRz3e>l|4AVok=b}{0C90G6#widqY#Y7*<9N%y>y{bSrTA}m1q5jZ6 zAyet_#!2i`a>6@Q@?^I!Qfg=i@_WA6muZP4>|{nJu~;^`Am3-2ci-f(q`J7Rt$jmF z7EX%C24j|;UVUGco;I@jW_4*PeRX5`JAqrp``Y&hM)kEizjJdZ9rL5_mOpu*aj59S zaYU5=nf0(G2iJI*vT+i|IL#{K)uhZexg3AA+Y^(l_QA|}lmZX!d&KH=9uF>qg^g!S z1V`2eTYk)h$(Q052;BR5eD&k&tDE;xMOrK))end@4!=Ht9^0OM06kWJ^`j^b>;_>^ zy3^h8-!<7MlH2*#325rqJ9q~VT1lXs~rN+TS^TiHl+u&%eV*NfkT>sUZTAiNi zS@wvR7d^CF=4kBJR9up*nrCJ#))+D%<;CLfpMbSW$nk2gSOq#^k?guWSN>+I9WC+RC)Yx4_k&|AZ;t0H?%UT$eh|;- z=@+sN@bl44H`!*?lgTtH`|kuN+&LOz!>qqG6Hd`?s4C-tp^gufr*FU*{jqRhRd7bl zX93OZw<*GG3G@S`_8+Bp8q{3p&lkCH<<71y@75?s(Ul*1`d#e3XOB()`>{fUY@6$E zGlWy*|9I??^0fZhV_&k!AWTW%8q_+B-{__@5_uEapZb04SavKlQqDhj;n)9xr<$j& z-G1-W8`Du^;F3&z`ac$A*Gx!!CyRqwP5qpgu)K<0cjU1G>~YFCKD0Oc`H=snKQPEw z-i}c53Y3}no`Dw~0%m~GU*;xQwt1{g7cKnEaco`AdcoYup7yT3zRPbur3lNV<;a`I z&x?{akjF|euysMxAd&y33(Rf^SUPGY!AaIfex%1m-?H~xawKIDYr7gat+&7?gUc|6 zVg-1T(57?O$e_doNbIZ7x}`V9{veB?@aE!s{inUP25zMb>5g_? zvfuAA$6jx!a`9KOja3A@`89Y=18bH?YWA_@c5Pz(!r8u?0{f2ibl(Ngf)BXq zkCiAa9CfgisoL@7e9C^LLC6v4g>o^k(&jDPvioIVR3`-ci89=D;4MQ0Hzpe4X78DvPp@Kbq}J zXVfH}}`J;N#gVQJo1brbmk#m&vxt$s%A+Qzr1r@Rw{N|rOmLKR|pvQ zb>(fS(42=Wy?~DR&Aj;caGAy|uz)d6f@8c~VAy#T*?VWhMuB19J{@+WtiQ|zc8Cro zJNz}tMyHuy0{_DIXD;IwTV3}%zDChO&&EC?swIz~rG8D~WKrJrx0Ul)j{mhQfJ{~c zZ-lIz>@lm8@tKvQ#e<>fH1%~XaKIGuo7Edm9-R;v=-H0G<{ot6_j+c-j{$WbfghZ8 z0=^*wckhCa#^x)i7`cLtmx^bjaU+-2(d6z4{W+bF%A2Q$MiL1QO~mP;@ejfZsKF)5 z=T>Fpqcg|%J`6SZwfybXrNorMB3_?S$GFmH$5i}{r*6)*B81Qq))wBY0H1%zLp`I}#xWMBI< zSS4VIycprv3@gDO^4Jj3!g}lo@V;G-ZA5Mer~sz4W87-jZ?_qlpzHuS6y_Qc5yC%x*!*>1V+b7=h9FD5#>la?Ikx#yfL!l1U%ngdMC z_ep0yz`w_NRj!!EyA?c$x1H^Km+oNQ`Rdr0eyysmVD+fKCSndKJ*7P+C2ehuVJ`qmG{I-qg3n30mbxfuu=X0gN<@7 zk;4bC93-e{hF#(I!up=jsKv%M_d~-zT@T zYzGyLTkHaA6B>-cUPR`f=L^j06XZ4-+tStc(zDiFiZkygv}PFgN-wiMyT))XC)msw zZsrJR5}fP@4zixdA1(&bY3w9rf01$}xAy{nu%A7gV}(8cLDhqdTgW|UOn>L-gD%J3 zaolD4X=mHK)3ndvYG0;JkDIX7uwJnMxg#?oK6HS3vPYb-LT6awgpk@jhTo7I)i}4k zyN6L4%Gbw5JO)G=*UL(}UAsN5&Ti{vb^}ha1w{G#Zg}&{nZb#7LJR65-Fqwd>=p~O zs!S?^y_FkC&UjOaQ;~)i7x-Xi@6&C(Ll~WUnY4bIcsc{o(2*U#rSjin&DV|;(#>%D zF@X>(1m;A%QDJ>=OEy(z*q>9*#(EiBWTepxykt4_gEn=qegc5h^d{I(h1+;7!=1k!X!2GR^b|RpGUB)p2ktkrskHF(Jj>zMV43C*}QCk0xfbnrT zU76#oo-o13r=41 zBAKs8e!vAjxUIeYZw5C+Y>7G6uusl&{Z$W0La~|E+rX(12Dh+BK&|=tbAP(MjP?vV zbW4zUSql|-E=iw$ z+p8{d0kNcOUu~->YtQHmXM$S!2#As6^Mv!n9s233zg_p4Ktow79m;G?-%;S8WH9-f(A@zR0q(#UK%$~_+V{Auh3{R}Svh6#y6Cv~STIWRAFmVE)jBKBG z9w^y|ci98-cYm4?b=SD&Nq6uEA8xT#+Y#w0-*7ojt|}JjFpN*zl<_D1NWI*ucrQ5rC0N;8di_y#!%Q z9ABf$bIbH9*iY^){1UYJvqAbeVwKv}ZVKqRM;9i{S$OG;Bfr>XIN}T?B%4$JpXzF1S8g?)n@idZAm9uPpf+RiRIKeA^oI=XoY&CHH0eEb-ez8b-U*{n8Xe5**mbaQ zz~d*>%|}Xg_B#%pIYTs7j+O0yahdp%SsotPPc5qlq!WD-!GPKd_hHa#bgEQ z?8bZ!SO-CmBasnjxyqe~)Wi&B*Oi`+{&L^54)L3%$4iqwdJ><83s|83-%(SpUl&tmhgI5NJ~uTrYkWR%0Oz zR{50)3(r1lh>bqxt{BIc+>?+=Vec;i@dMpBzfRM?YNd&R$(1}%6{b<@c;6XnRPt3O z5_rp@8bxzygb;jG`PFns27|_+`tJ65krQex8T@z`0l5Jl4!r!XZ!tBQjy>#6co&^5 zzRO48AE}ZlyzaTs4KW~UbNz49Ix4d5LJN;?f z6Y|{c8?@xyRf?>^F8|`Qq#EUn*TEb(rN=yTK|q-IRylqO8Y@{}jzXEYehv+>9?a55 zY4?tos<+eP)_3i0ubM{qYlU2nRp2(-F}SrFYkAiMaH9eM)99yflPSVcW^GO;<^t6q zZ~_57cvb2a4eiybZAbyL-S=lgO7*W#jy>VR9>X-m8Ycep6dNnTtk1}(>&LXb(-W01@p4D(^IU;PhEy!Yo9fpd|h zM#6L#A@95N1;5;Y!8I4R|*+BrIRefg#5NjQNP)JE60BCa&I6FW}%DX}&E~ zf%0n5Rr(_3@&q1$V*TNnGhUkhRXdH=Yov>a89hN|1LrMWa-^BYj|@I+N=fP zNS)y?H7BC%Rrvf?Y`DK*cq)z>xH48br5ulz6#y08kDbjdl<@ioCXV~ znXdBF7#?IcHk9s3K6;i!cpa{lYZ%-&S0UNOp{f9vR=? zOSu;uN_%H7#nmM+6bvS-Y7oy?E*kWQic{n6tfh1p8P3_D3+Oxs`ew>OjfN%^3>AmG zygU&TvY=t&3Ps~wNei`lZuIO)nBpJ37@N(#{D)8zVNJzzj=SZ(V1|3<7+?DjuWvM> zVRg22>pZ#!Iqx=J_0Tvneer=?)C1Mn^0*P|oo8%Y%+IfWu_%dlDGiBG@l^pl za$>aNV#9h~%(;}u&kHzsmFIn*Xlxv)2Q`|Z%Mp3@w-A^na`kQjCWzwxGElF!=y#~j zLuV7JUi@S1?QMQ4`?0#_&n+$AwvDM&2R2bRN`)=9gc{O0Wgc>D8>B zZllcxx_+OsfV;HzOKNtfP3CWB`+{Hja0XBTi*x7!G|26!(F1}X5?I`my?t_9De^c1 z$gTrmwEPbk|8E6j+&AW264#we^sJ207~sM28H~FgWBvjS=yTj2sqOUAyAC7v*pK-~ zw^3BQD`Nz_Q#X!{qc3yx8jRjZJYkS`DDBRd&G(_lZ@z-|be}h+z0u&247;yYoV{d4 zJexyz(z;VRXSN%d%_~8SYr>uBh)UllB_rBzVt+)BBBsGGkwuZ>W+swS^(l~rhBeT#}!A% zbMCa57siSnoEZViPaYg%e0;@gS^nv9?*su>J3PhFe#s+VT^ji%{vin7>I~ULkFyS5 ziCFIte*E{HpTP+$eHyzHKMGv&ii)9q3M0pH>Q>{mY1jT3K!^*OPsfV;J(%(!RA}h$ z2!fOFfz|QH>=y*&flpn$v7ci*K4|x6iZ66wxBvHp2$g=m1{w;Myq{bUPsc;%PFiQ= zPz6`r(D!HaN{ojUVXJo25fTbQ*o2&@U@eWi$(*fOKxxnFk;X@a&7UeDr2b(IUG&)|4G5--otm zX!cgIbsb?u4(^>h{D$$cAwaA($Lm3i-0ZUlo!2P7*5R^>W1I-v##{#_)L1W%P39!} z16$VneO#_E&eX0;fvbK&ZZn9>n){^W+ndklyDyX45qx=#PF4`$28bl6t6n{RWcJz5 zPDkpByuFvZ78po9q!-F}v)|-i@~X5~DMb)~_$QkY5ce@^iKoycjnJPd<;b zyOKZVJ+SV_r5>+z&^_$KS!4?SRk`}Bi&OA#AIJdH=5txwwn=}uDu3Zdb*_e>o6-eG zFCjN^g_P!)Bc-@*tm8n6=Ht*l&W#VZt4wD>BVs28gskGG-}lz`SbUw~A8S;Vb?mD! z^XG0&ake)vUKwxgS=Fmwl_f50lsPXnX6m-4_Nr`G4HaJJvDrY3 zM6%a@!7Ji2`=WGdPxo8p1H|egUi~xok8i9k*%l6t=2W~A9&{7zeb;+&)p77hVW9hL zidK@$J+9>T+LvMwHW1OKY0PhIntz??qLqnq6e=EM4RK#ojN_ z8}Gz>M+?5cb)5@F$!_X2q1R7ImDvosIg-n=R2J=qel8{za^8@8@MX-Ix7u;0Coi$j zcGEd~nRjh{x7}8~SiOggWAJkZCZ$({!qSkIUq1-f;nPA3vEM>VUr$lbMMAtX4< ztK8PYkd=k8-CJ$!4ddrah)0in#Wmr(no?{!o>`--mvUviu zSpI%5$QXCpMMGb#dZ%Kf0}ql59+(Ijx+m`t?L*<{yg-*0<4-Bb%ho zn4{D(3BOzgtioZX^vU~R9aoMevF^7rSN=}JAxSR?-HYn!g& z%l*H!82bz(&RB%Z8&nM5kdKp{e4Amq?71Z(riT)2yxYSgQw(Rnz%U~w#Ot7@MVK3J z3bmZ`#F*(Bku~X50|;6csmk@id}*HNmiby#_rEukjX0~^v^B9$qVTzjXK>wBzyDRzG($-Ga z%~Oru8iY;L`KI~-o9>cZhyz3Hs*UG@u=7kK%5?2yamabGbA#B~HR2tH>A}X0s{xhM z$EOMwRqV(^ADv@W73in+uJFdojuJ>FtC`(q94oa9Ve~(r&`)ohnwaf#TH2Ure+bLuNx!k ze)QVdJepa$KU@7}dK;-Mj!v}f%Ih41@Y8ZBuSdL_D}%N+30DX<`kiLplFBvJog7Ez zHr58mzrIEfy0$5E>+Gm^A5p{e!JSq?VB63#dhQViH*v}vD}h%(52s&dE+82S)bq4 zASU9R`YL(yAQux_HZ@LVyTfR*R*-ea{D@%AOG2ULMBe}@n0^{N*2RMVjri2FjVTm; z^qO<*!-5>X9`+*nGZlN%6cb~P9ifUfPz8)NUY?fn&vo)DG`3xl`Bn4XwDN#$`N6~h z#?EAyLJPa`+rG8&<*wm#*S1ikiP*94p8ZGcxlWu?onktS!&9p5J+=aE`vUAc+-Jm1&7{9wq zeTgDp`0=kldpCuHD_507rH1Ldor)`yJ4sB}$R#GX}rV} zUXznv66;=c7mP(5yp;4yrT_Z;?w#%A#+PhPeYXcD-`6hmXJ1-#yXrO=gXwqDV15d! zAL9M{_B?#~@uG55|Bh*kE3Owg6Y{_c|6c&`l#`d-`uE)>Kbt(mzx{ez;ci;l?|P-Q z5e3fJHRYqa@p&rl35l7315QR#Wd88Q%gw7Rm3DQ7#JboFZT-X+r_;TN8J}5n`6j7R z#>Rc!RzhiQC`M+i%s4=$BV}M{KKpL0 zQFz4~ukvN3*;E$4{b`kF&GQW;tfN9IR_+yHHWJRyy}O*^DH`2V6gsj>4i?pS46^7v zu*axrT?HE}tve7QuE5=yB@(Ec7&%))DDXeF2IcBJ2PG4nknzQBnz)9FiaBzDvhu!lY*qL`CCDWo10FW*%xVX<>L;_WYxP3 ztnLz5ayk%+b~5WgoL2kAIIQ|YnJDcr7uECoxjsLZ?6ms*g13fTZjV;mS$$bNdiEKI z(Ih`>ns0^QF#!s?Mgh8x+=y(E3(Ix%k~}6ionJpz`FwA!l%mcsdCSy?cwEkSInb_$ z3H9c~M!ZN)P$x8aJU+U9+BkEYTLe#QMwn#8dRn{78s3M@)#u^6-*$7|A61jWbfC_N z)LHv+BHG6KM;cA}4j8%UB)f54w1gxoG22jpGKigbI88?uGiwTIvw7bVwP;K5(Jiv2KO%Zm6n_}qZSIp6eY8 z4P@fgDwhKj1)}9!L8NSyW?Yo#%xSxCF|Kt=mS@(4jNI?L8y;EPoV5wKkh~+xHQJG| z6D8Cqd)Xqj@jSKxV>o#4n{zv^C(>yXcXzryiWV3B)~wRr$FHd%T9)^!g%Anfq$;5>Ont^MjI-I&bB%qC^4N>TlAa4yGSXVQgxc`QvY#ot4usn)F5AeVEwnT zGjTNpuOebB7I`L3%ysr%LHKIw%>uLK=?}m?87|l3*`Ahx(K7dB83~-Q|L_QrBraMVYl*`cQ~Q{QHLqo*#7Gg#4e4ZblrA=0xA-x$jjcjwXR3~_5ey#~xiuay1h zI#IElh)L?N6@RP}&cThzqIau+2d5)&g+lYQgO7``-8(S&YW;Jss> z;6JagW#A8SX}riknP$xkW!SYskyFlPe9g@5z1BtHQ4fY_vY8i`+f}oIRtGCysClk8 z?i3%d6>4O%$;rL^gY*hkyH~!jm#L=I=|_}(fze*E*DRT;I&Qzege{cd0%UAw zydeuSm#1QiP_cG=?PI~9n3qZ$p1rgUoTdm@t#K@YKWyr_df2D8eY2aW-R!Pd-!&P(VB{C|> zGgel7sRrNe*hs*pVw@)mnn&F&&-|pdtXMY6Z29P^gCZN;-5$2gq>h{L%`Vd@x|ioX zeeTO&?Sq?3nk7rwrHwq}#jz;qn?6=}LV!l)uP0n%&vvpmlr0dvn?eWNdz*|5a;HPml`-upf&Ur zt$I=R3*xU<*PIs{m{43|Ylap&ZRrkHZI>g?S{x=@5Q{G7Yo8S?GjiLaV;9vniQ62? z$f@gZmK92-YX@k}7WZ-A&PF*zTPfd{0GGPSlpp7~u#6mUhkMu7Mr^tHaWx_*`qjoQ z%ZRXvL02P(g=qaI-sAm61L(pmB78XRuY!~yVUzSp(3kMB7SVxCB9%_R zevIEJ+cAxvHb`}5Bdt10q?Vy$oOSrlgrC!y%MViOm1SpF2n84_6n}pQ&*Ou@l@tl z^rLltt`|Xios-kWKi>CS-5qQ&-|BY?+Z`w;N~%bX)p`H$S=56@$?=pVx?cmm5j{z< z?S+rVTd&Dk%Gxm{h`?FZ7ZmkTR?(%m@H^urUlfHc(BfRrZ@f+H#7(~;3VB*xmLlD= zjTRq$TD}(1IKyVT`t{SzOFQa8?Gziqx?TO@IGGrI4tbkXK;#z(@yX!bL5 zii-2|ffLK%Sa92*zc58StR{By(=SGWTc8}~9s%nH{+Hw9Mu!8<(BOZo89I8|xUSne zFJwKlyOM(yU!R;244X|?EwX6q*<~swStvdnENu6sD=+{4+OplRCxZE%?Q}V1hjQG0 zzuX_%@0lZer@6es8)x6JWPYQSCAWrK((R}7MC46Q&NMmb9A{Ub7+7l-X?;8pYmfVS zMpy+;aGW9gBt+^h+S_y!aH~^Ee54ykR@p|DN+xG9?ltkgb*I;6EsHu>H$k$Sw7$zp zx>Wz#E^I8r#_XZ4gX}V$FB59<+o0!)DVF(Vo~pu*>9BPWG9-0TA z+nx^jgOtec_d^g74~uwF?F>9#x}VTJDNGCyzdbKOG=c)g8rS;CzLcGbF(PU%qs7?0UDdtX5JE8W|TQ$NmBmTAUW556x{UK~H_ zg?b`J9_%ync!}pSI@FQXGRdCxaufaKId57l?)PoGhlBoriKysetj*K6iD@o#?aJBv z#gp1w`rOw-$Ue1Mod<+2F)(wuUrPLzdL7EROU|q|4`4uFqAhDg|?qh>McHq4)U*2^A-xB#HIr6pMGWh6KA(5HR zpi?q8i0P8nZj-m2#}WgR6oQW~)O2WHl_dEjE#L8diS}FlTF_j_D1|$)W)?Cf*gNZ7 zOHO%h?871vpYJXmZ8kPc#0I;@xDL>#a^Ko3s77QPMu#o)Sh|D>4MYj>du|j8YfP22 z&Dx>2PotaH@e|C!N9x1sZ+fe~EO#8}9*`z?EUS@RvrzjSY9WMak4DYhA~6NtwCt=A z_T@QIO(vygEb0Z6ephOn_pn{Mm#&>U(~frLZRmNggD0p|CDoo1Utd_^o$ZG<_`aX2$5cXTVED3BF98uKi(Hs*#KPME%6?s+}U#F^W(ZPts8%5OG0LxdNCCH-8Xvu}8ygr|h%>p76O z53D<3Bq-0B>qKF*Gw+JHaOburjZFf12|XF&6>uK}yh=-gW+4Q!ue$trpV-lIUl3=} zk^e$l$G*e!{tMiV)wDKM3HxM{EF_`055(vx{wn?PqAgi5Oq9^mvA|E#+%BXSD?0hd zSCPFeY?CMeP#3AV(EmzR@!rHQF3SNAckeIQCFQ5jcksqWp!%^nMqnS}wsbph$apIn z3W;Qqbfb&&WbfxW2P-Rhro7g{p_&Ot$?V*Od%APnr!VTAfBeIYj(<~7?HA3<>NqEQGc&i>T=K zkS=ZQj=nJQn|P~yDPef^V5h(NR!eCySvX)dId+-62A+6#h2dKuov3#AZv66N`?w!V zmLSf9mELu-r731q96L!d1V4ihP7sSfE#eWxdrL0e;a!o6`}~OK#5H?Mj$)sNDb2D` zmy^+AhQFdEJn?7Jw6*ml??*^HY!|@NkyoC3gj#gGRk~*By20hEz&oKYn8~HH^;%aj z3dhYm`>i{1>A7tH+Au)nf@Z0fvhz5BB%abK{FS6gmUM_nRM2{FO*97>jp z`Kqo{h{X?_0$i8Ei5P(haQfGE9+-WWf7By8US?Q1c=<63m0Hf&9(_ZH5}oqGU_b|< z`Dmx3N~;c43}`9GUCAE~@OxUN1n+nFuxZzNf#FI13ywNnNE+{cQphMEaMATqMqC%H z?053VAc<;g3`%}{Np;&D)AquY8yq@&ECIvHI}|4STP@pl2!AMMyBO3zOwpn_HKqDFh0hB3CegJe@K9pjT({?wEL)jof=R#c| z#+LqjI}#9WlB~+0=2YB*MM_vd9ai(K-IGo*!@cp7*9G)>OzU|w=U^qug5%RG^}`S{ z_6CrhYNs{D2&~iET0l~g@0LsAV+xZyjR+xi)2~A}EH>tS+x7CKcijCX3^BDJTF=fb zABix@N84x* za_A7u*>#PQl^CyG25J!a3$XH&+U^h<{LkH=(~;VdO_Ki}N>;gmhzH+!T#eq@Y2rzt!6eNKWV8ow8(v-a4Yn-E8#x`1A z0A#_3T*9aAOCvp!T3N3wEIY4+R(2v6BT{yq1FqUngmN+3g%O!qSVJvhX6+fVFJluO ztWTj3>6y+AK_NWm0`1j%NXnPPvs?+}mbcNaASquHUplD*g?ej^&%EP9k3%so(!bP?GMD z@4r@|UJ)kl5gby4F(!#W{BgfK z)YKDX=hQO{AusXzbe#IrM}R~uO0P%$^C{Lc`1#MTKtq(D2?JSkY-jnipMSmgX%sv) z`t}7aCFU*`u! z4~rGEH2=4anZZ_05t&UXm10k$VX>LFA?(x$#=kapB?!KMbUad%0eE+!di3e{(}br0sPxNVXgr9FKMuG&`~Vi{lLa+)9yAwp7cNAArJ%K&u3~kQ9M-w zN$}}l);>RKsodWydhLc+a$T{azQoH29`m?r$MbJX!7=;*udr_RSW42ez(;s$XSSOP z?|-~<0lt5kb5VtIZUH6ur2h6~o!{SYqx=nnAWRAal=vVYS<~QCTp16~zpu3o|6lo_ z)U%s_tK+WA&p*k4f5Sr^;W`@k)wPfv<7<%Axr$v<|G5@Dd@uBVK?+BC>N|Ym8jRAW z)|&n6k}w}2p9EKVSc;hnZ&3iVp+Mfwe}_JBBP0#i{EONN%3==@A4~yOO61=`hBpC` zC6pK=b&c{=5Pb4YI&|z$0n6X}=YyBX#O3X#z61;UPI+A(|9`%c69NA`4zrVT!+Q<` z0>{;j-vGqMGw#aNEw~H=aJ6+o_sTS?#J;QQN>dk1zmfiGqM$qXw)Y7zM)!&u978gx zp^@Ah&zo;=%)2#&f|o5p;rVqb$oV_g2w{21B6pF+R-Dh)FK#-w0SUMXrY%I`V%B!Q;s|v z>anp}>#;T~)RCdx?D(dAb!BlPas)E&U5`Fhk(sY#VB$V=YU}_c`C8@I9tGucY)uSe zzR-C281z#xa>gI@ni@LIcyaU}27;vGz^sE0|&Z|wDB{$K(ACvrup zqQ@xr4XPVsJG)wK)FHSE)`R%DiZzw5FZK&U!%D}S(`WDZ)WFEl!_fb*k+1<{RE4#T z+zZ{v9oo9ZS3NH(zKMllsIBF$zm_`o2hNqw$vQ*kZ)4iZ3n8A0bgit2dug%{hA_T& z%d_nXInn-b81Lnk+=%59j? zE=;whl8&dVC;Id`v?(Xx-KR|Z-rFv~IL`16nCKJ=ccNc3=;7M(O-~U|ns-3&=QYhq zB;r1oi{6dKyT!&wX!`2o2q%$v-lXI0&3xEzgqMsv>2{S*E%e}4 zewsD>K5}fpbxzVgkyONGzXtM)A><}J0S&w@tcSu~#OuAG^SBOl4KdE!!ba|*>>G0t zRExG&m^eo?SM3u;Pj6K7dxk*CLh|J>QUzMORAo)0z)ZP?ZMq~S>8fW|HJ@uk!x}bS z;|Tj_BGeXam}>{ddO&Cc*i%Ch%vDPXFYWIL*y_C^wL)D@6Ax4dr}E=8q?=y8C=*k-OG_arWIS8 zt}V`!WM#JKSkHRM2!SFG2~s5catX?2pwB>m)toW<^1X%8+LW2qaUYef#KpD(sye5} zhRB2XCQGV3&;n@Y-E4k_u&a1hoxD3wc9s{O!d_186^f51>?lNs5o9hJ#+FXM%`lm6 zPd|F1e0`z5QCVfyz0A#hb7cZ4E08GVv;E9w*RHPUEcGJQ{fs;@Xs~|LVQstQmCgsc zHluH8Y*$~2r~#K)R7fSZ{Dk4+HY$Tz4aVjBNiZb%PAAE2$%RhGwI%)TJH92&oh5sj zYH=X%H9`$90A>i$}Sj z5*hbew6cOGu5EAL|BthNTWOpX#x9vfx=3iqgDlOxy(O^ZF}#V;Dv zC!>DzpdHdC56}=MYs{EByr*j-xK;)|VF1o4`!Bzy+NwhiZb-@TG4Q65YOI+O$>V8g zjY|DuYqJ4Yrnk73mT4FnK-l?%>{`BeDdL;y@TVsW?V76L)vJRru;T8(zHbMrr&C#G z6cpbB%lNoloPf&iIAu5}|4scT1Fu8Sq`v{~Oj&1g#nm{$Z!k+<1f;HHEasc~+|)If zxa?S+Z!PoOOQcChclVR}-aD8Y4s4rRo2QfzY{TE#N#DVoZsM=irIr~ygrh2-PyK+J zZdmx)A@@Je@$HraOwAh~z3k6zN~=}be8cFH;&qb2G-2}0N$&CKGrZ_MdR5=t_A)&d zD;eF>($Yz|cMb-H=to9|O`l$j&i~qIDW;n*A0V@=Q;=RE{9yE@BBA?a&izNs?YOr* z(c+3+(Jn`2ei!Y2+rJ!XO{k?GCs@yP<(Xb?CAst)dC$bixJCqFiJ{mthl%N44(FLF za3*D%{SDBg7nx)rfhxzcp!-!NaYX55Owu*q4P-! zWBZI~lAC+BAzEXa8eS;o;YB`h&};69R#H&h$&lx^K_)KD6f{Sa^x)nR$|uvvP0{Bg zrM|$>)Z!OYcV_NKRZLHsMJx952}f4|wC~FYCZjgmoe3K{6@a+yQ4{EkSjF_Ua(cLO z=>_y$nJCl)2JS40Zo1u)pm6jQ@qwYur?)7dljZNhZ7IJ7m7(_retuJ*?zJwxKUI}f zu>}$oQnRup2E$@fzB0FyYkimwL^yAfIx@m~!WgJ}#Eo}wC@{=6vBZGc`;II75_r&m#R3BVpICtm|-` zjn_E~pE;m&Ggb1ck%J#=dF?pXb=K6{Bx2J*HV}~;?R{h1%$8CcSh%&eOP0|UU-@k3 z9Qe>1&pi_2mZE+y7l8v(F9%&+-Wlk@eL*oKtQdH@&O5^!E=ti$n2mRBD#5_ai_0#} zOH(yt1yI4oj-$kU5DOy60d}8cL=ztg2$tw(O}iHzS1C;l03II z$M`mZ6^A}!7m85j1l@u+eRdYoA8t8=0BsDpkS^p*QCz$*jEFU-wv&Sfk(`iT4b~iz zwGN*c6TJo=w=#?2ViuWs(KoJ~mi!CR5655zW(aWUk0BQ(>iD2C=!|vWxjFGE#wqi* zl<)&sl^E&n86%K85^}|1En|rG6wYG6l5{)2ZGFUkvG9k&DN0G*5Ma>8vF zC?agQvJmaqQ`^xu3U@JOg8{0o>Ns7wv8=zCzWrq2j)~Q0+YM(h<+=rCc2N4)^JV-` zol86$SVPJ;)mSOyaTu|B;$KT4b+ZCxv_@|8PxhB&9r?>*6eoK(r|re$`T#PGuUpyX ztjmZ$YTW**crP-*cU;*UW}no*`0N2g=aQfT$+W1x@S;s_dSa$z+MwswhC-^)ysPl# zxNK7`$n^9y8t_=V-%>J6?rRN$r)!bf>Y!7l-T>LUk={PCKuZq8y41ECTlh-z_rdSe zQ5rKCLpj3-RZOT+=fh|Jq>m$(_>jNzc4YH3^hnDJ;az+b?ta4yv)ax>$G{vS7%1JU zDeORC=DnSXqUSw*=dqMe(|K9T&J&12%;Ub);>`OCkOC#xiHitS+HG_3DPu zY=+hkPQl7&ZQUD?F~2ZeOSE9)NSMJPD6e)Z&nX|?DHJ8(Ql-K>oV}Rlsta@@DY47Q z+k(lr&3oexT*O$XR1@xSWkue?oaa_=!)!sx zqD6mSh4H`Q7iL@0r7As|+0u^x$GfquX6K^6vy_R}f&k4<&;=BGr$SLj< z{et<(sAi3lR+Z@`+4H%SumWhAl zfzDnFP_=N3K5gu?-!OA93T7bWg9TF}lXN{+)=>2Q5N$#Ej<>EedA{LqM}aO;D6tOb zlG&F(t(o7pMe#Ux-A=GCw3K=$y&}4bS_38~F$SZ-BEz3c#~LHHqFsvuSGZx5%r65< zwfzrJ@&K4kFPeyv?_f5f_x`oeO*5dw1<>5XJErifkYQ+EH}K9}(;PV3RKLbxMySsa zY&UT-@&sBTKqj`NT-_PJ%I~>;P?5*>are3x9di&U&0hv2N!lCqpIVczDGYv_0^`o& z{NMhwi%4vW;<`uhejaR(z`NykS_ou9=p9T-0)B5!rN$;<%Uu$N@57`Z!{|pUoC$(i zI1f9f3qZ`%?H&Xbx(7^oh^^FOe!DgG+|~P&KR+-}k%gJ2BF~uLDQaz*Mk`z`wCHGH zTwb2-JwIdUc+=&LjKicF*p;f?{sx&X2i{LJJ}{h#8~kFOkodd#{G9Y+p>;n&+++PA zJ5TCdq=*_(>PxKi_q;wzp zxM8ZiuuM*g%1VM}ry#KmGo)DW9>XEjbJIuJXU_CU`8yUZf{TY!ASCC<9?YW$|t zfb-oZ;fD@~%PV%iyf0s-LMP$$qc3W)HnsAZ?az-4k>+@f?IPqMpv}{$s~sIKyttXJ z(}mmdE`E=wI>Z0o?Iqwx3of2qY3S#WojZpp{Qm*eR3lli9g@)P|YARQWPag}<| zZl`4dZS!z-0mN&B`4xv~>ozcHTJop!$gYN3*#_p9`&6UOwKTY9$5)E3G?iOB9w*28 ztO_aya4@ao5Yxwb@j`uf(S3c*9q#T-3zCdPVcg7pd;frD&NNG}oC_5@R>8r9TKq_A zlqECK8n&z25LSnM`IR!XvKmTAt4kO|8%##x*Fq~jZFJT(UuIs1g7yS*2DBkvdZn9c z^R{cTGxcmf-{b*da63toQx`u0XuCrc2K#bJm||6y-((#^R=~GnF86{9skwQnX|}rt zcDCO?NiD;Cd=qS>Ovy_n6cKa>X}?f!F_r|M@HM;c1-+$7RVja2aqnkVV#e&P9uE~d zlW1*&OJi2gRpb%`KLZG-o;TGSR(iO%tQ6r~-Yh9xa)~~Kcyg4C?(5wv+$fdC7I(iwXv{)O^t35zb6n?F)7uNt|=rr=C={0;+i1X4IVZd*PG!v z5M}SzY1U3{%;RF})PHyH_CV0LmhY=a9|sr|gQYdwN{8FPDsCCM&K9nIgCsMf!LHoF z6D=s38N#=b$wuuqx(BY6pj4S$pEJ%OstQc5FE1zHJYn*Eb5DP3eaLwl^f7ZPAuq20e0z$2z!0h zCfx97dTNaL!91T@mFEGke?SR6&%+^nE3_s_RhG_aQy)QVKu*}8WLxYx-YZqOa^X}J zreImSUh`ha_3EF15!tANB~7(Z_`4D zaG_J;?atadbF1S(yoEsKmP3;@ipG7_aw^T4cQ&)SElyX_n#6dIX@VRZkjdiGGIisO zj+PKpPoZ&^mj40N*~I(}w+*DKead8s>~OeZAhWGL>Ze8Wm{u`ROJu%AmP5kqf0?19VLuZjt{J%Zo)eOIt^|m{Qyo!e24vFHm z4kJ1jVnP1vk54Co?0>ArJ5YXnnq3DPuYfOK*6i+=segMTtj_9*uL~8qwm9BMC)J{K zAoEC&puHM(twSe%!lcVu=))oCwu9q8!ZX|OCc9uf&?T5u&r?&&p2qO0?gGWcO%Tm> zTd`~L{NrQ($NN}M;$6to&o+1eSMDWpczNs)cpY9n+0Au%Q~KOLPeD1)tE1mQMM($7 zEs&}l$HyqbKnZ$1_5J`6gi+Al1FO;>F#j8nV9~=Dy&2>c|l_RpQX;f4{ze-sYdsE(OQ_?ehNX8u5rB z>cGTD!R3EI@$bC{{pmhP^(_4B_Wx@EfaMivxbi>S_%EdTA87o8NBm!f8~81TkidxY V*yT?m+u*;e3M%s17jHfK{{R9XB?15d literal 0 HcmV?d00001 diff --git a/network-common/pom.xml b/network-common/pom.xml new file mode 100644 index 0000000..9201457 --- /dev/null +++ b/network-common/pom.xml @@ -0,0 +1,109 @@ + + + + + 4.0.0 + + com.bytedance.inf + css-root + 1.0.0 + ../pom.xml + + + css-network-common + jar + Cloud Shuffle Service Network Common + + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + + io.netty + netty-all + + + org.apache.commons + commons-lang3 + + + + org.fusesource.leveldbjni + leveldbjni-all + 1.8 + + + + io.dropwizard.metrics + metrics-core + + + + io.dropwizard.metrics + metrics-jvm + + + + org.slf4j + slf4j-api + provided + + + com.google.code.findbugs + jsr305 + + + com.google.guava + guava + compile + + + org.apache.commons + commons-crypto + + + + + log4j + log4j + test + + + junit + junit + test + + + org.mockito + mockito-core + test + + + org.slf4j + slf4j-log4j12 + test + + + diff --git a/network-common/src/main/java/com/bytedance/css/network/TransportContext.java b/network-common/src/main/java/com/bytedance/css/network/TransportContext.java new file mode 100644 index 0000000..916193a --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/TransportContext.java @@ -0,0 +1,231 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network; + +import java.io.Closeable; +import java.util.ArrayList; +import java.util.List; + +import com.bytedance.css.network.util.IOMode; +import com.codahale.metrics.Counter; +import io.netty.channel.Channel; +import io.netty.channel.ChannelPipeline; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.socket.SocketChannel; +import io.netty.handler.timeout.IdleStateHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.client.TransportClientBootstrap; +import com.bytedance.css.network.client.TransportClientFactory; +import com.bytedance.css.network.client.TransportResponseHandler; +import com.bytedance.css.network.protocol.MessageDecoder; +import com.bytedance.css.network.protocol.MessageEncoder; +import com.bytedance.css.network.server.RpcHandler; +import com.bytedance.css.network.server.TransportChannelHandler; +import com.bytedance.css.network.server.TransportRequestHandler; +import com.bytedance.css.network.server.TransportServer; +import com.bytedance.css.network.server.TransportServerBootstrap; +import com.bytedance.css.network.server.ChunkFetchRequestHandler; +import com.bytedance.css.network.util.NettyUtils; +import com.bytedance.css.network.util.TransportConf; +import com.bytedance.css.network.util.TransportFrameDecoder; + +/** + * Contains the context to create a {@link TransportServer}, {@link TransportClientFactory}, and to + * setup Netty Channel pipelines with a + * {@link com.bytedance.css.network.server.TransportChannelHandler}. + * + * There are two communication protocols that the TransportClient provides, control-plane RPCs and + * data-plane "chunk fetching". The handling of the RPCs is performed outside of the scope of the + * TransportContext (i.e., by a user-provided handler), and it is responsible for setting up streams + * which can be streamed through the data plane in chunks using zero-copy IO. + * + * The TransportServer and TransportClientFactory both create a TransportChannelHandler for each + * channel. As each TransportChannelHandler contains a TransportClient, this enables server + * processes to send messages back to the client on an existing channel. + */ +public class TransportContext implements Closeable { + private static final Logger logger = LoggerFactory.getLogger(TransportContext.class); + + private final TransportConf conf; + private final RpcHandler rpcHandler; + private final boolean closeIdleConnections; + // Number of registered connections to the shuffle service + private Counter registeredConnections = new Counter(); + + /** + * Force to create MessageEncoder and MessageDecoder so that we can make sure they will be created + * before switching the current context class loader to ExecutorClassLoader. + * + * Netty's MessageToMessageEncoder uses Javassist to generate a matcher class and the + * implementation calls "Class.forName" to check if this calls is already generated. If the + * following two objects are created in "ExecutorClassLoader.findClass", it will cause + * "ClassCircularityError". This is because loading this Netty generated class will call + * "ExecutorClassLoader.findClass" to search this class, and "ExecutorClassLoader" will try to use + * RPC to load it and cause to load the non-exist matcher class again. JVM will report + * `ClassCircularityError` to prevent such infinite recursion. (See SPARK-17714) + */ + private static final MessageEncoder ENCODER = MessageEncoder.INSTANCE; + private static final MessageDecoder DECODER = MessageDecoder.INSTANCE; + + // Separate thread pool for handling ChunkFetchRequest. This helps to enable throttling + // max number of TransportServer worker threads that are blocked on writing response + // of ChunkFetchRequest message back to the client via the underlying channel. + private final EventLoopGroup chunkFetchWorkers; + + public TransportContext(TransportConf conf, RpcHandler rpcHandler) { + this(conf, rpcHandler, false); + } + + public TransportContext(TransportConf conf, RpcHandler rpcHandler, boolean closeIdleConnections) { + this(conf, rpcHandler, closeIdleConnections, false); + } + public TransportContext( + TransportConf conf, + RpcHandler rpcHandler, + boolean closeIdleConnections, + boolean isClientOnly) { + this.conf = conf; + this.rpcHandler = rpcHandler; + this.closeIdleConnections = closeIdleConnections; + + if (conf.getModuleName() != null && + conf.getModuleName().equalsIgnoreCase("fetch") && + !isClientOnly && conf.separateChunkFetchRequest()) { + chunkFetchWorkers = NettyUtils.createEventLoop( + IOMode.valueOf(conf.ioMode()), + conf.chunkFetchHandlerThreads(), + "shuffle-chunk-fetch-handler"); + } else { + chunkFetchWorkers = null; + } + } + + /** + * Initializes a ClientFactory which runs the given TransportClientBootstraps prior to returning + * a new Client. Bootstraps will be executed synchronously, and must run successfully in order + * to create a Client. + */ + public TransportClientFactory createClientFactory(List bootstraps) { + return new TransportClientFactory(this, bootstraps); + } + + public TransportClientFactory createClientFactory() { + return createClientFactory(new ArrayList<>()); + } + + /** Create a server which will attempt to bind to a specific port. */ + public TransportServer createServer(int port, List bootstraps) { + return new TransportServer(this, null, port, rpcHandler, bootstraps); + } + + /** Create a server which will attempt to bind to a specific host and port. */ + public TransportServer createServer( + String host, int port, List bootstraps) { + return new TransportServer(this, host, port, rpcHandler, bootstraps); + } + + /** Creates a new server, binding to any available ephemeral port. */ + public TransportServer createServer(List bootstraps) { + return createServer(0, bootstraps); + } + + public TransportServer createServer() { + return createServer(0, new ArrayList<>()); + } + + public TransportChannelHandler initializePipeline(SocketChannel channel) { + return initializePipeline(channel, rpcHandler); + } + + /** + * Initializes a client or server Netty Channel Pipeline which encodes/decodes messages and + * has a {@link com.bytedance.css.network.server.TransportChannelHandler} to handle request or + * response messages. + * + * @param channel The channel to initialize. + * @param channelRpcHandler The RPC handler to use for the channel. + * + * @return Returns the created TransportChannelHandler, which includes a TransportClient that can + * be used to communicate on this channel. The TransportClient is directly associated with a + * ChannelHandler to ensure all users of the same channel get the same TransportClient object. + */ + public TransportChannelHandler initializePipeline( + SocketChannel channel, + RpcHandler channelRpcHandler) { + try { + TransportChannelHandler channelHandler = createChannelHandler(channel, channelRpcHandler); + ChannelPipeline pipeline = channel.pipeline() + .addLast("encoder", ENCODER) + .addLast(TransportFrameDecoder.HANDLER_NAME, NettyUtils.createFrameDecoder()) + .addLast("decoder", DECODER) + .addLast("idleStateHandler", new IdleStateHandler(0, 0, conf.connectionTimeoutMs() / 1000)) + // NOTE: Chunks are currently guaranteed to be returned in the order of request, but this + // would require more logic to guarantee if this were not part of the same event loop. + .addLast("handler", channelHandler); + // Use a separate EventLoopGroup to handle ChunkFetchRequest messages for shuffle rpcs. + if (chunkFetchWorkers != null) { + ChunkFetchRequestHandler chunkFetchHandler = new ChunkFetchRequestHandler( + channelHandler.getClient(), rpcHandler.getStreamManager(), + conf.maxChunksBeingTransferred(), true /* syncModeEnabled */); + pipeline.addLast(chunkFetchWorkers, "chunkFetchHandler", chunkFetchHandler); + } + return channelHandler; + } catch (RuntimeException e) { + logger.error("Error while initializing Netty pipeline", e); + throw e; + } + } + + /** + * Creates the server- and client-side handler which is used to handle both RequestMessages and + * ResponseMessages. The channel is expected to have been successfully created, though certain + * properties (such as the remoteAddress()) may not be available yet. + */ + private TransportChannelHandler createChannelHandler(Channel channel, RpcHandler rpcHandler) { + TransportResponseHandler responseHandler = new TransportResponseHandler(channel); + TransportClient client = new TransportClient(channel, responseHandler); + boolean separateChunkFetchRequest = conf.separateChunkFetchRequest(); + ChunkFetchRequestHandler chunkFetchRequestHandler = null; + if (!separateChunkFetchRequest) { + chunkFetchRequestHandler = new ChunkFetchRequestHandler( + client, rpcHandler.getStreamManager(), + conf.maxChunksBeingTransferred(), false /* syncModeEnabled */); + } + TransportRequestHandler requestHandler = new TransportRequestHandler(channel, client, + rpcHandler, conf.maxChunksBeingTransferred(), chunkFetchRequestHandler); + return new TransportChannelHandler(client, responseHandler, requestHandler, + conf.connectionTimeoutMs(), separateChunkFetchRequest, closeIdleConnections, this); + } + + public TransportConf getConf() { return conf; } + + public Counter getRegisteredConnections() { + return registeredConnections; + } + + public void close() { + if (chunkFetchWorkers != null) { + chunkFetchWorkers.shutdownGracefully(); + } + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/buffer/FileSegmentManagedBuffer.java b/network-common/src/main/java/com/bytedance/css/network/buffer/FileSegmentManagedBuffer.java new file mode 100644 index 0000000..c3519b9 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/buffer/FileSegmentManagedBuffer.java @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.buffer; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.StandardOpenOption; + +import com.google.common.io.ByteStreams; +import io.netty.channel.DefaultFileRegion; + +import com.bytedance.css.network.util.JavaUtils; +import com.bytedance.css.network.util.LimitedInputStream; +import com.bytedance.css.network.util.TransportConf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +/** + * A {@link ManagedBuffer} backed by a segment in a file. + */ +public final class FileSegmentManagedBuffer extends ManagedBuffer { + private final TransportConf conf; + private final File file; + private final long offset; + private final long length; + + public FileSegmentManagedBuffer(TransportConf conf, File file, long offset, long length) { + this.conf = conf; + this.file = file; + this.offset = offset; + this.length = length; + } + + @Override + public long size() { + return length; + } + + @Override + public ByteBuffer nioByteBuffer() throws IOException { + FileChannel channel = null; + try { + channel = new RandomAccessFile(file, "r").getChannel(); + // Just copy the buffer if it's sufficiently small, as memory mapping has a high overhead. + if (length < conf.memoryMapBytes()) { + ByteBuffer buf = ByteBuffer.allocate((int) length); + channel.position(offset); + while (buf.remaining() != 0) { + if (channel.read(buf) == -1) { + throw new IOException(String.format("Reached EOF before filling buffer\n" + + "offset=%s\nfile=%s\nbuf.remaining=%s", + offset, file.getAbsoluteFile(), buf.remaining())); + } + } + buf.flip(); + return buf; + } else { + return channel.map(FileChannel.MapMode.READ_ONLY, offset, length); + } + } catch (IOException e) { + String errorMessage = "Error in reading " + this; + try { + if (channel != null) { + long size = channel.size(); + errorMessage = "Error in reading " + this + " (actual file length " + size + ")"; + } + } catch (IOException ignored) { + // ignore + } + throw new IOException(errorMessage, e); + } finally { + JavaUtils.closeQuietly(channel); + } + } + + @Override + public InputStream createInputStream() throws IOException { + FileInputStream is = null; + boolean shouldClose = true; + try { + is = new FileInputStream(file); + ByteStreams.skipFully(is, offset); + InputStream r = new LimitedInputStream(is, length); + shouldClose = false; + return r; + } catch (IOException e) { + String errorMessage = "Error in reading " + this; + if (is != null) { + long size = file.length(); + errorMessage = "Error in reading " + this + " (actual file length " + size + ")"; + } + throw new IOException(errorMessage, e); + } finally { + if (shouldClose) { + JavaUtils.closeQuietly(is); + } + } + } + + @Override + public ManagedBuffer retain() { + return this; + } + + @Override + public ManagedBuffer release() { + return this; + } + + @Override + public Object convertToNetty() throws IOException { + if (conf.lazyFileDescriptor()) { + return new DefaultFileRegion(file, offset, length); + } else { + FileChannel fileChannel = FileChannel.open(file.toPath(), StandardOpenOption.READ); + return new DefaultFileRegion(fileChannel, offset, length); + } + } + + public File getFile() { return file; } + + public long getOffset() { return offset; } + + public long getLength() { return length; } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("file", file) + .append("offset", offset) + .append("length", length) + .toString(); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/buffer/ManagedBuffer.java b/network-common/src/main/java/com/bytedance/css/network/buffer/ManagedBuffer.java new file mode 100644 index 0000000..1685bea --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/buffer/ManagedBuffer.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.buffer; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; + +/** + * This interface provides an immutable view for data in the form of bytes. The implementation + * should specify how the data is provided: + * + * - {@link FileSegmentManagedBuffer}: data backed by part of a file + * - {@link NioManagedBuffer}: data backed by a NIO ByteBuffer + * - {@link NettyManagedBuffer}: data backed by a Netty ByteBuf + * + * The concrete buffer implementation might be managed outside the JVM garbage collector. + * For example, in the case of {@link NettyManagedBuffer}, the buffers are reference counted. + * In that case, if the buffer is going to be passed around to a different thread, retain/release + * should be called. + */ +public abstract class ManagedBuffer { + + /** + * Number of bytes of the data. If this buffer will decrypt for all of the views into the data, + * this is the size of the decrypted data. + */ + public abstract long size(); + + /** + * Exposes this buffer's data as an NIO ByteBuffer. Changing the position and limit of the + * returned ByteBuffer should not affect the content of this buffer. + */ + // TODO: Deprecate this, usage may require expensive memory mapping or allocation. + public abstract ByteBuffer nioByteBuffer() throws IOException; + + /** + * Exposes this buffer's data as an InputStream. The underlying implementation does not + * necessarily check for the length of bytes read, so the caller is responsible for making sure + * it does not go over the limit. + */ + public abstract InputStream createInputStream() throws IOException; + + /** + * Increment the reference count by one if applicable. + */ + public abstract ManagedBuffer retain(); + + /** + * If applicable, decrement the reference count by one and deallocates the buffer if the + * reference count reaches zero. + */ + public abstract ManagedBuffer release(); + + /** + * Convert the buffer into an Netty object, used to write the data out. The return value is either + * a {@link io.netty.buffer.ByteBuf} or a {@link io.netty.channel.FileRegion}. + * + * If this method returns a ByteBuf, then that buffer's reference count will be incremented and + * the caller will be responsible for releasing this new reference. + */ + public abstract Object convertToNetty() throws IOException; +} diff --git a/network-common/src/main/java/com/bytedance/css/network/buffer/NettyManagedBuffer.java b/network-common/src/main/java/com/bytedance/css/network/buffer/NettyManagedBuffer.java new file mode 100644 index 0000000..83f3767 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/buffer/NettyManagedBuffer.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.buffer; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufInputStream; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +/** + * A {@link ManagedBuffer} backed by a Netty {@link ByteBuf}. + */ +public class NettyManagedBuffer extends ManagedBuffer { + private final ByteBuf buf; + + public NettyManagedBuffer(ByteBuf buf) { + this.buf = buf; + } + + public ByteBuf getBuf() { + return buf.duplicate(); + } + + @Override + public long size() { + return buf.readableBytes(); + } + + @Override + public ByteBuffer nioByteBuffer() throws IOException { + return buf.nioBuffer(); + } + + @Override + public InputStream createInputStream() throws IOException { + return new ByteBufInputStream(buf); + } + + @Override + public ManagedBuffer retain() { + buf.retain(); + return this; + } + + @Override + public ManagedBuffer release() { + buf.release(); + return this; + } + + @Override + public Object convertToNetty() throws IOException { + return buf.duplicate().retain(); + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("buf", buf) + .toString(); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/buffer/NioManagedBuffer.java b/network-common/src/main/java/com/bytedance/css/network/buffer/NioManagedBuffer.java new file mode 100644 index 0000000..7ef47dc --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/buffer/NioManagedBuffer.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.buffer; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; + +import io.netty.buffer.ByteBufInputStream; +import io.netty.buffer.Unpooled; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +/** + * A {@link ManagedBuffer} backed by {@link ByteBuffer}. + */ +public class NioManagedBuffer extends ManagedBuffer { + private final ByteBuffer buf; + + public NioManagedBuffer(ByteBuffer buf) { + this.buf = buf; + } + + @Override + public long size() { + return buf.remaining(); + } + + @Override + public ByteBuffer nioByteBuffer() throws IOException { + return buf.duplicate(); + } + + @Override + public InputStream createInputStream() throws IOException { + return new ByteBufInputStream(Unpooled.wrappedBuffer(buf)); + } + + @Override + public ManagedBuffer retain() { + return this; + } + + @Override + public ManagedBuffer release() { + return this; + } + + @Override + public Object convertToNetty() throws IOException { + return Unpooled.wrappedBuffer(buf); + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("buf", buf) + .toString(); + } +} + diff --git a/network-common/src/main/java/com/bytedance/css/network/client/ChunkFetchFailureException.java b/network-common/src/main/java/com/bytedance/css/network/client/ChunkFetchFailureException.java new file mode 100644 index 0000000..f649f35 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/client/ChunkFetchFailureException.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.client; + +/** + * General exception caused by a remote exception while fetching a chunk. + */ +public class ChunkFetchFailureException extends RuntimeException { + public ChunkFetchFailureException(String errorMsg, Throwable cause) { + super(errorMsg, cause); + } + + public ChunkFetchFailureException(String errorMsg) { + super(errorMsg); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/client/ChunkReceivedCallback.java b/network-common/src/main/java/com/bytedance/css/network/client/ChunkReceivedCallback.java new file mode 100644 index 0000000..b33ea05 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/client/ChunkReceivedCallback.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.client; + +import com.bytedance.css.network.buffer.ManagedBuffer; + +/** + * Callback for the result of a single chunk result. For a single stream, the callbacks are + * guaranteed to be called by the same thread in the same order as the requests for chunks were + * made. + * + * Note that if a general stream failure occurs, all outstanding chunk requests may be failed. + */ +public interface ChunkReceivedCallback { + /** + * Called upon receipt of a particular chunk. + * + * The given buffer will initially have a refcount of 1, but will be release()'d as soon as this + * call returns. You must therefore either retain() the buffer or copy its contents before + * returning. + */ + void onSuccess(int chunkIndex, ManagedBuffer buffer); + + /** + * Called upon failure to fetch a particular chunk. Note that this may actually be called due + * to failure to fetch a prior chunk in this stream. + * + * After receiving a failure, the stream may or may not be valid. The client should not assume + * that the server's side of the stream has been closed. + */ + void onFailure(int chunkIndex, Throwable e); +} diff --git a/network-common/src/main/java/com/bytedance/css/network/client/RpcResponseCallback.java b/network-common/src/main/java/com/bytedance/css/network/client/RpcResponseCallback.java new file mode 100644 index 0000000..596fd19 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/client/RpcResponseCallback.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.client; + +import java.nio.ByteBuffer; + +/** + * Callback for the result of a single RPC. This will be invoked once with either success or + * failure. + */ +public interface RpcResponseCallback { + /** + * Successful serialized result from server. + * + * After `onSuccess` returns, `response` will be recycled and its content will become invalid. + * Please copy the content of `response` if you want to use it after `onSuccess` returns. + */ + void onSuccess(ByteBuffer response); + + /** Exception either propagated from server or raised on client side. */ + void onFailure(Throwable e); +} diff --git a/network-common/src/main/java/com/bytedance/css/network/client/StreamCallback.java b/network-common/src/main/java/com/bytedance/css/network/client/StreamCallback.java new file mode 100644 index 0000000..0814541 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/client/StreamCallback.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.client; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * Callback for streaming data. Stream data will be offered to the + * {@link #onData(String, ByteBuffer)} method as it arrives. Once all the stream data is received, + * {@link #onComplete(String)} will be called. + *

    + * The network library guarantees that a single thread will call these methods at a time, but + * different call may be made by different threads. + */ +public interface StreamCallback { + /** Called upon receipt of stream data. */ + void onData(String streamId, ByteBuffer buf) throws IOException; + + /** Called when all data from the stream has been received. */ + void onComplete(String streamId) throws IOException; + + /** Called if there's an error reading data from the stream. */ + void onFailure(String streamId, Throwable cause) throws IOException; +} diff --git a/network-common/src/main/java/com/bytedance/css/network/client/StreamCallbackWithID.java b/network-common/src/main/java/com/bytedance/css/network/client/StreamCallbackWithID.java new file mode 100644 index 0000000..2f36d74 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/client/StreamCallbackWithID.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.client; + +public interface StreamCallbackWithID extends StreamCallback { + String getID(); +} diff --git a/network-common/src/main/java/com/bytedance/css/network/client/StreamInterceptor.java b/network-common/src/main/java/com/bytedance/css/network/client/StreamInterceptor.java new file mode 100644 index 0000000..cfc65f3 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/client/StreamInterceptor.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.client; + +import java.nio.ByteBuffer; +import java.nio.channels.ClosedChannelException; + +import io.netty.buffer.ByteBuf; + +import com.bytedance.css.network.protocol.Message; +import com.bytedance.css.network.server.MessageHandler; +import com.bytedance.css.network.util.TransportFrameDecoder; + +/** + * An interceptor that is registered with the frame decoder to feed stream data to a + * callback. + */ +public class StreamInterceptor implements TransportFrameDecoder.Interceptor { + + private final MessageHandler handler; + private final String streamId; + private final long byteCount; + private final StreamCallback callback; + private long bytesRead; + + public StreamInterceptor( + MessageHandler handler, + String streamId, + long byteCount, + StreamCallback callback) { + this.handler = handler; + this.streamId = streamId; + this.byteCount = byteCount; + this.callback = callback; + this.bytesRead = 0; + } + + @Override + public void exceptionCaught(Throwable cause) throws Exception { + deactivateStream(); + callback.onFailure(streamId, cause); + } + + @Override + public void channelInactive() throws Exception { + deactivateStream(); + callback.onFailure(streamId, new ClosedChannelException()); + } + + private void deactivateStream() { + if (handler instanceof TransportResponseHandler) { + // we only have to do this for TransportResponseHandler as it exposes numOutstandingFetches + // (there is no extra cleanup that needs to happen) + ((TransportResponseHandler) handler).deactivateStream(); + } + } + + @Override + public boolean handle(ByteBuf buf) throws Exception { + int toRead = (int) Math.min(buf.readableBytes(), byteCount - bytesRead); + ByteBuffer nioBuffer = buf.readSlice(toRead).nioBuffer(); + + int available = nioBuffer.remaining(); + callback.onData(streamId, nioBuffer); + bytesRead += available; + if (bytesRead > byteCount) { + RuntimeException re = new IllegalStateException(String.format( + "Read too many bytes? Expected %d, but read %d.", byteCount, bytesRead)); + callback.onFailure(streamId, re); + deactivateStream(); + throw re; + } else if (bytesRead == byteCount) { + deactivateStream(); + callback.onComplete(streamId); + } + + return bytesRead != byteCount; + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/client/TransportClient.java b/network-common/src/main/java/com/bytedance/css/network/client/TransportClient.java new file mode 100644 index 0000000..8444396 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/client/TransportClient.java @@ -0,0 +1,390 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.client; + +import java.io.Closeable; +import java.io.IOException; +import java.net.SocketAddress; +import java.nio.ByteBuffer; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import javax.annotation.Nullable; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.util.concurrent.SettableFuture; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.GenericFutureListener; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.buffer.NioManagedBuffer; +import com.bytedance.css.network.protocol.*; + +import static com.bytedance.css.network.util.NettyUtils.getRemoteAddress; + +/** + * Client for fetching consecutive chunks of a pre-negotiated stream. This API is intended to allow + * efficient transfer of a large amount of data, broken up into chunks with size ranging from + * hundreds of KB to a few MB. + * + * Note that while this client deals with the fetching of chunks from a stream (i.e., data plane), + * the actual setup of the streams is done outside the scope of the transport layer. The convenience + * method "sendRPC" is provided to enable control plane communication between the client and server + * to perform this setup. + * + * For example, a typical workflow might be: + * client.sendRPC(new OpenFile("/foo")) --> returns StreamId = 100 + * client.fetchChunk(streamId = 100, chunkIndex = 0, callback) + * client.fetchChunk(streamId = 100, chunkIndex = 1, callback) + * ... + * client.sendRPC(new CloseStream(100)) + * + * Construct an instance of TransportClient using {@link TransportClientFactory}. A single + * TransportClient may be used for multiple streams, but any given stream must be restricted to a + * single client, in order to avoid out-of-order responses. + * + * NB: This class is used to make requests to the server, while {@link TransportResponseHandler} is + * responsible for handling responses from the server. + * + * Concurrency: thread safe and can be called from multiple threads. + */ +public class TransportClient implements Closeable { + private static final Logger logger = LoggerFactory.getLogger(TransportClient.class); + + private final Channel channel; + private final TransportResponseHandler handler; + @Nullable private String clientId; + private volatile boolean timedOut; + + public TransportClient(Channel channel, TransportResponseHandler handler) { + this.channel = Preconditions.checkNotNull(channel); + this.handler = Preconditions.checkNotNull(handler); + this.timedOut = false; + } + + public Channel getChannel() { + return channel; + } + + public boolean isActive() { + return !timedOut && (channel.isOpen() || channel.isActive()); + } + + public SocketAddress getSocketAddress() { + return channel.remoteAddress(); + } + + /** + * Returns the ID used by the client to authenticate itself when authentication is enabled. + * + * @return The client ID, or null if authentication is disabled. + */ + public String getClientId() { + return clientId; + } + + /** + * Sets the authenticated client ID. This is meant to be used by the authentication layer. + * + * Trying to set a different client ID after it's been set will result in an exception. + */ + public void setClientId(String id) { + Preconditions.checkState(clientId == null, "Client ID has already been set."); + this.clientId = id; + } + + /** + * Requests a single chunk from the remote side, from the pre-negotiated streamId. + * + * Chunk indices go from 0 onwards. It is valid to request the same chunk multiple times, though + * some streams may not support this. + * + * Multiple fetchChunk requests may be outstanding simultaneously, and the chunks are guaranteed + * to be returned in the same order that they were requested, assuming only a single + * TransportClient is used to fetch the chunks. + * + * @param streamId Identifier that refers to a stream in the remote StreamManager. This should + * be agreed upon by client and server beforehand. + * @param chunkIndex 0-based index of the chunk to fetch + * @param callback Callback invoked upon successful receipt of chunk, or upon any failure. + */ + public void fetchChunk( + long streamId, + int chunkIndex, + ChunkReceivedCallback callback) { + if (logger.isDebugEnabled()) { + logger.debug("Sending fetch chunk request {} to {}", chunkIndex, getRemoteAddress(channel)); + } + + StreamChunkId streamChunkId = new StreamChunkId(streamId, chunkIndex); + StdChannelListener listener = new StdChannelListener(streamChunkId) { + @Override + void handleFailure(String errorMsg, Throwable cause) { + handler.removeFetchRequest(streamChunkId); + callback.onFailure(chunkIndex, new IOException(errorMsg, cause)); + } + }; + handler.addFetchRequest(streamChunkId, callback); + + channel.writeAndFlush(new ChunkFetchRequest(streamChunkId)).addListener(listener); + } + + /** + * Request to stream the data with the given stream ID from the remote end. + * + * @param streamId The stream to fetch. + * @param callback Object to call with the stream data. + */ + public void stream(String streamId, StreamCallback callback) { + StdChannelListener listener = new StdChannelListener(streamId) { + @Override + void handleFailure(String errorMsg, Throwable cause) throws Exception { + callback.onFailure(streamId, new IOException(errorMsg, cause)); + } + }; + if (logger.isDebugEnabled()) { + logger.debug("Sending stream request for {} to {}", streamId, getRemoteAddress(channel)); + } + + // Need to synchronize here so that the callback is added to the queue and the RPC is + // written to the socket atomically, so that callbacks are called in the right order + // when responses arrive. + synchronized (this) { + handler.addStreamCallback(streamId, callback); + channel.writeAndFlush(new StreamRequest(streamId)).addListener(listener); + } + } + + /** + * Sends an opaque message to the RpcHandler on the server-side. The callback will be invoked + * with the server's response or upon any failure. + * + * @param message The message to send. + * @param callback Callback to handle the RPC's reply. + * @return The RPC's id. + */ + public long sendRpc(ByteBuffer message, RpcResponseCallback callback) { + if (logger.isTraceEnabled()) { + logger.trace("Sending RPC to {}", getRemoteAddress(channel)); + } + + long requestId = requestId(); + handler.addRpcRequest(requestId, callback); + + RpcChannelListener listener = new RpcChannelListener(requestId, callback); + channel.writeAndFlush(new RpcRequest(requestId, new NioManagedBuffer(message))) + .addListener(listener); + + return requestId; + } + + private static final AtomicLong counter = new AtomicLong(); + public static long dataRequestId() { + return counter.getAndIncrement(); + } + + public ChannelFuture batchPushData(BatchPushDataRequest batchPushDataRequest, RpcResponseCallback callback) { + if (logger.isTraceEnabled()) { + logger.trace("Batch Pushing data to {}", getRemoteAddress(channel)); + } + + long requestId = dataRequestId(); + handler.addRpcRequest(requestId, callback); + + batchPushDataRequest.requestId = requestId; + + RpcChannelListener listener = new RpcChannelListener(requestId, callback); + return channel.writeAndFlush(batchPushDataRequest).addListener(listener); + } + + /** + * Send data to the remote end as a stream. This differs from stream() in that this is a request + * to *send* data to the remote end, not to receive it from the remote. + * + * @param meta meta data associated with the stream, which will be read completely on the + * receiving end before the stream itself. + * @param data this will be streamed to the remote end to allow for transferring large amounts + * of data without reading into memory. + * @param callback handles the reply -- onSuccess will only be called when both message and data + * are received successfully. + */ + public long uploadStream( + ManagedBuffer meta, + ManagedBuffer data, + RpcResponseCallback callback) { + if (logger.isTraceEnabled()) { + logger.trace("Sending RPC to {}", getRemoteAddress(channel)); + } + + long requestId = requestId(); + handler.addRpcRequest(requestId, callback); + + RpcChannelListener listener = new RpcChannelListener(requestId, callback); + channel.writeAndFlush(new UploadStream(requestId, meta, data)).addListener(listener); + + return requestId; + } + + /** + * Synchronously sends an opaque message to the RpcHandler on the server-side, waiting for up to + * a specified timeout for a response. + */ + public ByteBuffer sendRpcSync(ByteBuffer message, long timeoutMs) { + final SettableFuture result = SettableFuture.create(); + + sendRpc(message, new RpcResponseCallback() { + @Override + public void onSuccess(ByteBuffer response) { + try { + ByteBuffer copy = ByteBuffer.allocate(response.remaining()); + copy.put(response); + // flip "copy" to make it readable + copy.flip(); + result.set(copy); + } catch (Throwable t) { + logger.warn("Error in responding PRC callback", t); + result.setException(t); + } + } + + @Override + public void onFailure(Throwable e) { + result.setException(e); + } + }); + + try { + return result.get(timeoutMs, TimeUnit.MILLISECONDS); + } catch (ExecutionException e) { + throw Throwables.propagate(e.getCause()); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + + /** + * Sends an opaque message to the RpcHandler on the server-side. No reply is expected for the + * message, and no delivery guarantees are made. + * + * @param message The message to send. + */ + public void send(ByteBuffer message) { + channel.writeAndFlush(new OneWayMessage(new NioManagedBuffer(message))); + } + + /** + * Removes any state associated with the given RPC. + * + * @param requestId The RPC id returned by {@link #sendRpc(ByteBuffer, RpcResponseCallback)}. + */ + public void removeRpcRequest(long requestId) { + handler.removeRpcRequest(requestId); + } + + /** Mark this channel as having timed out. */ + public void timeOut() { + this.timedOut = true; + } + + @VisibleForTesting + public TransportResponseHandler getHandler() { + return handler; + } + + @Override + public void close() { + // close is a local operation and should finish with milliseconds; timeout just to be safe + channel.close().awaitUninterruptibly(10, TimeUnit.SECONDS); + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("remoteAdress", channel.remoteAddress()) + .append("clientId", clientId) + .append("isActive", isActive()) + .toString(); + } + + private static long requestId() { + return Math.abs(UUID.randomUUID().getLeastSignificantBits()); + } + + private class StdChannelListener + implements GenericFutureListener> { + final long startTime; + final Object requestId; + + StdChannelListener(Object requestId) { + this.startTime = System.currentTimeMillis(); + this.requestId = requestId; + } + + @Override + public void operationComplete(Future future) throws Exception { + if (future.isSuccess()) { + if (logger.isTraceEnabled()) { + long timeTaken = System.currentTimeMillis() - startTime; + logger.trace("Sending request {} to {} took {} ms", requestId, + getRemoteAddress(channel), timeTaken); + } + } else { + String errorMsg = String.format("Failed to send RPC %s to %s: %s", requestId, + getRemoteAddress(channel), future.cause()); + logger.error(errorMsg, future.cause()); + channel.close(); + try { + handleFailure(errorMsg, future.cause()); + } catch (Exception e) { + logger.error("Uncaught exception in RPC response callback handler!", e); + } + } + } + + void handleFailure(String errorMsg, Throwable cause) throws Exception {} + } + + private class RpcChannelListener extends StdChannelListener { + final long rpcRequestId; + final RpcResponseCallback callback; + + RpcChannelListener(long rpcRequestId, RpcResponseCallback callback) { + super("RPC " + rpcRequestId); + this.rpcRequestId = rpcRequestId; + this.callback = callback; + } + + @Override + void handleFailure(String errorMsg, Throwable cause) { + handler.removeRpcRequest(rpcRequestId); + callback.onFailure(new IOException(errorMsg, cause)); + } + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/client/TransportClientBootstrap.java b/network-common/src/main/java/com/bytedance/css/network/client/TransportClientBootstrap.java new file mode 100644 index 0000000..cb2a0ad --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/client/TransportClientBootstrap.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.client; + +import io.netty.channel.Channel; + +/** + * A bootstrap which is executed on a TransportClient before it is returned to the user. + * This enables an initial exchange of information (e.g., SASL authentication tokens) on a once-per- + * connection basis. + * + * Since connections (and TransportClients) are reused as much as possible, it is generally + * reasonable to perform an expensive bootstrapping operation, as they often share a lifespan with + * the JVM itself. + */ +public interface TransportClientBootstrap { + /** Performs the bootstrapping operation, throwing an exception on failure. */ + void doBootstrap(TransportClient client, Channel channel) throws RuntimeException; +} diff --git a/network-common/src/main/java/com/bytedance/css/network/client/TransportClientFactory.java b/network-common/src/main/java/com/bytedance/css/network/client/TransportClientFactory.java new file mode 100644 index 0000000..3475fc5 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/client/TransportClientFactory.java @@ -0,0 +1,354 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.client; + +import java.io.Closeable; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.util.List; +import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicReference; + +import com.codahale.metrics.MetricSet; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.collect.Lists; +import io.netty.bootstrap.Bootstrap; +import io.netty.buffer.NettyMemoryMetrics; +import io.netty.buffer.PooledByteBufAllocator; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelOption; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.socket.SocketChannel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.bytedance.css.network.TransportContext; +import com.bytedance.css.network.server.TransportChannelHandler; +import com.bytedance.css.network.util.*; + +/** + * Factory for creating {@link TransportClient}s by using createClient. + * + * The factory maintains a connection pool to other hosts and should return the same + * TransportClient for the same remote host. It also shares a single worker thread pool for + * all TransportClients. + * + * TransportClients will be reused whenever possible. Prior to completing the creation of a new + * TransportClient, all given {@link TransportClientBootstrap}s will be run. + */ +public class TransportClientFactory implements Closeable { + + /** A simple data structure to track the pool of clients between two peer nodes. */ + private static class ClientPool { + TransportClient[] clients; + Object[] locks; + volatile long lastConnectionFailed; + + ClientPool(int size) { + clients = new TransportClient[size]; + locks = new Object[size]; + for (int i = 0; i < size; i++) { + locks[i] = new Object(); + } + lastConnectionFailed = 0; + } + } + + private static final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class); + + private final TransportContext context; + private final TransportConf conf; + private final List clientBootstraps; + private final ConcurrentHashMap connectionPool; + + /** Random number generator for picking connections between peers. */ + private final Random rand; + private final int numConnectionsPerPeer; + + private final Class socketChannelClass; + private EventLoopGroup workerGroup; + private PooledByteBufAllocator pooledAllocator; + private final NettyMemoryMetrics metrics; + private NettyMemoryMetrics cssMetrics; + private final int fastFailTimeWindow; + + public TransportClientFactory( + TransportContext context, + List clientBootstraps) { + this.context = Preconditions.checkNotNull(context); + this.conf = context.getConf(); + this.clientBootstraps = Lists.newArrayList(Preconditions.checkNotNull(clientBootstraps)); + this.connectionPool = new ConcurrentHashMap<>(); + this.numConnectionsPerPeer = conf.numConnectionsPerPeer(); + this.rand = new Random(); + + IOMode ioMode = IOMode.valueOf(conf.ioMode()); + this.socketChannelClass = NettyUtils.getClientChannelClass(ioMode); + this.workerGroup = NettyUtils.createEventLoop( + ioMode, + conf.clientThreads(), + conf.getModuleName() + "-client"); + if (conf.sharedByteBufAllocators()) { + this.pooledAllocator = NettyUtils.getSharedPooledByteBufAllocator( + conf.preferDirectBufsForSharedByteBufAllocators(), false /* allowCache */); + } else { + this.pooledAllocator = NettyUtils.createPooledByteBufAllocator( + conf.preferDirectBufs(), false /* allowCache */, conf.clientThreads()); + } + this.metrics = new NettyMemoryMetrics( + this.pooledAllocator, conf.getModuleName() + "-client", conf); + fastFailTimeWindow = (int)(conf.ioRetryWaitTimeMs() * 0.95); + } + + public void registerCssMetrics(String cssMetricsPrefix) { + synchronized (this) { + if (cssMetrics == null) { + String cssNettyPrefix = String.format("%s#%s", cssMetricsPrefix, conf.getModuleName() + "-client"); + cssMetrics = new NettyMemoryMetrics(pooledAllocator, cssNettyPrefix, conf); + } + } + } + + public MetricSet getAllMetrics() { + return metrics; + } + + public MetricSet getAllCssMetrics() { + return cssMetrics; + } + + /** + * Create a {@link TransportClient} connecting to the given remote host / port. + * + * We maintains an array of clients (size determined by css.shuffle.io.numConnectionsPerPeer) + * and randomly picks one to use. If no client was previously created in the randomly selected + * spot, this function creates a new client and places it there. + * + * Prior to the creation of a new TransportClient, we will execute all + * {@link TransportClientBootstrap}s that are registered with this factory. + * + * This blocks until a connection is successfully established and fully bootstrapped. + * + * Concurrency: This method is safe to call from multiple threads. + */ + public TransportClient createClient(String remoteHost, int remotePort) throws IOException, InterruptedException { + return createClient(remoteHost, remotePort, -1); + } + + public TransportClient createClient(String remoteHost, int remotePort, int seed) + throws IOException, InterruptedException { + return createClient(remoteHost, remotePort, seed, false); + } + + /** + * try best to pick the ClientConnection from pool according to the provided seed, + * if same seed passed in, try to re-use the last connection if possible. + * @param remoteHost + * @param remotePort + * @param seed + * @return + * @throws IOException + * @throws InterruptedException + */ + public TransportClient createClient(String remoteHost, int remotePort, int seed, boolean fastFail) + throws IOException, InterruptedException { + // Get connection from the connection pool first. + // If it is not found or not active, create a new one. + // Use unresolved address here to avoid DNS resolution each time we creates a client. + final InetSocketAddress unresolvedAddress = + InetSocketAddress.createUnresolved(remoteHost, remotePort); + + // Create the ClientPool if we don't have it yet. + ClientPool clientPool = connectionPool.get(unresolvedAddress); + if (clientPool == null) { + connectionPool.putIfAbsent(unresolvedAddress, new ClientPool(numConnectionsPerPeer)); + clientPool = connectionPool.get(unresolvedAddress); + } + + int clientIndex = seed < 0 ? rand.nextInt(numConnectionsPerPeer) : seed % numConnectionsPerPeer; + TransportClient cachedClient = clientPool.clients[clientIndex]; + + if (cachedClient != null && cachedClient.isActive()) { + // Make sure that the channel will not timeout by updating the last use time of the + // handler. Then check that the client is still alive, in case it timed out before + // this code was able to update things. + TransportChannelHandler handler = cachedClient.getChannel().pipeline() + .get(TransportChannelHandler.class); + synchronized (handler) { + handler.getResponseHandler().updateTimeOfLastRequest(); + } + + if (cachedClient.isActive()) { + logger.trace("Returning cached connection to {}: {}", + cachedClient.getSocketAddress(), cachedClient); + return cachedClient; + } + } + + // If we reach here, we don't have an existing connection open. Let's create a new one. + // Multiple threads might race here to create new connections. Keep only one of them active. + final long preResolveHost = System.nanoTime(); + final InetSocketAddress resolvedAddress = new InetSocketAddress(remoteHost, remotePort); + final long hostResolveTimeMs = (System.nanoTime() - preResolveHost) / 1000000; + final String resolvMsg = resolvedAddress.isUnresolved() ? "failed" : "succeed"; + if (hostResolveTimeMs > 2000) { + logger.warn("DNS resolution {} for {} took {} ms", + resolvMsg, resolvedAddress, hostResolveTimeMs); + } else { + logger.trace("DNS resolution {} for {} took {} ms", + resolvMsg, resolvedAddress, hostResolveTimeMs); + } + + synchronized (clientPool.locks[clientIndex]) { + cachedClient = clientPool.clients[clientIndex]; + + if (cachedClient != null) { + if (cachedClient.isActive()) { + logger.trace("Returning cached connection to {}: {}", resolvedAddress, cachedClient); + return cachedClient; + } else { + logger.info("Found inactive connection to {}, creating a new one.", resolvedAddress); + } + } + // If this connection should fast fail when last connection failed in last fast fail time + // window and it did, fail this connection directly. + if (fastFail && System.currentTimeMillis() - clientPool.lastConnectionFailed < + fastFailTimeWindow) { + throw new IOException( + String.format("Connecting to %s failed in the last %s ms, fail this connection directly", + resolvedAddress, fastFailTimeWindow)); + } + try { + clientPool.clients[clientIndex] = createClient(resolvedAddress); + clientPool.lastConnectionFailed = 0; + } catch (IOException e) { + clientPool.lastConnectionFailed = System.currentTimeMillis(); + throw e; + } + return clientPool.clients[clientIndex]; + } + } + + /** + * Create a completely new {@link TransportClient} to the given remote host / port. + * This connection is not pooled. + * + * As with {@link #createClient(String, int)}, this method is blocking. + */ + public TransportClient createUnmanagedClient(String remoteHost, int remotePort) + throws IOException, InterruptedException { + final InetSocketAddress address = new InetSocketAddress(remoteHost, remotePort); + return createClient(address); + } + + /** Create a completely new {@link TransportClient} to the remote address. */ + private TransportClient createClient(InetSocketAddress address) + throws IOException, InterruptedException { + logger.debug("Creating new connection to {}", address); + + Bootstrap bootstrap = new Bootstrap(); + bootstrap.group(workerGroup) + .channel(socketChannelClass) + // Disable Nagle's Algorithm since we don't want packets to wait + .option(ChannelOption.TCP_NODELAY, true) + .option(ChannelOption.SO_KEEPALIVE, true) + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.connectionTimeoutMs()) + .option(ChannelOption.ALLOCATOR, pooledAllocator); + + if (conf.receiveBuf() > 0) { + bootstrap.option(ChannelOption.SO_RCVBUF, conf.receiveBuf()); + } + + if (conf.sendBuf() > 0) { + bootstrap.option(ChannelOption.SO_SNDBUF, conf.sendBuf()); + } + + final AtomicReference clientRef = new AtomicReference<>(); + final AtomicReference channelRef = new AtomicReference<>(); + + bootstrap.handler(new ChannelInitializer() { + @Override + public void initChannel(SocketChannel ch) { + TransportChannelHandler clientHandler = context.initializePipeline(ch); + clientRef.set(clientHandler.getClient()); + channelRef.set(ch); + } + }); + + // Connect to the remote server + long preConnect = System.nanoTime(); + ChannelFuture cf = bootstrap.connect(address); + if (!cf.await(conf.connectionTimeoutMs())) { + throw new IOException( + String.format("Connecting to %s timed out (%s ms)", address, conf.connectionTimeoutMs())); + } else if (cf.cause() != null) { + throw new IOException(String.format("Failed to connect to %s", address), cf.cause()); + } + + TransportClient client = clientRef.get(); + Channel channel = channelRef.get(); + assert client != null : "Channel future completed successfully with null client"; + + // Execute any client bootstraps synchronously before marking the Client as successful. + long preBootstrap = System.nanoTime(); + logger.debug("Connection to {} successful, running bootstraps...", address); + try { + for (TransportClientBootstrap clientBootstrap : clientBootstraps) { + clientBootstrap.doBootstrap(client, channel); + } + } catch (Exception e) { // catch non-RuntimeExceptions too as bootstrap may be written in Scala + long bootstrapTimeMs = (System.nanoTime() - preBootstrap) / 1000000; + logger.error("Exception while bootstrapping client after " + bootstrapTimeMs + " ms", e); + client.close(); + throw Throwables.propagate(e); + } + long postBootstrap = System.nanoTime(); + + logger.info("Successfully created connection to {} after {} ms ({} ms spent in bootstraps)", + address, (postBootstrap - preConnect) / 1000000, (postBootstrap - preBootstrap) / 1000000); + + return client; + } + + /** Close all connections in the connection pool, and shutdown the worker thread pool. */ + @Override + public void close() { + // Go through all clients and close them if they are active. + for (ClientPool clientPool : connectionPool.values()) { + for (int i = 0; i < clientPool.clients.length; i++) { + TransportClient client = clientPool.clients[i]; + if (client != null) { + clientPool.clients[i] = null; + JavaUtils.closeQuietly(client); + } + } + } + connectionPool.clear(); + + if (workerGroup != null && !workerGroup.isShuttingDown()) { + workerGroup.shutdownGracefully(); + } + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/client/TransportResponseHandler.java b/network-common/src/main/java/com/bytedance/css/network/client/TransportResponseHandler.java new file mode 100644 index 0000000..10e5cfd --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/client/TransportResponseHandler.java @@ -0,0 +1,273 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.client; + +import java.io.IOException; +import java.util.Map; +import java.util.Queue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicLong; + +import com.google.common.annotations.VisibleForTesting; +import io.netty.channel.Channel; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.bytedance.css.network.protocol.ChunkFetchFailure; +import com.bytedance.css.network.protocol.ChunkFetchSuccess; +import com.bytedance.css.network.protocol.ResponseMessage; +import com.bytedance.css.network.protocol.RpcFailure; +import com.bytedance.css.network.protocol.RpcResponse; +import com.bytedance.css.network.protocol.StreamChunkId; +import com.bytedance.css.network.protocol.StreamFailure; +import com.bytedance.css.network.protocol.StreamResponse; +import com.bytedance.css.network.server.MessageHandler; +import static com.bytedance.css.network.util.NettyUtils.getRemoteAddress; +import com.bytedance.css.network.util.TransportFrameDecoder; + +/** + * Handler that processes server responses, in response to requests issued from a + * [[TransportClient]]. It works by tracking the list of outstanding requests (and their callbacks). + * + * Concurrency: thread safe and can be called from multiple threads. + */ +public class TransportResponseHandler extends MessageHandler { + private static final Logger logger = LoggerFactory.getLogger(TransportResponseHandler.class); + + private final Channel channel; + + private final Map outstandingFetches; + + private final Map outstandingRpcs; + + private final Queue> streamCallbacks; + private volatile boolean streamActive; + + /** Records the time (in system nanoseconds) that the last fetch or RPC request was sent. */ + private final AtomicLong timeOfLastRequestNs; + + public TransportResponseHandler(Channel channel) { + this.channel = channel; + this.outstandingFetches = new ConcurrentHashMap<>(); + this.outstandingRpcs = new ConcurrentHashMap<>(); + this.streamCallbacks = new ConcurrentLinkedQueue<>(); + this.timeOfLastRequestNs = new AtomicLong(0); + } + + public void addFetchRequest(StreamChunkId streamChunkId, ChunkReceivedCallback callback) { + updateTimeOfLastRequest(); + outstandingFetches.put(streamChunkId, callback); + } + + public void removeFetchRequest(StreamChunkId streamChunkId) { + outstandingFetches.remove(streamChunkId); + } + + public void addRpcRequest(long requestId, RpcResponseCallback callback) { + updateTimeOfLastRequest(); + outstandingRpcs.put(requestId, callback); + } + + public void removeRpcRequest(long requestId) { + outstandingRpcs.remove(requestId); + } + + public void addStreamCallback(String streamId, StreamCallback callback) { + updateTimeOfLastRequest(); + streamCallbacks.offer(ImmutablePair.of(streamId, callback)); + } + + @VisibleForTesting + public void deactivateStream() { + streamActive = false; + } + + /** + * Fire the failure callback for all outstanding requests. This is called when we have an + * uncaught exception or pre-mature connection termination. + */ + private void failOutstandingRequests(Throwable cause) { + for (Map.Entry entry : outstandingFetches.entrySet()) { + try { + entry.getValue().onFailure(entry.getKey().chunkIndex, cause); + } catch (Exception e) { + logger.warn("ChunkReceivedCallback.onFailure throws exception", e); + } + } + for (Map.Entry entry : outstandingRpcs.entrySet()) { + try { + entry.getValue().onFailure(cause); + } catch (Exception e) { + logger.warn("RpcResponseCallback.onFailure throws exception", e); + } + } + for (Pair entry : streamCallbacks) { + try { + entry.getValue().onFailure(entry.getKey(), cause); + } catch (Exception e) { + logger.warn("StreamCallback.onFailure throws exception", e); + } + } + + // It's OK if new fetches appear, as they will fail immediately. + outstandingFetches.clear(); + outstandingRpcs.clear(); + streamCallbacks.clear(); + } + + @Override + public void channelActive() { + } + + @Override + public void channelInactive() { + if (numOutstandingRequests() > 0) { + String remoteAddress = getRemoteAddress(channel); + logger.error("Still have {} requests outstanding when connection from {} is closed", + numOutstandingRequests(), remoteAddress); + failOutstandingRequests(new IOException("Connection from " + remoteAddress + " closed")); + } + } + + @Override + public void exceptionCaught(Throwable cause) { + if (numOutstandingRequests() > 0) { + String remoteAddress = getRemoteAddress(channel); + logger.error("Still have {} requests outstanding when connection from {} is closed", + numOutstandingRequests(), remoteAddress); + failOutstandingRequests(cause); + } + } + + @Override + public void handle(ResponseMessage message) throws Exception { + if (message instanceof ChunkFetchSuccess) { + ChunkFetchSuccess resp = (ChunkFetchSuccess) message; + ChunkReceivedCallback listener = outstandingFetches.get(resp.streamChunkId); + if (listener == null) { + logger.warn("Ignoring response for block {} from {} since it is not outstanding", + resp.streamChunkId, getRemoteAddress(channel)); + resp.body().release(); + } else { + outstandingFetches.remove(resp.streamChunkId); + listener.onSuccess(resp.streamChunkId.chunkIndex, resp.body()); + resp.body().release(); + } + } else if (message instanceof ChunkFetchFailure) { + ChunkFetchFailure resp = (ChunkFetchFailure) message; + ChunkReceivedCallback listener = outstandingFetches.get(resp.streamChunkId); + if (listener == null) { + logger.warn("Ignoring response for block {} from {} ({}) since it is not outstanding", + resp.streamChunkId, getRemoteAddress(channel), resp.errorString); + } else { + outstandingFetches.remove(resp.streamChunkId); + listener.onFailure(resp.streamChunkId.chunkIndex, new ChunkFetchFailureException( + "Failure while fetching " + resp.streamChunkId + ": " + resp.errorString)); + } + } else if (message instanceof RpcResponse) { + RpcResponse resp = (RpcResponse) message; + RpcResponseCallback listener = outstandingRpcs.get(resp.requestId); + if (listener == null) { + logger.warn("Ignoring response for RPC {} from {} ({} bytes) since it is not outstanding", + resp.requestId, getRemoteAddress(channel), resp.body().size()); + resp.body().release(); + } else { + outstandingRpcs.remove(resp.requestId); + try { + listener.onSuccess(resp.body().nioByteBuffer()); + } finally { + resp.body().release(); + } + } + } else if (message instanceof RpcFailure) { + RpcFailure resp = (RpcFailure) message; + RpcResponseCallback listener = outstandingRpcs.get(resp.requestId); + if (listener == null) { + logger.warn("Ignoring response for RPC {} from {} ({}) since it is not outstanding", + resp.requestId, getRemoteAddress(channel), resp.errorString); + } else { + outstandingRpcs.remove(resp.requestId); + listener.onFailure(new RuntimeException(resp.errorString)); + } + } else if (message instanceof StreamResponse) { + StreamResponse resp = (StreamResponse) message; + Pair entry = streamCallbacks.poll(); + if (entry != null) { + StreamCallback callback = entry.getValue(); + if (resp.byteCount > 0) { + StreamInterceptor interceptor = new StreamInterceptor<>( + this, resp.streamId, resp.byteCount, callback); + try { + TransportFrameDecoder frameDecoder = (TransportFrameDecoder) + channel.pipeline().get(TransportFrameDecoder.HANDLER_NAME); + frameDecoder.setInterceptor(interceptor); + streamActive = true; + } catch (Exception e) { + logger.error("Error installing stream handler.", e); + deactivateStream(); + } + } else { + try { + callback.onComplete(resp.streamId); + } catch (Exception e) { + logger.warn("Error in stream handler onComplete().", e); + } + } + } else { + logger.error("Could not find callback for StreamResponse."); + } + } else if (message instanceof StreamFailure) { + StreamFailure resp = (StreamFailure) message; + Pair entry = streamCallbacks.poll(); + if (entry != null) { + StreamCallback callback = entry.getValue(); + try { + callback.onFailure(resp.streamId, new RuntimeException(resp.error)); + } catch (IOException ioe) { + logger.warn("Error in stream failure handler.", ioe); + } + } else { + logger.warn("Stream failure with unknown callback: {}", resp.error); + } + } else { + throw new IllegalStateException("Unknown response type: " + message.type()); + } + } + + /** Returns total number of outstanding requests (fetch requests + rpcs) */ + public int numOutstandingRequests() { + return outstandingFetches.size() + outstandingRpcs.size() + streamCallbacks.size() + + (streamActive ? 1 : 0); + } + + /** Returns the time in nanoseconds of when the last request was sent out. */ + public long getTimeOfLastRequestNs() { + return timeOfLastRequestNs.get(); + } + + /** Updates the time of the last request to the current system time. */ + public void updateTimeOfLastRequest() { + timeOfLastRequestNs.set(System.nanoTime()); + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/crypto/AuthClientBootstrap.java b/network-common/src/main/java/com/bytedance/css/network/crypto/AuthClientBootstrap.java new file mode 100644 index 0000000..bc0fa33 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/crypto/AuthClientBootstrap.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.crypto; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.security.GeneralSecurityException; + +import com.google.common.base.Throwables; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.client.TransportClientBootstrap; +import com.bytedance.css.network.sasl.SaslClientBootstrap; +import com.bytedance.css.network.sasl.SecretKeyHolder; +import com.bytedance.css.network.util.TransportConf; + +/** + * Bootstraps a {@link TransportClient} by performing authentication using Spark's auth protocol. + * + * This bootstrap falls back to using the SASL bootstrap if the server throws an error during + * authentication, and the configuration allows it. This is used for backwards compatibility + * with external shuffle services that do not support the new protocol. + * + * It also automatically falls back to SASL if the new encryption backend is disabled, so that + * callers only need to install this bootstrap when authentication is enabled. + */ +public class AuthClientBootstrap implements TransportClientBootstrap { + + private static final Logger LOG = LoggerFactory.getLogger(AuthClientBootstrap.class); + + private final TransportConf conf; + private final String appId; + private final SecretKeyHolder secretKeyHolder; + + public AuthClientBootstrap( + TransportConf conf, + String appId, + SecretKeyHolder secretKeyHolder) { + this.conf = conf; + // TODO: right now this behaves like the SASL backend, because when executors start up + // they don't necessarily know the app ID. So they send a hardcoded "user" that is defined + // in the SecurityManager, which will also always return the same secret (regardless of the + // user name). All that's needed here is for this "user" to match on both sides, since that's + // required by the protocol. At some point, though, it would be better for the actual app ID + // to be provided here. + this.appId = appId; + this.secretKeyHolder = secretKeyHolder; + } + + @Override + public void doBootstrap(TransportClient client, Channel channel) { + if (!conf.encryptionEnabled()) { + LOG.debug("AES encryption disabled, using old auth protocol."); + doSaslAuth(client, channel); + return; + } + + try { + doSparkAuth(client, channel); + client.setClientId(appId); + } catch (GeneralSecurityException | IOException e) { + throw Throwables.propagate(e); + } catch (RuntimeException e) { + // There isn't a good exception that can be caught here to know whether it's really + // OK to switch back to SASL (because the server doesn't speak the new protocol). So + // try it anyway, and in the worst case things will fail again. + if (conf.saslFallback()) { + LOG.warn("New auth protocol failed, trying SASL.", e); + doSaslAuth(client, channel); + } else { + throw e; + } + } + } + + private void doSparkAuth(TransportClient client, Channel channel) + throws GeneralSecurityException, IOException { + + String secretKey = secretKeyHolder.getSecretKey(appId); + try (AuthEngine engine = new AuthEngine(appId, secretKey, conf)) { + ClientChallenge challenge = engine.challenge(); + ByteBuf challengeData = Unpooled.buffer(challenge.encodedLength()); + challenge.encode(challengeData); + + ByteBuffer responseData = + client.sendRpcSync(challengeData.nioBuffer(), conf.authRTTimeoutMs()); + ServerResponse response = ServerResponse.decodeMessage(responseData); + + engine.validate(response); + engine.sessionCipher().addToChannel(channel); + } + } + + private void doSaslAuth(TransportClient client, Channel channel) { + SaslClientBootstrap sasl = new SaslClientBootstrap(conf, appId, secretKeyHolder); + sasl.doBootstrap(client, channel); + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/crypto/AuthEngine.java b/network-common/src/main/java/com/bytedance/css/network/crypto/AuthEngine.java new file mode 100644 index 0000000..dc419a8 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/crypto/AuthEngine.java @@ -0,0 +1,317 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.crypto; + +import java.io.Closeable; +import java.io.IOException; +import java.math.BigInteger; +import java.security.GeneralSecurityException; +import java.util.Arrays; +import java.util.Properties; +import javax.crypto.Cipher; +import javax.crypto.SecretKey; +import javax.crypto.SecretKeyFactory; +import javax.crypto.ShortBufferException; +import javax.crypto.spec.IvParameterSpec; +import javax.crypto.spec.PBEKeySpec; +import javax.crypto.spec.SecretKeySpec; +import static java.nio.charset.StandardCharsets.UTF_8; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.primitives.Bytes; +import org.apache.commons.crypto.cipher.CryptoCipher; +import org.apache.commons.crypto.cipher.CryptoCipherFactory; +import org.apache.commons.crypto.random.CryptoRandom; +import org.apache.commons.crypto.random.CryptoRandomFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.bytedance.css.network.util.TransportConf; + +/** + * A helper class for abstracting authentication and key negotiation details. This is used by + * both client and server sides, since the operations are basically the same. + */ +class AuthEngine implements Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(AuthEngine.class); + private static final BigInteger ONE = new BigInteger(new byte[] { 0x1 }); + + private final byte[] appId; + private final char[] secret; + private final TransportConf conf; + private final Properties cryptoConf; + private final CryptoRandom random; + + private byte[] authNonce; + + @VisibleForTesting + byte[] challenge; + + private TransportCipher sessionCipher; + private CryptoCipher encryptor; + private CryptoCipher decryptor; + + AuthEngine(String appId, String secret, TransportConf conf) throws GeneralSecurityException { + this.appId = appId.getBytes(UTF_8); + this.conf = conf; + this.cryptoConf = conf.cryptoConf(); + this.secret = secret.toCharArray(); + this.random = CryptoRandomFactory.getCryptoRandom(cryptoConf); + } + + /** + * Create the client challenge. + * + * @return A challenge to be sent the remote side. + */ + ClientChallenge challenge() throws GeneralSecurityException { + this.authNonce = randomBytes(conf.encryptionKeyLength() / Byte.SIZE); + SecretKeySpec authKey = generateKey(conf.keyFactoryAlgorithm(), conf.keyFactoryIterations(), + authNonce, conf.encryptionKeyLength()); + initializeForAuth(conf.cipherTransformation(), authNonce, authKey); + + this.challenge = randomBytes(conf.encryptionKeyLength() / Byte.SIZE); + return new ClientChallenge(new String(appId, UTF_8), + conf.keyFactoryAlgorithm(), + conf.keyFactoryIterations(), + conf.cipherTransformation(), + conf.encryptionKeyLength(), + authNonce, + challenge(appId, authNonce, challenge)); + } + + /** + * Validates the client challenge, and create the encryption backend for the channel from the + * parameters sent by the client. + * + * @param clientChallenge The challenge from the client. + * @return A response to be sent to the client. + */ + ServerResponse respond(ClientChallenge clientChallenge) + throws GeneralSecurityException { + + SecretKeySpec authKey = generateKey(clientChallenge.kdf, clientChallenge.iterations, + clientChallenge.nonce, clientChallenge.keyLength); + initializeForAuth(clientChallenge.cipher, clientChallenge.nonce, authKey); + + byte[] challenge = validateChallenge(clientChallenge.nonce, clientChallenge.challenge); + byte[] response = challenge(appId, clientChallenge.nonce, rawResponse(challenge)); + byte[] sessionNonce = randomBytes(conf.encryptionKeyLength() / Byte.SIZE); + byte[] inputIv = randomBytes(conf.ivLength()); + byte[] outputIv = randomBytes(conf.ivLength()); + + SecretKeySpec sessionKey = generateKey(clientChallenge.kdf, clientChallenge.iterations, + sessionNonce, clientChallenge.keyLength); + this.sessionCipher = new TransportCipher(cryptoConf, clientChallenge.cipher, sessionKey, + inputIv, outputIv); + + // Note the IVs are swapped in the response. + return new ServerResponse(response, encrypt(sessionNonce), encrypt(outputIv), encrypt(inputIv)); + } + + /** + * Validates the server response and initializes the cipher to use for the session. + * + * @param serverResponse The response from the server. + */ + void validate(ServerResponse serverResponse) throws GeneralSecurityException { + byte[] response = validateChallenge(authNonce, serverResponse.response); + + byte[] expected = rawResponse(challenge); + Preconditions.checkArgument(Arrays.equals(expected, response)); + + byte[] nonce = decrypt(serverResponse.nonce); + byte[] inputIv = decrypt(serverResponse.inputIv); + byte[] outputIv = decrypt(serverResponse.outputIv); + + SecretKeySpec sessionKey = generateKey(conf.keyFactoryAlgorithm(), conf.keyFactoryIterations(), + nonce, conf.encryptionKeyLength()); + this.sessionCipher = new TransportCipher(cryptoConf, conf.cipherTransformation(), sessionKey, + inputIv, outputIv); + } + + TransportCipher sessionCipher() { + Preconditions.checkState(sessionCipher != null); + return sessionCipher; + } + + @Override + public void close() throws IOException { + // Close ciphers (by calling "doFinal()" with dummy data) and the random instance so that + // internal state is cleaned up. Error handling here is just for paranoia, and not meant to + // accurately report the errors when they happen. + RuntimeException error = null; + byte[] dummy = new byte[8]; + if (encryptor != null) { + try { + doCipherOp(Cipher.ENCRYPT_MODE, dummy, true); + } catch (Exception e) { + error = new RuntimeException(e); + } + encryptor = null; + } + if (decryptor != null) { + try { + doCipherOp(Cipher.DECRYPT_MODE, dummy, true); + } catch (Exception e) { + error = new RuntimeException(e); + } + decryptor = null; + } + random.close(); + + if (error != null) { + throw error; + } + } + + @VisibleForTesting + byte[] challenge(byte[] appId, byte[] nonce, byte[] challenge) throws GeneralSecurityException { + return encrypt(Bytes.concat(appId, nonce, challenge)); + } + + @VisibleForTesting + byte[] rawResponse(byte[] challenge) { + BigInteger orig = new BigInteger(challenge); + BigInteger response = orig.add(ONE); + return response.toByteArray(); + } + + private byte[] decrypt(byte[] in) throws GeneralSecurityException { + return doCipherOp(Cipher.DECRYPT_MODE, in, false); + } + + private byte[] encrypt(byte[] in) throws GeneralSecurityException { + return doCipherOp(Cipher.ENCRYPT_MODE, in, false); + } + + private void initializeForAuth(String cipher, byte[] nonce, SecretKeySpec key) + throws GeneralSecurityException { + + // commons-crypto currently only supports ciphers that require an initial vector; so + // create a dummy vector so that we can initialize the ciphers. In the future, if + // different ciphers are supported, this will have to be configurable somehow. + byte[] iv = new byte[conf.ivLength()]; + System.arraycopy(nonce, 0, iv, 0, Math.min(nonce.length, iv.length)); + + CryptoCipher _encryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf); + _encryptor.init(Cipher.ENCRYPT_MODE, key, new IvParameterSpec(iv)); + this.encryptor = _encryptor; + + CryptoCipher _decryptor = CryptoCipherFactory.getCryptoCipher(cipher, cryptoConf); + _decryptor.init(Cipher.DECRYPT_MODE, key, new IvParameterSpec(iv)); + this.decryptor = _decryptor; + } + + /** + * Validates an encrypted challenge as defined in the protocol, and returns the byte array + * that corresponds to the actual challenge data. + */ + private byte[] validateChallenge(byte[] nonce, byte[] encryptedChallenge) + throws GeneralSecurityException { + + byte[] challenge = decrypt(encryptedChallenge); + checkSubArray(appId, challenge, 0); + checkSubArray(nonce, challenge, appId.length); + return Arrays.copyOfRange(challenge, appId.length + nonce.length, challenge.length); + } + + private SecretKeySpec generateKey(String kdf, int iterations, byte[] salt, int keyLength) + throws GeneralSecurityException { + + SecretKeyFactory factory = SecretKeyFactory.getInstance(kdf); + PBEKeySpec spec = new PBEKeySpec(secret, salt, iterations, keyLength); + + long start = System.nanoTime(); + SecretKey key = factory.generateSecret(spec); + long end = System.nanoTime(); + + LOG.debug("Generated key with {} iterations in {} us.", conf.keyFactoryIterations(), + (end - start) / 1000); + + return new SecretKeySpec(key.getEncoded(), conf.keyAlgorithm()); + } + + private byte[] doCipherOp(int mode, byte[] in, boolean isFinal) + throws GeneralSecurityException { + + CryptoCipher cipher; + switch (mode) { + case Cipher.ENCRYPT_MODE: + cipher = encryptor; + break; + case Cipher.DECRYPT_MODE: + cipher = decryptor; + break; + default: + throw new IllegalArgumentException(String.valueOf(mode)); + } + + Preconditions.checkState(cipher != null, "Cipher is invalid because of previous error."); + + try { + int scale = 1; + while (true) { + int size = in.length * scale; + byte[] buffer = new byte[size]; + try { + int outSize = isFinal ? cipher.doFinal(in, 0, in.length, buffer, 0) + : cipher.update(in, 0, in.length, buffer, 0); + if (outSize != buffer.length) { + byte[] output = new byte[outSize]; + System.arraycopy(buffer, 0, output, 0, output.length); + return output; + } else { + return buffer; + } + } catch (ShortBufferException e) { + // Try again with a bigger buffer. + scale *= 2; + } + } + } catch (InternalError ie) { + // SPARK-25535. The commons-cryto library will throw InternalError if something goes wrong, + // and leave bad state behind in the Java wrappers, so it's not safe to use them afterwards. + if (mode == Cipher.ENCRYPT_MODE) { + this.encryptor = null; + } else { + this.decryptor = null; + } + throw ie; + } + } + + private byte[] randomBytes(int count) { + byte[] bytes = new byte[count]; + random.nextBytes(bytes); + return bytes; + } + + /** Checks that the "test" array is in the data array starting at the given offset. */ + private void checkSubArray(byte[] test, byte[] data, int offset) { + Preconditions.checkArgument(data.length >= test.length + offset); + for (int i = 0; i < test.length; i++) { + Preconditions.checkArgument(test[i] == data[i + offset]); + } + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/crypto/AuthRpcHandler.java b/network-common/src/main/java/com/bytedance/css/network/crypto/AuthRpcHandler.java new file mode 100644 index 0000000..17be86e --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/crypto/AuthRpcHandler.java @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.crypto; + +import java.nio.ByteBuffer; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.bytedance.css.network.client.RpcResponseCallback; +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.sasl.SecretKeyHolder; +import com.bytedance.css.network.sasl.SaslRpcHandler; +import com.bytedance.css.network.server.AbstractAuthRpcHandler; +import com.bytedance.css.network.server.RpcHandler; +import com.bytedance.css.network.util.TransportConf; + +/** + * RPC Handler which performs authentication using Spark's auth protocol before delegating to a + * child RPC handler. If the configuration allows, this handler will delegate messages to a SASL + * RPC handler for further authentication, to support for clients that do not support Spark's + * protocol. + * + * The delegate will only receive messages if the given connection has been successfully + * authenticated. A connection may be authenticated at most once. + */ +class AuthRpcHandler extends AbstractAuthRpcHandler { + private static final Logger LOG = LoggerFactory.getLogger(AuthRpcHandler.class); + + /** Transport configuration. */ + private final TransportConf conf; + + /** The client channel. */ + private final Channel channel; + + /** Class which provides secret keys which are shared by server and client on a per-app basis. */ + private final SecretKeyHolder secretKeyHolder; + + /** RPC handler for auth handshake when falling back to SASL auth. */ + @VisibleForTesting + SaslRpcHandler saslHandler; + + AuthRpcHandler( + TransportConf conf, + Channel channel, + RpcHandler delegate, + SecretKeyHolder secretKeyHolder) { + super(delegate); + this.conf = conf; + this.channel = channel; + this.secretKeyHolder = secretKeyHolder; + } + + @Override + protected boolean doAuthChallenge( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { + if (saslHandler != null) { + return saslHandler.doAuthChallenge(client, message, callback); + } + + int position = message.position(); + int limit = message.limit(); + + ClientChallenge challenge; + try { + challenge = ClientChallenge.decodeMessage(message); + LOG.debug("Received new auth challenge for client {}.", channel.remoteAddress()); + } catch (RuntimeException e) { + if (conf.saslFallback()) { + LOG.warn("Failed to parse new auth challenge, reverting to SASL for client {}.", + channel.remoteAddress()); + saslHandler = new SaslRpcHandler(conf, channel, null, secretKeyHolder); + message.position(position); + message.limit(limit); + return saslHandler.doAuthChallenge(client, message, callback); + } else { + LOG.debug("Unexpected challenge message from client {}, closing channel.", + channel.remoteAddress()); + callback.onFailure(new IllegalArgumentException("Unknown challenge message.")); + channel.close(); + } + return false; + } + + // Here we have the client challenge, so perform the new auth protocol and set up the channel. + AuthEngine engine = null; + try { + String secret = secretKeyHolder.getSecretKey(challenge.appId); + Preconditions.checkState(secret != null, + "Trying to authenticate non-registered app %s.", challenge.appId); + LOG.debug("Authenticating challenge for app {}.", challenge.appId); + engine = new AuthEngine(challenge.appId, secret, conf); + ServerResponse response = engine.respond(challenge); + ByteBuf responseData = Unpooled.buffer(response.encodedLength()); + response.encode(responseData); + callback.onSuccess(responseData.nioBuffer()); + engine.sessionCipher().addToChannel(channel); + client.setClientId(challenge.appId); + } catch (Exception e) { + // This is a fatal error: authentication has failed. Close the channel explicitly. + LOG.debug("Authentication failed for client {}, closing channel.", channel.remoteAddress()); + callback.onFailure(new IllegalArgumentException("Authentication failed.")); + channel.close(); + return false; + } finally { + if (engine != null) { + try { + engine.close(); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + } + + LOG.debug("Authorization successful for client {}.", channel.remoteAddress()); + return true; + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/crypto/AuthServerBootstrap.java b/network-common/src/main/java/com/bytedance/css/network/crypto/AuthServerBootstrap.java new file mode 100644 index 0000000..7372296 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/crypto/AuthServerBootstrap.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.crypto; + +import io.netty.channel.Channel; + +import com.bytedance.css.network.sasl.SaslServerBootstrap; +import com.bytedance.css.network.sasl.SecretKeyHolder; +import com.bytedance.css.network.server.RpcHandler; +import com.bytedance.css.network.server.TransportServerBootstrap; +import com.bytedance.css.network.util.TransportConf; + +/** + * A bootstrap which is executed on a TransportServer's client channel once a client connects + * to the server, enabling authentication using Spark's auth protocol (and optionally SASL for + * clients that don't support the new protocol). + * + * It also automatically falls back to SASL if the new encryption backend is disabled, so that + * callers only need to install this bootstrap when authentication is enabled. + */ +public class AuthServerBootstrap implements TransportServerBootstrap { + + private final TransportConf conf; + private final SecretKeyHolder secretKeyHolder; + + public AuthServerBootstrap(TransportConf conf, SecretKeyHolder secretKeyHolder) { + this.conf = conf; + this.secretKeyHolder = secretKeyHolder; + } + + public RpcHandler doBootstrap(Channel channel, RpcHandler rpcHandler) { + if (!conf.encryptionEnabled()) { + TransportServerBootstrap sasl = new SaslServerBootstrap(conf, secretKeyHolder); + return sasl.doBootstrap(channel, rpcHandler); + } + + return new AuthRpcHandler(conf, channel, rpcHandler, secretKeyHolder); + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/crypto/ClientChallenge.java b/network-common/src/main/java/com/bytedance/css/network/crypto/ClientChallenge.java new file mode 100644 index 0000000..53d634b --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/crypto/ClientChallenge.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.crypto; + +import java.nio.ByteBuffer; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; + +import com.bytedance.css.network.protocol.Encodable; +import com.bytedance.css.network.protocol.Encoders; + +/** + * The client challenge message, used to initiate authentication. + * + * Please see crypto/README.md for more details of implementation. + */ +public class ClientChallenge implements Encodable { + /** Serialization tag used to catch incorrect payloads. */ + private static final byte TAG_BYTE = (byte) 0xFA; + + public final String appId; + public final String kdf; + public final int iterations; + public final String cipher; + public final int keyLength; + public final byte[] nonce; + public final byte[] challenge; + + public ClientChallenge( + String appId, + String kdf, + int iterations, + String cipher, + int keyLength, + byte[] nonce, + byte[] challenge) { + this.appId = appId; + this.kdf = kdf; + this.iterations = iterations; + this.cipher = cipher; + this.keyLength = keyLength; + this.nonce = nonce; + this.challenge = challenge; + } + + @Override + public int encodedLength() { + return 1 + 4 + 4 + + Encoders.Strings.encodedLength(appId) + + Encoders.Strings.encodedLength(kdf) + + Encoders.Strings.encodedLength(cipher) + + Encoders.ByteArrays.encodedLength(nonce) + + Encoders.ByteArrays.encodedLength(challenge); + } + + @Override + public void encode(ByteBuf buf) { + buf.writeByte(TAG_BYTE); + Encoders.Strings.encode(buf, appId); + Encoders.Strings.encode(buf, kdf); + buf.writeInt(iterations); + Encoders.Strings.encode(buf, cipher); + buf.writeInt(keyLength); + Encoders.ByteArrays.encode(buf, nonce); + Encoders.ByteArrays.encode(buf, challenge); + } + + public static ClientChallenge decodeMessage(ByteBuffer buffer) { + ByteBuf buf = Unpooled.wrappedBuffer(buffer); + + if (buf.readByte() != TAG_BYTE) { + throw new IllegalArgumentException("Expected ClientChallenge, received something else."); + } + + return new ClientChallenge( + Encoders.Strings.decode(buf), + Encoders.Strings.decode(buf), + buf.readInt(), + Encoders.Strings.decode(buf), + buf.readInt(), + Encoders.ByteArrays.decode(buf), + Encoders.ByteArrays.decode(buf)); + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/crypto/README.md b/network-common/src/main/java/com/bytedance/css/network/crypto/README.md new file mode 100644 index 0000000..14df703 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/crypto/README.md @@ -0,0 +1,158 @@ +Spark Auth Protocol and AES Encryption Support +============================================== + +This file describes an auth protocol used by Spark as a more secure alternative to DIGEST-MD5. This +protocol is built on symmetric key encryption, based on the assumption that the two endpoints being +authenticated share a common secret, which is how Spark authentication currently works. The protocol +provides mutual authentication, meaning that after the negotiation both parties know that the remote +side knows the shared secret. The protocol is influenced by the ISO/IEC 9798 protocol, although it's +not an implementation of it. + +This protocol could be replaced with TLS PSK, except no PSK ciphers are available in the currently +released JREs. + +The protocol aims at solving the following shortcomings in Spark's current usage of DIGEST-MD5: + +- MD5 is an aging hash algorithm with known weaknesses, and a more secure alternative is desired. +- DIGEST-MD5 has a pre-defined set of ciphers for which it can generate keys. The only + viable, supported cipher these days is 3DES, and a more modern alternative is desired. +- Encrypting AES session keys with 3DES doesn't solve the issue, since the weakest link + in the negotiation would still be MD5 and 3DES. + +The protocol assumes that the shared secret is generated and distributed in a secure manner. + +The protocol always negotiates encryption keys. If encryption is not desired, the existing +SASL-based authentication, or no authentication at all, can be chosen instead. + +When messages are described below, it's expected that the implementation should support +arbitrary sizes for fields that don't have a fixed size. + +Client Challenge +---------------- + +The auth negotiation is started by the client. The client starts by generating an encryption +key based on the application's shared secret, and a nonce. + + KEY = KDF(SECRET, SALT, KEY_LENGTH) + +Where: +- KDF(): a key derivation function that takes a secret, a salt, a configurable number of + iterations, and a configurable key length. +- SALT: a byte sequence used to salt the key derivation function. +- KEY_LENGTH: length of the encryption key to generate. + + +The client generates a message with the following content: + + CLIENT_CHALLENGE = ( + APP_ID, + KDF, + ITERATIONS, + CIPHER, + KEY_LENGTH, + ANONCE, + ENC(APP_ID || ANONCE || CHALLENGE)) + +Where: + +- APP_ID: the application ID which the server uses to identify the shared secret. +- KDF: the key derivation function described above. +- ITERATIONS: number of iterations to run the KDF when generating keys. +- CIPHER: the cipher used to encrypt data. +- KEY_LENGTH: length of the encryption keys to generate, in bits. +- ANONCE: the nonce used as the salt when generating the auth key. +- ENC(): an encryption function that uses the cipher and the generated key. This function + will also be used in the definition of other messages below. +- CHALLENGE: a byte sequence used as a challenge to the server. +- ||: concatenation operator. + +When strings are used where byte arrays are expected, the UTF-8 representation of the string +is assumed. + +To respond to the challenge, the server should consider the byte array as representing an +arbitrary-length integer, and respond with the value of the integer plus one. + + +Server Response And Challenge +----------------------------- + +Once the client challenge is received, the server will generate the same auth key by +using the same algorithm the client has used. It will then verify the client challenge: +if the APP_ID and ANONCE fields match, the server knows that the client has the shared +secret. The server then creates a response to the client challenge, to prove that it also +has the secret key, and provides parameters to be used when creating the session key. + +The following describes the response from the server: + + SERVER_CHALLENGE = ( + ENC(APP_ID || ANONCE || RESPONSE), + ENC(SNONCE), + ENC(INIV), + ENC(OUTIV)) + +Where: + +- RESPONSE: the server's response to the client challenge. +- SNONCE: a nonce to be used as salt when generating the session key. +- INIV: initialization vector used to initialize the input channel of the client. +- OUTIV: initialization vector used to initialize the output channel of the client. + +At this point the server considers the client to be authenticated, and will try to +decrypt any data further sent by the client using the session key. + + +Default Algorithms +------------------ + +Configuration options are available for the KDF and cipher algorithms to use. + +The default KDF is "PBKDF2WithHmacSHA1". Users should be able to select any algorithm +from those supported by the `javax.crypto.SecretKeyFactory` class, as long as they support +PBEKeySpec when generating keys. The default number of iterations was chosen to take a +reasonable amount of time on modern CPUs. See the documentation in TransportConf for more +details. + +The default cipher algorithm is "AES/CTR/NoPadding". Users should be able to select any +algorithm supported by the commons-crypto library. It should allow the cipher to operate +in stream mode. + +The default key length is 128 (bits). + + +Implementation Details +---------------------- + +The commons-crypto library currently only supports AES ciphers, and requires an initialization +vector (IV). This first version of the protocol does not explicitly include the IV in the client +challenge message. Instead, the IV should be derived from the nonce, including the needed bytes, and +padding the IV with zeroes in case the nonce is not long enough. + +Future versions of the protocol might add support for new ciphers and explicitly include needed +configuration parameters in the messages. + + +Threat Assessment +----------------- + +The protocol is secure against different forms of attack: + +* Eavesdropping: the protocol is built on the assumption that it's computationally infeasible + to calculate the original secret from the encrypted messages. Neither the secret nor any + encryption keys are transmitted on the wire, encrypted or not. + +* Man-in-the-middle: because the protocol performs mutual authentication, both ends need to + know the shared secret to be able to decrypt session data. Even if an attacker is able to insert a + malicious "proxy" between endpoints, the attacker won't be able to read any of the data exchanged + between client and server, nor insert arbitrary commands for the server to execute. + +* Replay attacks: the use of nonces when generating keys prevents an attacker from being able to + just replay messages sniffed from the communication channel. + +An attacker may replay the client challenge and successfully "prove" to a server that it "knows" the +shared secret. But the attacker won't be able to decrypt the server's response, and thus won't be +able to generate a session key, which will make it hard to craft a valid, encrypted message that the +server will be able to understand. This will cause the server to close the connection as soon as the +attacker tries to send any command to the server. The attacker can just hold the channel open for +some time, which will be closed when the server times out the channel. These issues could be +separately mitigated by adding a shorter timeout for the first message after authentication, and +potentially by adding host blacklists if a possible attack is detected from a particular host. diff --git a/network-common/src/main/java/com/bytedance/css/network/crypto/ServerResponse.java b/network-common/src/main/java/com/bytedance/css/network/crypto/ServerResponse.java new file mode 100644 index 0000000..72dca74 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/crypto/ServerResponse.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.crypto; + +import java.nio.ByteBuffer; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; + +import com.bytedance.css.network.protocol.Encodable; +import com.bytedance.css.network.protocol.Encoders; + +/** + * Server's response to client's challenge. + * + * Please see crypto/README.md for more details. + */ +public class ServerResponse implements Encodable { + /** Serialization tag used to catch incorrect payloads. */ + private static final byte TAG_BYTE = (byte) 0xFB; + + public final byte[] response; + public final byte[] nonce; + public final byte[] inputIv; + public final byte[] outputIv; + + public ServerResponse( + byte[] response, + byte[] nonce, + byte[] inputIv, + byte[] outputIv) { + this.response = response; + this.nonce = nonce; + this.inputIv = inputIv; + this.outputIv = outputIv; + } + + @Override + public int encodedLength() { + return 1 + + Encoders.ByteArrays.encodedLength(response) + + Encoders.ByteArrays.encodedLength(nonce) + + Encoders.ByteArrays.encodedLength(inputIv) + + Encoders.ByteArrays.encodedLength(outputIv); + } + + @Override + public void encode(ByteBuf buf) { + buf.writeByte(TAG_BYTE); + Encoders.ByteArrays.encode(buf, response); + Encoders.ByteArrays.encode(buf, nonce); + Encoders.ByteArrays.encode(buf, inputIv); + Encoders.ByteArrays.encode(buf, outputIv); + } + + public static ServerResponse decodeMessage(ByteBuffer buffer) { + ByteBuf buf = Unpooled.wrappedBuffer(buffer); + + if (buf.readByte() != TAG_BYTE) { + throw new IllegalArgumentException("Expected ServerResponse, received something else."); + } + + return new ServerResponse( + Encoders.ByteArrays.decode(buf), + Encoders.ByteArrays.decode(buf), + Encoders.ByteArrays.decode(buf), + Encoders.ByteArrays.decode(buf)); + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/crypto/TransportCipher.java b/network-common/src/main/java/com/bytedance/css/network/crypto/TransportCipher.java new file mode 100644 index 0000000..3374c53 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/crypto/TransportCipher.java @@ -0,0 +1,377 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.crypto; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.WritableByteChannel; +import java.util.Properties; +import javax.crypto.spec.SecretKeySpec; +import javax.crypto.spec.IvParameterSpec; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.*; +import org.apache.commons.crypto.stream.CryptoInputStream; +import org.apache.commons.crypto.stream.CryptoOutputStream; + +import com.bytedance.css.network.util.AbstractFileRegion; +import com.bytedance.css.network.util.ByteArrayReadableChannel; +import com.bytedance.css.network.util.ByteArrayWritableChannel; + +/** + * Cipher for encryption and decryption. + */ +public class TransportCipher { + @VisibleForTesting + static final String ENCRYPTION_HANDLER_NAME = "TransportEncryption"; + private static final String DECRYPTION_HANDLER_NAME = "TransportDecryption"; + @VisibleForTesting + static final int STREAM_BUFFER_SIZE = 1024 * 32; + + private final Properties conf; + private final String cipher; + private final SecretKeySpec key; + private final byte[] inIv; + private final byte[] outIv; + + public TransportCipher( + Properties conf, + String cipher, + SecretKeySpec key, + byte[] inIv, + byte[] outIv) { + this.conf = conf; + this.cipher = cipher; + this.key = key; + this.inIv = inIv; + this.outIv = outIv; + } + + public String getCipherTransformation() { + return cipher; + } + + @VisibleForTesting + SecretKeySpec getKey() { + return key; + } + + /** The IV for the input channel (i.e. output channel of the remote side). */ + public byte[] getInputIv() { + return inIv; + } + + /** The IV for the output channel (i.e. input channel of the remote side). */ + public byte[] getOutputIv() { + return outIv; + } + + @VisibleForTesting + CryptoOutputStream createOutputStream(WritableByteChannel ch) throws IOException { + return new CryptoOutputStream(cipher, conf, ch, key, new IvParameterSpec(outIv)); + } + + @VisibleForTesting + CryptoInputStream createInputStream(ReadableByteChannel ch) throws IOException { + return new CryptoInputStream(cipher, conf, ch, key, new IvParameterSpec(inIv)); + } + + /** + * Add handlers to channel. + * + * @param ch the channel for adding handlers + * @throws IOException + */ + public void addToChannel(Channel ch) throws IOException { + ch.pipeline() + .addFirst(ENCRYPTION_HANDLER_NAME, new EncryptionHandler(this)) + .addFirst(DECRYPTION_HANDLER_NAME, new DecryptionHandler(this)); + } + + @VisibleForTesting + static class EncryptionHandler extends ChannelOutboundHandlerAdapter { + private final ByteArrayWritableChannel byteChannel; + private final CryptoOutputStream cos; + private boolean isCipherValid; + + EncryptionHandler(TransportCipher cipher) throws IOException { + byteChannel = new ByteArrayWritableChannel(STREAM_BUFFER_SIZE); + cos = cipher.createOutputStream(byteChannel); + isCipherValid = true; + } + + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) + throws Exception { + ctx.write(createEncryptedMessage(msg), promise); + } + + @VisibleForTesting + EncryptedMessage createEncryptedMessage(Object msg) { + return new EncryptedMessage(this, cos, msg, byteChannel); + } + + @Override + public void close(ChannelHandlerContext ctx, ChannelPromise promise) throws Exception { + try { + if (isCipherValid) { + cos.close(); + } + } finally { + super.close(ctx, promise); + } + } + + /** + * SPARK-25535. Workaround for CRYPTO-141. Avoid further interaction with the underlying cipher + * after an error occurs. + */ + void reportError() { + this.isCipherValid = false; + } + + boolean isCipherValid() { + return isCipherValid; + } + } + + private static class DecryptionHandler extends ChannelInboundHandlerAdapter { + private final CryptoInputStream cis; + private final ByteArrayReadableChannel byteChannel; + private boolean isCipherValid; + + DecryptionHandler(TransportCipher cipher) throws IOException { + byteChannel = new ByteArrayReadableChannel(); + cis = cipher.createInputStream(byteChannel); + isCipherValid = true; + } + + @Override + public void channelRead(ChannelHandlerContext ctx, Object data) throws Exception { + ByteBuf buffer = (ByteBuf) data; + + try { + if (!isCipherValid) { + throw new IOException("Cipher is in invalid state."); + } + byte[] decryptedData = new byte[buffer.readableBytes()]; + byteChannel.feedData(buffer); + + int offset = 0; + while (offset < decryptedData.length) { + // SPARK-25535: workaround for CRYPTO-141. + try { + offset += cis.read(decryptedData, offset, decryptedData.length - offset); + } catch (InternalError ie) { + isCipherValid = false; + throw ie; + } + } + + ctx.fireChannelRead(Unpooled.wrappedBuffer(decryptedData, 0, decryptedData.length)); + } finally { + buffer.release(); + } + } + + @Override + public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { + // We do the closing of the stream / channel in handlerRemoved(...) as + // this method will be called in all cases: + // + // - when the Channel becomes inactive + // - when the handler is removed from the ChannelPipeline + try { + if (isCipherValid) { + cis.close(); + } + } finally { + super.handlerRemoved(ctx); + } + } + } + + @VisibleForTesting + static class EncryptedMessage extends AbstractFileRegion { + private final boolean isByteBuf; + private final ByteBuf buf; + private final FileRegion region; + private final long count; + private final CryptoOutputStream cos; + private final EncryptionHandler handler; + private long transferred; + + // Due to streaming issue CRYPTO-125: https://issues.apache.org/jira/browse/CRYPTO-125, it has + // to utilize two helper ByteArrayWritableChannel for streaming. One is used to receive raw data + // from upper handler, another is used to store encrypted data. + private ByteArrayWritableChannel byteEncChannel; + private ByteArrayWritableChannel byteRawChannel; + + private ByteBuffer currentEncrypted; + + EncryptedMessage( + EncryptionHandler handler, + CryptoOutputStream cos, + Object msg, + ByteArrayWritableChannel ch) { + Preconditions.checkArgument(msg instanceof ByteBuf || msg instanceof FileRegion, + "Unrecognized message type: %s", msg.getClass().getName()); + this.handler = handler; + this.isByteBuf = msg instanceof ByteBuf; + this.buf = isByteBuf ? (ByteBuf) msg : null; + this.region = isByteBuf ? null : (FileRegion) msg; + this.transferred = 0; + this.byteRawChannel = new ByteArrayWritableChannel(STREAM_BUFFER_SIZE); + this.cos = cos; + this.byteEncChannel = ch; + this.count = isByteBuf ? buf.readableBytes() : region.count(); + } + + @Override + public long count() { + return count; + } + + @Override + public long position() { + return 0; + } + + @Override + public long transferred() { + return transferred; + } + + @Override + public EncryptedMessage touch(Object o) { + super.touch(o); + if (region != null) { + region.touch(o); + } + if (buf != null) { + buf.touch(o); + } + return this; + } + + @Override + public EncryptedMessage retain(int increment) { + super.retain(increment); + if (region != null) { + region.retain(increment); + } + if (buf != null) { + buf.retain(increment); + } + return this; + } + + @Override + public boolean release(int decrement) { + if (region != null) { + region.release(decrement); + } + if (buf != null) { + buf.release(decrement); + } + return super.release(decrement); + } + + @Override + public long transferTo(WritableByteChannel target, long position) throws IOException { + Preconditions.checkArgument(position == transferred(), "Invalid position."); + + if (transferred == count) { + return 0; + } + + long totalBytesWritten = 0L; + do { + if (currentEncrypted == null) { + encryptMore(); + } + + long remaining = currentEncrypted.remaining(); + if (remaining == 0) { + // Just for safety to avoid endless loop. It usually won't happen, but since the + // underlying `region.transferTo` is allowed to transfer 0 bytes, we should handle it for + // safety. + currentEncrypted = null; + byteEncChannel.reset(); + return totalBytesWritten; + } + + long bytesWritten = target.write(currentEncrypted); + totalBytesWritten += bytesWritten; + transferred += bytesWritten; + if (bytesWritten < remaining) { + // break as the underlying buffer in "target" is full + break; + } + currentEncrypted = null; + byteEncChannel.reset(); + } while (transferred < count); + + return totalBytesWritten; + } + + private void encryptMore() throws IOException { + if (!handler.isCipherValid()) { + throw new IOException("Cipher is in invalid state."); + } + byteRawChannel.reset(); + + if (isByteBuf) { + int copied = byteRawChannel.write(buf.nioBuffer()); + buf.skipBytes(copied); + } else { + region.transferTo(byteRawChannel, region.transferred()); + } + + try { + cos.write(byteRawChannel.getData(), 0, byteRawChannel.length()); + cos.flush(); + } catch (InternalError ie) { + handler.reportError(); + throw ie; + } + + currentEncrypted = ByteBuffer.wrap(byteEncChannel.getData(), + 0, byteEncChannel.length()); + } + + @Override + protected void deallocate() { + byteRawChannel.reset(); + byteEncChannel.reset(); + if (region != null) { + region.release(); + } + if (buf != null) { + buf.release(); + } + } + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/AbstractMessage.java b/network-common/src/main/java/com/bytedance/css/network/protocol/AbstractMessage.java new file mode 100644 index 0000000..b483db3 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/AbstractMessage.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import com.google.common.base.Objects; + +import com.bytedance.css.network.buffer.ManagedBuffer; + +/** + * Abstract class for messages which optionally contain a body kept in a separate buffer. + */ +public abstract class AbstractMessage implements Message { + private final ManagedBuffer body; + private final boolean isBodyInFrame; + + protected AbstractMessage() { + this(null, false); + } + + protected AbstractMessage(ManagedBuffer body, boolean isBodyInFrame) { + this.body = body; + this.isBodyInFrame = isBodyInFrame; + } + + @Override + public ManagedBuffer body() { + return body; + } + + @Override + public boolean isBodyInFrame() { + return isBodyInFrame; + } + + protected boolean equals(AbstractMessage other) { + return isBodyInFrame == other.isBodyInFrame && Objects.equal(body, other.body); + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/AbstractResponseMessage.java b/network-common/src/main/java/com/bytedance/css/network/protocol/AbstractResponseMessage.java new file mode 100644 index 0000000..9e96d2c --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/AbstractResponseMessage.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import com.bytedance.css.network.buffer.ManagedBuffer; + +/** + * Abstract class for response messages. + */ +public abstract class AbstractResponseMessage extends AbstractMessage implements ResponseMessage { + + protected AbstractResponseMessage(ManagedBuffer body, boolean isBodyInFrame) { + super(body, isBodyInFrame); + } + + public abstract ResponseMessage createFailureResponse(String error); +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/BatchPushDataRequest.java b/network-common/src/main/java/com/bytedance/css/network/protocol/BatchPushDataRequest.java new file mode 100644 index 0000000..815ab0b --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/BatchPushDataRequest.java @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.buffer.NettyManagedBuffer; +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +public final class BatchPushDataRequest extends AbstractMessage implements RequestMessage { + + // applicationId-shuffleId + public final String shuffleKey; + public int[] reducerIds; + public int epochId; + public int[] offsets; + public int mapperId; + public long clientStartTime; + public long requestId; + + // extra header that need to be used in writer & partition lazy creation + public int replicaIndex; + public String shuffleMode; // 0 for disk 1 for hdfs + public String epochRotateThreshold; + + public BatchPushDataRequest( + String shuffleKey, int[] reducerIds, int epochId, int[] offsets, int mapperId, + int replicaIndex, String shuffleMode, String epochRotateThreshold, long clientStartTime, + ManagedBuffer body) { + this(shuffleKey, reducerIds, epochId, offsets, mapperId, replicaIndex, + shuffleMode, epochRotateThreshold, clientStartTime, 0L, body); + } + + public BatchPushDataRequest( + String shuffleKey, int[] reducerIds, int epochId, int[] offsets, int mapperId, + int replicaIndex, String shuffleMode, String epochRotateThreshold, long clientStartTime, + long requestId, ManagedBuffer body) { + super(body, true); + this.shuffleKey = shuffleKey; + this.reducerIds = reducerIds; + this.epochId = epochId; + this.offsets = offsets; + this.mapperId = mapperId; + this.shuffleMode = shuffleMode; + this.replicaIndex = replicaIndex; + this.epochRotateThreshold = epochRotateThreshold; + this.clientStartTime = clientStartTime; + this.requestId = requestId; + } + + @Override + public Type type() { + return Type.BatchPushDataRequest; + } + + @Override + public int encodedLength() { + return 8 + 8 + 4 + 4 + 4 + + Encoders.IntArrays.encodedLength(reducerIds) + + Encoders.IntArrays.encodedLength(offsets) + + Encoders.Strings.encodedLength(shuffleKey) + + Encoders.Strings.encodedLength(shuffleMode) + + Encoders.Strings.encodedLength(epochRotateThreshold); + } + + @Override + public void encode(ByteBuf buf) { + buf.writeLong(requestId); + Encoders.Strings.encode(buf, shuffleKey); + Encoders.IntArrays.encode(buf, reducerIds); + buf.writeInt(epochId); + Encoders.IntArrays.encode(buf, offsets); + buf.writeInt(mapperId); + buf.writeInt(replicaIndex); + Encoders.Strings.encode(buf, shuffleMode); + Encoders.Strings.encode(buf, epochRotateThreshold); + buf.writeLong(clientStartTime); + } + + public static BatchPushDataRequest decode(ByteBuf buf) { + long requestId = buf.readLong(); + String shuffleKey = Encoders.Strings.decode(buf); + int[] reducerIds = Encoders.IntArrays.decode(buf); + int epochId = buf.readInt(); + int[] offsets = Encoders.IntArrays.decode(buf); + int mapperId = buf.readInt(); + int replicaIndex = buf.readInt(); + String shuffleMode = Encoders.Strings.decode(buf); + String epochRotateThreshold = Encoders.Strings.decode(buf); + long clientStartTime = buf.readLong(); + return new BatchPushDataRequest(shuffleKey, reducerIds, epochId, offsets, mapperId, replicaIndex, + shuffleMode, epochRotateThreshold, clientStartTime, requestId, new NettyManagedBuffer(buf.retain())); + } + + @Override + public int hashCode() { + return Objects.hashCode(shuffleKey, reducerIds, epochId, offsets, mapperId, replicaIndex, requestId, body()); + } + + @Override + public boolean equals(Object other) { + if (other instanceof BatchPushDataRequest) { + BatchPushDataRequest o = (BatchPushDataRequest) other; + return requestId == o.requestId && super.equals(o); + } + return false; + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("shuffleKey", shuffleKey) + .append("reducerIds", reducerIds) + .append("epochId", epochId) + .append("offsets", offsets) + .append("mapperId", mapperId) + .append("replicaIndex", replicaIndex) + .append("clientStartTime", clientStartTime) + .append("requestId", requestId) + .append("shuffleMode", shuffleMode) + .append("epochRotateThreshold", epochRotateThreshold) + .toString(); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/ChunkFetchFailure.java b/network-common/src/main/java/com/bytedance/css/network/protocol/ChunkFetchFailure.java new file mode 100644 index 0000000..d042cf8 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/ChunkFetchFailure.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +/** + * Response to {@link ChunkFetchRequest} when there is an error fetching the chunk. + */ +public final class ChunkFetchFailure extends AbstractMessage implements ResponseMessage { + public final StreamChunkId streamChunkId; + public final String errorString; + + public ChunkFetchFailure(StreamChunkId streamChunkId, String errorString) { + this.streamChunkId = streamChunkId; + this.errorString = errorString; + } + + @Override + public Type type() { return Type.ChunkFetchFailure; } + + @Override + public int encodedLength() { + return streamChunkId.encodedLength() + Encoders.Strings.encodedLength(errorString); + } + + @Override + public void encode(ByteBuf buf) { + streamChunkId.encode(buf); + Encoders.Strings.encode(buf, errorString); + } + + public static ChunkFetchFailure decode(ByteBuf buf) { + StreamChunkId streamChunkId = StreamChunkId.decode(buf); + String errorString = Encoders.Strings.decode(buf); + return new ChunkFetchFailure(streamChunkId, errorString); + } + + @Override + public int hashCode() { + return Objects.hashCode(streamChunkId, errorString); + } + + @Override + public boolean equals(Object other) { + if (other instanceof ChunkFetchFailure) { + ChunkFetchFailure o = (ChunkFetchFailure) other; + return streamChunkId.equals(o.streamChunkId) && errorString.equals(o.errorString); + } + return false; + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamChunkId", streamChunkId) + .append("errorString", errorString) + .toString(); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/ChunkFetchRequest.java b/network-common/src/main/java/com/bytedance/css/network/protocol/ChunkFetchRequest.java new file mode 100644 index 0000000..0bea482 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/ChunkFetchRequest.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +/** + * Request to fetch a sequence of a single chunk of a stream. This will correspond to a single + * {@link com.bytedance.css.network.protocol.ResponseMessage} (either success or failure). + */ +public final class ChunkFetchRequest extends AbstractMessage implements RequestMessage { + public final StreamChunkId streamChunkId; + + public ChunkFetchRequest(StreamChunkId streamChunkId) { + this.streamChunkId = streamChunkId; + } + + @Override + public Type type() { return Type.ChunkFetchRequest; } + + @Override + public int encodedLength() { + return streamChunkId.encodedLength(); + } + + @Override + public void encode(ByteBuf buf) { + streamChunkId.encode(buf); + } + + public static ChunkFetchRequest decode(ByteBuf buf) { + return new ChunkFetchRequest(StreamChunkId.decode(buf)); + } + + @Override + public int hashCode() { + return streamChunkId.hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other instanceof ChunkFetchRequest) { + ChunkFetchRequest o = (ChunkFetchRequest) other; + return streamChunkId.equals(o.streamChunkId); + } + return false; + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamChunkId", streamChunkId) + .toString(); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/ChunkFetchSuccess.java b/network-common/src/main/java/com/bytedance/css/network/protocol/ChunkFetchSuccess.java new file mode 100644 index 0000000..98266d0 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/ChunkFetchSuccess.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.buffer.NettyManagedBuffer; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +/** + * Response to {@link ChunkFetchRequest} when a chunk exists and has been successfully fetched. + * + * Note that the server-side encoding of this messages does NOT include the buffer itself, as this + * may be written by Netty in a more efficient manner (i.e., zero-copy write). + * Similarly, the client-side decoding will reuse the Netty ByteBuf as the buffer. + */ +public final class ChunkFetchSuccess extends AbstractResponseMessage { + public final StreamChunkId streamChunkId; + + public ChunkFetchSuccess(StreamChunkId streamChunkId, ManagedBuffer buffer) { + super(buffer, true); + this.streamChunkId = streamChunkId; + } + + @Override + public Type type() { return Type.ChunkFetchSuccess; } + + @Override + public int encodedLength() { + return streamChunkId.encodedLength(); + } + + /** Encoding does NOT include 'buffer' itself. See {@link MessageEncoder}. */ + @Override + public void encode(ByteBuf buf) { + streamChunkId.encode(buf); + } + + @Override + public ResponseMessage createFailureResponse(String error) { + return new ChunkFetchFailure(streamChunkId, error); + } + + /** Decoding uses the given ByteBuf as our data, and will retain() it. */ + public static ChunkFetchSuccess decode(ByteBuf buf) { + StreamChunkId streamChunkId = StreamChunkId.decode(buf); + buf.retain(); + NettyManagedBuffer managedBuf = new NettyManagedBuffer(buf.duplicate()); + return new ChunkFetchSuccess(streamChunkId, managedBuf); + } + + @Override + public int hashCode() { + return Objects.hashCode(streamChunkId, body()); + } + + @Override + public boolean equals(Object other) { + if (other instanceof ChunkFetchSuccess) { + ChunkFetchSuccess o = (ChunkFetchSuccess) other; + return streamChunkId.equals(o.streamChunkId) && super.equals(o); + } + return false; + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamChunkId", streamChunkId) + .append("buffer", body()) + .toString(); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/Encodable.java b/network-common/src/main/java/com/bytedance/css/network/protocol/Encodable.java new file mode 100644 index 0000000..8c6ddd9 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/Encodable.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import io.netty.buffer.ByteBuf; + +/** + * Interface for an object which can be encoded into a ByteBuf. Multiple Encodable objects are + * stored in a single, pre-allocated ByteBuf, so Encodables must also provide their length. + * + * Encodable objects should provide a static "decode(ByteBuf)" method which is invoked by + * {@link MessageDecoder}. During decoding, if the object uses the ByteBuf as its data (rather than + * just copying data from it), then you must retain() the ByteBuf. + * + * Additionally, when adding a new Encodable Message, add it to {@link Message.Type}. + */ +public interface Encodable { + /** Number of bytes of the encoded form of this object. */ + int encodedLength(); + + /** + * Serializes this object by writing into the given ByteBuf. + * This method must write exactly encodedLength() bytes. + */ + void encode(ByteBuf buf); +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/Encoders.java b/network-common/src/main/java/com/bytedance/css/network/protocol/Encoders.java new file mode 100644 index 0000000..a055aad --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/Encoders.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import java.nio.charset.StandardCharsets; + +import io.netty.buffer.ByteBuf; + +/** Provides a canonical set of Encoders for simple types. */ +public class Encoders { + + /** Strings are encoded with their length followed by UTF-8 bytes. */ + public static class Strings { + public static int encodedLength(String s) { + return 4 + s.getBytes(StandardCharsets.UTF_8).length; + } + + public static void encode(ByteBuf buf, String s) { + byte[] bytes = s.getBytes(StandardCharsets.UTF_8); + buf.writeInt(bytes.length); + buf.writeBytes(bytes); + } + + public static String decode(ByteBuf buf) { + int length = buf.readInt(); + byte[] bytes = new byte[length]; + buf.readBytes(bytes); + return new String(bytes, StandardCharsets.UTF_8); + } + } + + /** Byte arrays are encoded with their length followed by bytes. */ + public static class ByteArrays { + public static int encodedLength(byte[] arr) { + return 4 + arr.length; + } + + public static void encode(ByteBuf buf, byte[] arr) { + buf.writeInt(arr.length); + buf.writeBytes(arr); + } + + public static byte[] decode(ByteBuf buf) { + int length = buf.readInt(); + byte[] bytes = new byte[length]; + buf.readBytes(bytes); + return bytes; + } + } + + /** String arrays are encoded with the number of strings followed by per-String encoding. */ + public static class StringArrays { + public static int encodedLength(String[] strings) { + int totalLength = 4; + for (String s : strings) { + totalLength += Strings.encodedLength(s); + } + return totalLength; + } + + public static void encode(ByteBuf buf, String[] strings) { + buf.writeInt(strings.length); + for (String s : strings) { + Strings.encode(buf, s); + } + } + + public static String[] decode(ByteBuf buf) { + int numStrings = buf.readInt(); + String[] strings = new String[numStrings]; + for (int i = 0; i < strings.length; i ++) { + strings[i] = Strings.decode(buf); + } + return strings; + } + } + + /** Integer arrays are encoded with their length followed by ints. */ + public static class IntArrays { + public static int encodedLength(int[] arr) { + return 4 + 4 * arr.length; + } + + public static void encode(ByteBuf buf, int[] arr) { + buf.writeInt(arr.length); + for (int i=0; i { + + private static final Logger logger = LoggerFactory.getLogger(MessageDecoder.class); + + public static final MessageDecoder INSTANCE = new MessageDecoder(); + + private MessageDecoder() {} + + @Override + public void decode(ChannelHandlerContext ctx, ByteBuf in, List out) { + Message.Type msgType = Message.Type.decode(in); + Message decoded = decode(msgType, in); + assert decoded.type() == msgType; + logger.trace("Received message {}: {}", msgType, decoded); + out.add(decoded); + } + + private Message decode(Message.Type msgType, ByteBuf in) { + switch (msgType) { + case ChunkFetchRequest: + return ChunkFetchRequest.decode(in); + + case ChunkFetchSuccess: + return ChunkFetchSuccess.decode(in); + + case ChunkFetchFailure: + return ChunkFetchFailure.decode(in); + + case RpcRequest: + return RpcRequest.decode(in); + + case RpcResponse: + return RpcResponse.decode(in); + + case RpcFailure: + return RpcFailure.decode(in); + + case OneWayMessage: + return OneWayMessage.decode(in); + + case StreamRequest: + return StreamRequest.decode(in); + + case StreamResponse: + return StreamResponse.decode(in); + + case StreamFailure: + return StreamFailure.decode(in); + + case UploadStream: + return UploadStream.decode(in); + + case BatchPushDataRequest: + return BatchPushDataRequest.decode(in); + + default: + throw new IllegalArgumentException("Unexpected message type: " + msgType); + } + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/MessageEncoder.java b/network-common/src/main/java/com/bytedance/css/network/protocol/MessageEncoder.java new file mode 100644 index 0000000..846fa80 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/MessageEncoder.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import java.util.List; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.MessageToMessageEncoder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Encoder used by the server side to encode server-to-client responses. + * This encoder is stateless so it is safe to be shared by multiple threads. + */ +@ChannelHandler.Sharable +public final class MessageEncoder extends MessageToMessageEncoder { + + private static final Logger logger = LoggerFactory.getLogger(MessageEncoder.class); + + public static final MessageEncoder INSTANCE = new MessageEncoder(); + + private MessageEncoder() {} + + /*** + * Encodes a Message by invoking its encode() method. For non-data messages, we will add one + * ByteBuf to 'out' containing the total frame length, the message type, and the message itself. + * In the case of a ChunkFetchSuccess, we will also add the ManagedBuffer corresponding to the + * data to 'out', in order to enable zero-copy transfer. + */ + @Override + public void encode(ChannelHandlerContext ctx, Message in, List out) throws Exception { + Object body = null; + long bodyLength = 0; + boolean isBodyInFrame = false; + + // If the message has a body, take it out to enable zero-copy transfer for the payload. + if (in.body() != null) { + try { + bodyLength = in.body().size(); + body = in.body().convertToNetty(); + isBodyInFrame = in.isBodyInFrame(); + } catch (Exception e) { + in.body().release(); + if (in instanceof AbstractResponseMessage) { + AbstractResponseMessage resp = (AbstractResponseMessage) in; + // Re-encode this message as a failure response. + String error = e.getMessage() != null ? e.getMessage() : "null"; + logger.error(String.format("Error processing %s for client %s", + in, ctx.channel().remoteAddress()), e); + encode(ctx, resp.createFailureResponse(error), out); + } else { + throw e; + } + return; + } + } + + Message.Type msgType = in.type(); + // All messages have the frame length, message type, and message itself. The frame length + // may optionally include the length of the body data, depending on what message is being + // sent. + int headerLength = 8 + msgType.encodedLength() + in.encodedLength(); + long frameLength = headerLength + (isBodyInFrame ? bodyLength : 0); + ByteBuf header = ctx.alloc().buffer(headerLength); + header.writeLong(frameLength); + msgType.encode(header); + in.encode(header); + assert header.writableBytes() == 0; + + if (body != null) { + // We transfer ownership of the reference on in.body() to MessageWithHeader. + // This reference will be freed when MessageWithHeader.deallocate() is called. + out.add(new MessageWithHeader(in.body(), header, body, bodyLength)); + } else { + out.add(header); + } + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/MessageWithHeader.java b/network-common/src/main/java/com/bytedance/css/network/protocol/MessageWithHeader.java new file mode 100644 index 0000000..56bf5e1 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/MessageWithHeader.java @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; +import javax.annotation.Nullable; + +import com.google.common.base.Preconditions; +import io.netty.buffer.ByteBuf; +import io.netty.channel.FileRegion; +import io.netty.util.ReferenceCountUtil; + +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.util.AbstractFileRegion; + +/** + * A wrapper message that holds two separate pieces (a header and a body). + * + * The header must be a ByteBuf, while the body can be a ByteBuf or a FileRegion. + */ +class MessageWithHeader extends AbstractFileRegion { + + @Nullable private final ManagedBuffer managedBuffer; + private final ByteBuf header; + private final int headerLength; + private final Object body; + private final long bodyLength; + private long totalBytesTransferred; + + /** + * When the write buffer size is larger than this limit, I/O will be done in chunks of this size. + * The size should not be too large as it will waste underlying memory copy. e.g. If network + * available buffer is smaller than this limit, the data cannot be sent within one single write + * operation while it still will make memory copy with this size. + */ + private static final int NIO_BUFFER_LIMIT = 256 * 1024; + + /** + * Construct a new MessageWithHeader. + * + * @param managedBuffer the {@link ManagedBuffer} that the message body came from. This needs to + * be passed in so that the buffer can be freed when this message is + * deallocated. Ownership of the caller's reference to this buffer is + * transferred to this class, so if the caller wants to continue to use the + * ManagedBuffer in other messages then they will need to call retain() on + * it before passing it to this constructor. This may be null if and only if + * `body` is a {@link FileRegion}. + * @param header the message header. + * @param body the message body. Must be either a {@link ByteBuf} or a {@link FileRegion}. + * @param bodyLength the length of the message body, in bytes. + */ + MessageWithHeader( + @Nullable ManagedBuffer managedBuffer, + ByteBuf header, + Object body, + long bodyLength) { + Preconditions.checkArgument(body instanceof ByteBuf || body instanceof FileRegion, + "Body must be a ByteBuf or a FileRegion."); + this.managedBuffer = managedBuffer; + this.header = header; + this.headerLength = header.readableBytes(); + this.body = body; + this.bodyLength = bodyLength; + } + + @Override + public long count() { + return headerLength + bodyLength; + } + + @Override + public long position() { + return 0; + } + + @Override + public long transferred() { + return totalBytesTransferred; + } + + /** + * This code is more complicated than you would think because we might require multiple + * transferTo invocations in order to transfer a single MessageWithHeader to avoid busy waiting. + * + * The contract is that the caller will ensure position is properly set to the total number + * of bytes transferred so far (i.e. value returned by transferred()). + */ + @Override + public long transferTo(final WritableByteChannel target, final long position) throws IOException { + Preconditions.checkArgument(position == totalBytesTransferred, "Invalid position."); + // Bytes written for header in this call. + long writtenHeader = 0; + if (header.readableBytes() > 0) { + writtenHeader = copyByteBuf(header, target); + totalBytesTransferred += writtenHeader; + if (header.readableBytes() > 0) { + return writtenHeader; + } + } + + // Bytes written for body in this call. + long writtenBody = 0; + if (body instanceof FileRegion) { + writtenBody = ((FileRegion) body).transferTo(target, totalBytesTransferred - headerLength); + } else if (body instanceof ByteBuf) { + writtenBody = copyByteBuf((ByteBuf) body, target); + } + totalBytesTransferred += writtenBody; + + return writtenHeader + writtenBody; + } + + @Override + protected void deallocate() { + header.release(); + ReferenceCountUtil.release(body); + if (managedBuffer != null) { + managedBuffer.release(); + } + } + + private int copyByteBuf(ByteBuf buf, WritableByteChannel target) throws IOException { + // SPARK-24578: cap the sub-region's size of returned nio buffer to improve the performance + // for the case that the passed-in buffer has too many components. + int length = Math.min(buf.readableBytes(), NIO_BUFFER_LIMIT); + // If the ByteBuf holds more then one ByteBuffer we should better call nioBuffers(...) + // to eliminate extra memory copies. + int written = 0; + if (buf.nioBufferCount() == 1) { + ByteBuffer buffer = buf.nioBuffer(buf.readerIndex(), length); + written = target.write(buffer); + } else { + ByteBuffer[] buffers = buf.nioBuffers(buf.readerIndex(), length); + for (ByteBuffer buffer: buffers) { + int remaining = buffer.remaining(); + int w = target.write(buffer); + written += w; + if (w < remaining) { + // Could not write all, we need to break now. + break; + } + } + } + buf.skipBytes(written); + return written; + } + + @Override + public MessageWithHeader touch(Object o) { + super.touch(o); + header.touch(o); + ReferenceCountUtil.touch(body, o); + return this; + } + + @Override + public MessageWithHeader retain(int increment) { + super.retain(increment); + header.retain(increment); + ReferenceCountUtil.retain(body, increment); + if (managedBuffer != null) { + for (int i = 0; i < increment; i++) { + managedBuffer.retain(); + } + } + return this; + } + + @Override + public boolean release(int decrement) { + header.release(decrement); + ReferenceCountUtil.release(body, decrement); + if (managedBuffer != null) { + for (int i = 0; i < decrement; i++) { + managedBuffer.release(); + } + } + return super.release(decrement); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/OneWayMessage.java b/network-common/src/main/java/com/bytedance/css/network/protocol/OneWayMessage.java new file mode 100644 index 0000000..d30109f --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/OneWayMessage.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.buffer.NettyManagedBuffer; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +/** + * A RPC that does not expect a reply, which is handled by a remote + * {@link com.bytedance.css.network.server.RpcHandler}. + */ +public final class OneWayMessage extends AbstractMessage implements RequestMessage { + + public OneWayMessage(ManagedBuffer body) { + super(body, true); + } + + @Override + public Type type() { return Type.OneWayMessage; } + + @Override + public int encodedLength() { + // The integer (a.k.a. the body size) is not really used, since that information is already + // encoded in the frame length. But this maintains backwards compatibility with versions of + // RpcRequest that use Encoders.ByteArrays. + return 4; + } + + @Override + public void encode(ByteBuf buf) { + // See comment in encodedLength(). + buf.writeInt((int) body().size()); + } + + public static OneWayMessage decode(ByteBuf buf) { + // See comment in encodedLength(). + buf.readInt(); + return new OneWayMessage(new NettyManagedBuffer(buf.retain())); + } + + @Override + public int hashCode() { + return Objects.hashCode(body()); + } + + @Override + public boolean equals(Object other) { + if (other instanceof OneWayMessage) { + OneWayMessage o = (OneWayMessage) other; + return super.equals(o); + } + return false; + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("body", body()) + .toString(); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/RequestMessage.java b/network-common/src/main/java/com/bytedance/css/network/protocol/RequestMessage.java new file mode 100644 index 0000000..e403fca --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/RequestMessage.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +/** Messages from the client to the server. */ +public interface RequestMessage extends Message { + // token interface +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/ResponseMessage.java b/network-common/src/main/java/com/bytedance/css/network/protocol/ResponseMessage.java new file mode 100644 index 0000000..7c57510 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/ResponseMessage.java @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +/** Messages from the server to the client. */ +public interface ResponseMessage extends Message { + // token interface +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/RpcFailure.java b/network-common/src/main/java/com/bytedance/css/network/protocol/RpcFailure.java new file mode 100644 index 0000000..14b54c9 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/RpcFailure.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +/** Response to {@link RpcRequest} for a failed RPC. */ +public final class RpcFailure extends AbstractMessage implements ResponseMessage { + public final long requestId; + public final String errorString; + + public RpcFailure(long requestId, String errorString) { + this.requestId = requestId; + this.errorString = errorString; + } + + @Override + public Type type() { return Type.RpcFailure; } + + @Override + public int encodedLength() { + return 8 + Encoders.Strings.encodedLength(errorString); + } + + @Override + public void encode(ByteBuf buf) { + buf.writeLong(requestId); + Encoders.Strings.encode(buf, errorString); + } + + public static RpcFailure decode(ByteBuf buf) { + long requestId = buf.readLong(); + String errorString = Encoders.Strings.decode(buf); + return new RpcFailure(requestId, errorString); + } + + @Override + public int hashCode() { + return Objects.hashCode(requestId, errorString); + } + + @Override + public boolean equals(Object other) { + if (other instanceof RpcFailure) { + RpcFailure o = (RpcFailure) other; + return requestId == o.requestId && errorString.equals(o.errorString); + } + return false; + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("requestId", requestId) + .append("errorString", errorString) + .toString(); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/RpcRequest.java b/network-common/src/main/java/com/bytedance/css/network/protocol/RpcRequest.java new file mode 100644 index 0000000..1df8f1d --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/RpcRequest.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.buffer.NettyManagedBuffer; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +/** + * A generic RPC which is handled by a remote {@link com.bytedance.css.network.server.RpcHandler}. + * This will correspond to a single + * {@link com.bytedance.css.network.protocol.ResponseMessage} (either success or failure). + */ +public final class RpcRequest extends AbstractMessage implements RequestMessage { + /** Used to link an RPC request with its response. */ + public final long requestId; + + public RpcRequest(long requestId, ManagedBuffer message) { + super(message, true); + this.requestId = requestId; + } + + @Override + public Type type() { return Type.RpcRequest; } + + @Override + public int encodedLength() { + // The integer (a.k.a. the body size) is not really used, since that information is already + // encoded in the frame length. But this maintains backwards compatibility with versions of + // RpcRequest that use Encoders.ByteArrays. + return 8 + 4; + } + + @Override + public void encode(ByteBuf buf) { + buf.writeLong(requestId); + // See comment in encodedLength(). + buf.writeInt((int) body().size()); + } + + public static RpcRequest decode(ByteBuf buf) { + long requestId = buf.readLong(); + // See comment in encodedLength(). + buf.readInt(); + return new RpcRequest(requestId, new NettyManagedBuffer(buf.retain())); + } + + @Override + public int hashCode() { + return Objects.hashCode(requestId, body()); + } + + @Override + public boolean equals(Object other) { + if (other instanceof RpcRequest) { + RpcRequest o = (RpcRequest) other; + return requestId == o.requestId && super.equals(o); + } + return false; + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("requestId", requestId) + .append("body", body()) + .toString(); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/RpcResponse.java b/network-common/src/main/java/com/bytedance/css/network/protocol/RpcResponse.java new file mode 100644 index 0000000..dd51c4b --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/RpcResponse.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.buffer.NettyManagedBuffer; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +/** Response to {@link RpcRequest} for a successful RPC. */ +public final class RpcResponse extends AbstractResponseMessage { + public final long requestId; + + public RpcResponse(long requestId, ManagedBuffer message) { + super(message, true); + this.requestId = requestId; + } + + @Override + public Type type() { return Type.RpcResponse; } + + @Override + public int encodedLength() { + // The integer (a.k.a. the body size) is not really used, since that information is already + // encoded in the frame length. But this maintains backwards compatibility with versions of + // RpcRequest that use Encoders.ByteArrays. + return 8 + 4; + } + + @Override + public void encode(ByteBuf buf) { + buf.writeLong(requestId); + // See comment in encodedLength(). + buf.writeInt((int) body().size()); + } + + @Override + public ResponseMessage createFailureResponse(String error) { + return new RpcFailure(requestId, error); + } + + public static RpcResponse decode(ByteBuf buf) { + long requestId = buf.readLong(); + // See comment in encodedLength(). + buf.readInt(); + return new RpcResponse(requestId, new NettyManagedBuffer(buf.retain())); + } + + @Override + public int hashCode() { + return Objects.hashCode(requestId, body()); + } + + @Override + public boolean equals(Object other) { + if (other instanceof RpcResponse) { + RpcResponse o = (RpcResponse) other; + return requestId == o.requestId && super.equals(o); + } + return false; + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("requestId", requestId) + .append("body", body()) + .toString(); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/StreamChunkId.java b/network-common/src/main/java/com/bytedance/css/network/protocol/StreamChunkId.java new file mode 100644 index 0000000..2ca74ed --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/StreamChunkId.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +/** +* Encapsulates a request for a particular chunk of a stream. +*/ +public final class StreamChunkId implements Encodable { + public final long streamId; + public final int chunkIndex; + + public StreamChunkId(long streamId, int chunkIndex) { + this.streamId = streamId; + this.chunkIndex = chunkIndex; + } + + @Override + public int encodedLength() { + return 8 + 4; + } + + public void encode(ByteBuf buffer) { + buffer.writeLong(streamId); + buffer.writeInt(chunkIndex); + } + + public static StreamChunkId decode(ByteBuf buffer) { + assert buffer.readableBytes() >= 8 + 4; + long streamId = buffer.readLong(); + int chunkIndex = buffer.readInt(); + return new StreamChunkId(streamId, chunkIndex); + } + + @Override + public int hashCode() { + return Objects.hashCode(streamId, chunkIndex); + } + + @Override + public boolean equals(Object other) { + if (other instanceof StreamChunkId) { + StreamChunkId o = (StreamChunkId) other; + return streamId == o.streamId && chunkIndex == o.chunkIndex; + } + return false; + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamId", streamId) + .append("chunkIndex", chunkIndex) + .toString(); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/StreamFailure.java b/network-common/src/main/java/com/bytedance/css/network/protocol/StreamFailure.java new file mode 100644 index 0000000..1b5ba62 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/StreamFailure.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +/** + * Message indicating an error when transferring a stream. + */ +public final class StreamFailure extends AbstractMessage implements ResponseMessage { + public final String streamId; + public final String error; + + public StreamFailure(String streamId, String error) { + this.streamId = streamId; + this.error = error; + } + + @Override + public Type type() { return Type.StreamFailure; } + + @Override + public int encodedLength() { + return Encoders.Strings.encodedLength(streamId) + Encoders.Strings.encodedLength(error); + } + + @Override + public void encode(ByteBuf buf) { + Encoders.Strings.encode(buf, streamId); + Encoders.Strings.encode(buf, error); + } + + public static StreamFailure decode(ByteBuf buf) { + String streamId = Encoders.Strings.decode(buf); + String error = Encoders.Strings.decode(buf); + return new StreamFailure(streamId, error); + } + + @Override + public int hashCode() { + return Objects.hashCode(streamId, error); + } + + @Override + public boolean equals(Object other) { + if (other instanceof StreamFailure) { + StreamFailure o = (StreamFailure) other; + return streamId.equals(o.streamId) && error.equals(o.error); + } + return false; + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamId", streamId) + .append("error", error) + .toString(); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/StreamRequest.java b/network-common/src/main/java/com/bytedance/css/network/protocol/StreamRequest.java new file mode 100644 index 0000000..24c99a0 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/StreamRequest.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +/** + * Request to stream data from the remote end. + *

    + * The stream ID is an arbitrary string that needs to be negotiated between the two endpoints before + * the data can be streamed. + */ +public final class StreamRequest extends AbstractMessage implements RequestMessage { + public final String streamId; + + public StreamRequest(String streamId) { + this.streamId = streamId; + } + + @Override + public Type type() { return Type.StreamRequest; } + + @Override + public int encodedLength() { + return Encoders.Strings.encodedLength(streamId); + } + + @Override + public void encode(ByteBuf buf) { + Encoders.Strings.encode(buf, streamId); + } + + public static StreamRequest decode(ByteBuf buf) { + String streamId = Encoders.Strings.decode(buf); + return new StreamRequest(streamId); + } + + @Override + public int hashCode() { + return Objects.hashCode(streamId); + } + + @Override + public boolean equals(Object other) { + if (other instanceof StreamRequest) { + StreamRequest o = (StreamRequest) other; + return streamId.equals(o.streamId); + } + return false; + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamId", streamId) + .toString(); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/StreamResponse.java b/network-common/src/main/java/com/bytedance/css/network/protocol/StreamResponse.java new file mode 100644 index 0000000..f088a0c --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/StreamResponse.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import com.bytedance.css.network.buffer.ManagedBuffer; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +/** + * Response to {@link StreamRequest} when the stream has been successfully opened. + *

    + * Note the message itself does not contain the stream data. That is written separately by the + * sender. The receiver is expected to set a temporary channel handler that will consume the + * number of bytes this message says the stream has. + */ +public final class StreamResponse extends AbstractResponseMessage { + public final String streamId; + public final long byteCount; + + public StreamResponse(String streamId, long byteCount, ManagedBuffer buffer) { + super(buffer, false); + this.streamId = streamId; + this.byteCount = byteCount; + } + + @Override + public Type type() { return Type.StreamResponse; } + + @Override + public int encodedLength() { + return 8 + Encoders.Strings.encodedLength(streamId); + } + + /** Encoding does NOT include 'buffer' itself. See {@link MessageEncoder}. */ + @Override + public void encode(ByteBuf buf) { + Encoders.Strings.encode(buf, streamId); + buf.writeLong(byteCount); + } + + @Override + public ResponseMessage createFailureResponse(String error) { + return new StreamFailure(streamId, error); + } + + public static StreamResponse decode(ByteBuf buf) { + String streamId = Encoders.Strings.decode(buf); + long byteCount = buf.readLong(); + return new StreamResponse(streamId, byteCount, null); + } + + @Override + public int hashCode() { + return Objects.hashCode(byteCount, streamId); + } + + @Override + public boolean equals(Object other) { + if (other instanceof StreamResponse) { + StreamResponse o = (StreamResponse) other; + return byteCount == o.byteCount && streamId.equals(o.streamId); + } + return false; + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamId", streamId) + .append("byteCount", byteCount) + .append("body", body()) + .toString(); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/UploadStream.java b/network-common/src/main/java/com/bytedance/css/network/protocol/UploadStream.java new file mode 100644 index 0000000..67f77f9 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/UploadStream.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import io.netty.buffer.ByteBuf; + +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.buffer.NettyManagedBuffer; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +/** + * An RPC with data that is sent outside of the frame, so it can be read as a stream. + */ +public final class UploadStream extends AbstractMessage implements RequestMessage { + /** Used to link an RPC request with its response. */ + public final long requestId; + public final ManagedBuffer meta; + public final long bodyByteCount; + + public UploadStream(long requestId, ManagedBuffer meta, ManagedBuffer body) { + super(body, false); // body is *not* included in the frame + this.requestId = requestId; + this.meta = meta; + bodyByteCount = body.size(); + } + + // this version is called when decoding the bytes on the receiving end. The body is handled + // separately. + private UploadStream(long requestId, ManagedBuffer meta, long bodyByteCount) { + super(null, false); + this.requestId = requestId; + this.meta = meta; + this.bodyByteCount = bodyByteCount; + } + + @Override + public Type type() { return Type.UploadStream; } + + @Override + public int encodedLength() { + // the requestId, meta size, meta and bodyByteCount (body is not included) + return 8 + 4 + ((int) meta.size()) + 8; + } + + @Override + public void encode(ByteBuf buf) { + buf.writeLong(requestId); + try { + ByteBuffer metaBuf = meta.nioByteBuffer(); + buf.writeInt(metaBuf.remaining()); + buf.writeBytes(metaBuf); + } catch (IOException io) { + throw new RuntimeException(io); + } + buf.writeLong(bodyByteCount); + } + + public static UploadStream decode(ByteBuf buf) { + long requestId = buf.readLong(); + int metaSize = buf.readInt(); + ManagedBuffer meta = new NettyManagedBuffer(buf.readRetainedSlice(metaSize)); + long bodyByteCount = buf.readLong(); + // This is called by the frame decoder, so the data is still null. We need a StreamInterceptor + // to read the data. + return new UploadStream(requestId, meta, bodyByteCount); + } + + @Override + public int hashCode() { + return Long.hashCode(requestId); + } + + @Override + public boolean equals(Object other) { + if (other instanceof UploadStream) { + UploadStream o = (UploadStream) other; + return requestId == o.requestId && super.equals(o); + } + return false; + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("requestId", requestId) + .append("body", body()) + .toString(); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/shuffle/BlockTransferMessage.java b/network-common/src/main/java/com/bytedance/css/network/protocol/shuffle/BlockTransferMessage.java new file mode 100644 index 0000000..02935fc --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/shuffle/BlockTransferMessage.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol.shuffle; + +import com.bytedance.css.network.protocol.Encodable; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; + +import java.nio.ByteBuffer; + + +public abstract class BlockTransferMessage implements Encodable { + protected abstract Type type(); + + /** Preceding every serialized message is its type, which allows us to deserialize it. */ + public enum Type { + OPEN_STREAM(0), STREAM_HANDLE(1); + + private final byte id; + + Type(int id) { + assert id < 128 : "Cannot have more than 128 message types"; + this.id = (byte) id; + } + + public byte id() { return id; } + } + + // NB: Java does not support static methods in interfaces, so we must put this in a static class. + public static class Decoder { + /** Deserializes the 'type' byte followed by the message itself. */ + public static BlockTransferMessage fromByteBuffer(ByteBuffer msg) { + ByteBuf buf = Unpooled.wrappedBuffer(msg); + byte type = buf.readByte(); + switch (type) { + case 0: return OpenStream.decode(buf); + case 1: return StreamHandle.decode(buf); + default: throw new IllegalArgumentException("Unknown message type: " + type); + } + } + } + + /** Serializes the 'type' byte followed by the message itself. */ + public ByteBuffer toByteBuffer() { + // Allow room for encoded message, plus the type byte + ByteBuf buf = Unpooled.buffer(encodedLength() + 1); + buf.writeByte(type().id); + encode(buf); + assert buf.writableBytes() == 0 : "Writable bytes remain: " + buf.writableBytes(); + return buf.nioBuffer(); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/shuffle/OpenStream.java b/network-common/src/main/java/com/bytedance/css/network/protocol/shuffle/OpenStream.java new file mode 100644 index 0000000..be8c356 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/shuffle/OpenStream.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol.shuffle; + +import com.bytedance.css.network.protocol.Encoders; +import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +import java.util.Objects; + +/** Request to read a set of chunks. Returns {@link StreamHandle}. */ +public class OpenStream extends BlockTransferMessage { + public final String shuffleKey; + public final String filePath; + public final int initChunkIndex; + + public OpenStream(String shuffleKey, String filePath, int initChunkIndex) { + this.shuffleKey = shuffleKey; + this.filePath = filePath; + this.initChunkIndex = initChunkIndex; + } + + @Override + protected Type type() { return Type.OPEN_STREAM; } + + @Override + public int hashCode() { + return Objects.hash(shuffleKey, filePath, initChunkIndex); + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("shuffleKey", shuffleKey) + .append("filePath", filePath) + .append("initChunkIndex", initChunkIndex) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other != null && other instanceof OpenStream) { + OpenStream o = (OpenStream) other; + return Objects.equals(shuffleKey, o.shuffleKey) + && Objects.equals(filePath, o.filePath) + && Objects.equals(initChunkIndex, o.initChunkIndex); + } + return false; + } + + @Override + public int encodedLength() { + return Encoders.Strings.encodedLength(shuffleKey) + + Encoders.Strings.encodedLength(filePath) + + 4; + } + + @Override + public void encode(ByteBuf buf) { + Encoders.Strings.encode(buf, shuffleKey); + Encoders.Strings.encode(buf, filePath); + buf.writeInt(initChunkIndex); + } + + public static OpenStream decode(ByteBuf buf) { + String shuffleKey = Encoders.Strings.decode(buf); + String filePath = Encoders.Strings.decode(buf); + int initChunkIndex = buf.readInt(); + return new OpenStream(shuffleKey, filePath, initChunkIndex); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/protocol/shuffle/StreamHandle.java b/network-common/src/main/java/com/bytedance/css/network/protocol/shuffle/StreamHandle.java new file mode 100644 index 0000000..9c22845 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/protocol/shuffle/StreamHandle.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol.shuffle; + +import io.netty.buffer.ByteBuf; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +import java.util.Objects; + +/** + * Identifier for a fixed number of chunks to read from a stream created by an "open stream" message. + */ +public class StreamHandle extends BlockTransferMessage { + public final long streamId; + public final int numChunks; + + public StreamHandle(long streamId, int numChunks) { + this.streamId = streamId; + this.numChunks = numChunks; + } + + @Override + protected Type type() { return Type.STREAM_HANDLE; } + + @Override + public int hashCode() { + return Objects.hash(streamId, numChunks); + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("streamId", streamId) + .append("numChunks", numChunks) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other != null && other instanceof StreamHandle) { + StreamHandle o = (StreamHandle) other; + return Objects.equals(streamId, o.streamId) + && Objects.equals(numChunks, o.numChunks); + } + return false; + } + + @Override + public int encodedLength() { + return 8 + 4; + } + + @Override + public void encode(ByteBuf buf) { + buf.writeLong(streamId); + buf.writeInt(numChunks); + } + + public static StreamHandle decode(ByteBuf buf) { + long streamId = buf.readLong(); + int numChunks = buf.readInt(); + return new StreamHandle(streamId, numChunks); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/sasl/SaslClientBootstrap.java b/network-common/src/main/java/com/bytedance/css/network/sasl/SaslClientBootstrap.java new file mode 100644 index 0000000..0b479b9 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/sasl/SaslClientBootstrap.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.sasl; + +import java.io.IOException; +import java.nio.ByteBuffer; +import javax.security.sasl.Sasl; +import javax.security.sasl.SaslException; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.client.TransportClientBootstrap; +import com.bytedance.css.network.util.JavaUtils; +import com.bytedance.css.network.util.TransportConf; + +/** + * Bootstraps a {@link TransportClient} by performing SASL authentication on the connection. The + * server should be setup with a {@link SaslRpcHandler} with matching keys for the given appId. + */ +public class SaslClientBootstrap implements TransportClientBootstrap { + private static final Logger logger = LoggerFactory.getLogger(SaslClientBootstrap.class); + + private final TransportConf conf; + private final String appId; + private final SecretKeyHolder secretKeyHolder; + + public SaslClientBootstrap(TransportConf conf, String appId, SecretKeyHolder secretKeyHolder) { + this.conf = conf; + this.appId = appId; + this.secretKeyHolder = secretKeyHolder; + } + + /** + * Performs SASL authentication by sending a token, and then proceeding with the SASL + * challenge-response tokens until we either successfully authenticate or throw an exception + * due to mismatch. + */ + @Override + public void doBootstrap(TransportClient client, Channel channel) { + SparkSaslClient saslClient = new SparkSaslClient(appId, secretKeyHolder, conf.saslEncryption()); + try { + byte[] payload = saslClient.firstToken(); + + while (!saslClient.isComplete()) { + SaslMessage msg = new SaslMessage(appId, payload); + ByteBuf buf = Unpooled.buffer(msg.encodedLength() + (int) msg.body().size()); + msg.encode(buf); + buf.writeBytes(msg.body().nioByteBuffer()); + + ByteBuffer response = client.sendRpcSync(buf.nioBuffer(), conf.authRTTimeoutMs()); + payload = saslClient.response(JavaUtils.bufferToArray(response)); + } + + client.setClientId(appId); + + if (conf.saslEncryption()) { + if (!SparkSaslServer.QOP_AUTH_CONF.equals(saslClient.getNegotiatedProperty(Sasl.QOP))) { + throw new RuntimeException( + new SaslException("Encryption requests by negotiated non-encrypted connection.")); + } + + SaslEncryption.addToChannel(channel, saslClient, conf.maxSaslEncryptedBlockSize()); + saslClient = null; + logger.debug("Channel {} configured for encryption.", client); + } + } catch (IOException ioe) { + throw new RuntimeException(ioe); + } finally { + if (saslClient != null) { + try { + // Once authentication is complete, the server will trust all remaining communication. + saslClient.dispose(); + } catch (RuntimeException e) { + logger.error("Error while disposing SASL client", e); + } + } + } + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/sasl/SaslEncryption.java b/network-common/src/main/java/com/bytedance/css/network/sasl/SaslEncryption.java new file mode 100644 index 0000000..08f0fb2 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/sasl/SaslEncryption.java @@ -0,0 +1,331 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.sasl; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; +import java.util.List; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelOutboundHandlerAdapter; +import io.netty.channel.ChannelPromise; +import io.netty.channel.FileRegion; +import io.netty.handler.codec.MessageToMessageDecoder; + +import com.bytedance.css.network.util.AbstractFileRegion; +import com.bytedance.css.network.util.ByteArrayWritableChannel; +import com.bytedance.css.network.util.NettyUtils; + +/** + * Provides SASL-based encryption for transport channels. The single method exposed by this + * class installs the needed channel handlers on a connected channel. + */ +class SaslEncryption { + + @VisibleForTesting + static final String ENCRYPTION_HANDLER_NAME = "saslEncryption"; + + /** + * Adds channel handlers that perform encryption / decryption of data using SASL. + * + * @param channel The channel. + * @param backend The SASL backend. + * @param maxOutboundBlockSize Max size in bytes of outgoing encrypted blocks, to control + * memory usage. + */ + static void addToChannel( + Channel channel, + SaslEncryptionBackend backend, + int maxOutboundBlockSize) { + channel.pipeline() + .addFirst(ENCRYPTION_HANDLER_NAME, new EncryptionHandler(backend, maxOutboundBlockSize)) + .addFirst("saslDecryption", new DecryptionHandler(backend)) + .addFirst("saslFrameDecoder", NettyUtils.createFrameDecoder()); + } + + private static class EncryptionHandler extends ChannelOutboundHandlerAdapter { + + private final int maxOutboundBlockSize; + private final SaslEncryptionBackend backend; + + EncryptionHandler(SaslEncryptionBackend backend, int maxOutboundBlockSize) { + this.backend = backend; + this.maxOutboundBlockSize = maxOutboundBlockSize; + } + + /** + * Wrap the incoming message in an implementation that will perform encryption lazily. This is + * needed to guarantee ordering of the outgoing encrypted packets - they need to be decrypted in + * the same order, and netty doesn't have an atomic ChannelHandlerContext.write() API, so it + * does not guarantee any ordering. + */ + @Override + public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) + throws Exception { + + ctx.write(new EncryptedMessage(backend, msg, maxOutboundBlockSize), promise); + } + + @Override + public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { + try { + backend.dispose(); + } finally { + super.handlerRemoved(ctx); + } + } + + } + + private static class DecryptionHandler extends MessageToMessageDecoder { + + private final SaslEncryptionBackend backend; + + DecryptionHandler(SaslEncryptionBackend backend) { + this.backend = backend; + } + + @Override + protected void decode(ChannelHandlerContext ctx, ByteBuf msg, List out) + throws Exception { + + byte[] data; + int offset; + int length = msg.readableBytes(); + if (msg.hasArray()) { + data = msg.array(); + offset = msg.arrayOffset(); + msg.skipBytes(length); + } else { + data = new byte[length]; + msg.readBytes(data); + offset = 0; + } + + out.add(Unpooled.wrappedBuffer(backend.unwrap(data, offset, length))); + } + + } + + @VisibleForTesting + static class EncryptedMessage extends AbstractFileRegion { + + private final SaslEncryptionBackend backend; + private final boolean isByteBuf; + private final ByteBuf buf; + private final FileRegion region; + private final int maxOutboundBlockSize; + + /** + * A channel used to buffer input data for encryption. The channel has an upper size bound + * so that if the input is larger than the allowed buffer, it will be broken into multiple + * chunks. Made non-final to enable lazy initialization, which saves memory. + */ + private ByteArrayWritableChannel byteChannel; + + private ByteBuf currentHeader; + private ByteBuffer currentChunk; + private long currentChunkSize; + private long currentReportedBytes; + private long unencryptedChunkSize; + private long transferred; + + EncryptedMessage(SaslEncryptionBackend backend, Object msg, int maxOutboundBlockSize) { + Preconditions.checkArgument(msg instanceof ByteBuf || msg instanceof FileRegion, + "Unrecognized message type: %s", msg.getClass().getName()); + this.backend = backend; + this.isByteBuf = msg instanceof ByteBuf; + this.buf = isByteBuf ? (ByteBuf) msg : null; + this.region = isByteBuf ? null : (FileRegion) msg; + this.maxOutboundBlockSize = maxOutboundBlockSize; + } + + /** + * Returns the size of the original (unencrypted) message. + * + * This makes assumptions about how netty treats FileRegion instances, because there's no way + * to know beforehand what will be the size of the encrypted message. Namely, it assumes + * that netty will try to transfer data from this message while + * transferred() < count(). So these two methods return, technically, wrong data, + * but netty doesn't know better. + */ + @Override + public long count() { + return isByteBuf ? buf.readableBytes() : region.count(); + } + + @Override + public long position() { + return 0; + } + + /** + * Returns an approximation of the amount of data transferred. See {@link #count()}. + */ + @Override + public long transferred() { + return transferred; + } + + @Override + public EncryptedMessage touch(Object o) { + super.touch(o); + if (buf != null) { + buf.touch(o); + } + if (region != null) { + region.touch(o); + } + return this; + } + + @Override + public EncryptedMessage retain(int increment) { + super.retain(increment); + if (buf != null) { + buf.retain(increment); + } + if (region != null) { + region.retain(increment); + } + return this; + } + + @Override + public boolean release(int decrement) { + if (region != null) { + region.release(decrement); + } + if (buf != null) { + buf.release(decrement); + } + return super.release(decrement); + } + + /** + * Transfers data from the original message to the channel, encrypting it in the process. + * + * This method also breaks down the original message into smaller chunks when needed. This + * is done to keep memory usage under control. This avoids having to copy the whole message + * data into memory at once, and can avoid ballooning memory usage when transferring large + * messages such as shuffle blocks. + * + * The {@link #transferred()} counter also behaves a little funny, in that it won't go forward + * until a whole chunk has been written. This is done because the code can't use the actual + * number of bytes written to the channel as the transferred count (see {@link #count()}). + * Instead, once an encrypted chunk is written to the output (including its header), the + * size of the original block will be added to the {@link #transferred()} amount. + */ + @Override + public long transferTo(final WritableByteChannel target, final long position) + throws IOException { + + Preconditions.checkArgument(position == transferred(), "Invalid position."); + + long reportedWritten = 0L; + long actuallyWritten = 0L; + do { + if (currentChunk == null) { + nextChunk(); + } + + if (currentHeader.readableBytes() > 0) { + int bytesWritten = target.write(currentHeader.nioBuffer()); + currentHeader.skipBytes(bytesWritten); + actuallyWritten += bytesWritten; + if (currentHeader.readableBytes() > 0) { + // Break out of loop if there are still header bytes left to write. + break; + } + } + + actuallyWritten += target.write(currentChunk); + if (!currentChunk.hasRemaining()) { + // Only update the count of written bytes once a full chunk has been written. + // See method javadoc. + long chunkBytesRemaining = unencryptedChunkSize - currentReportedBytes; + reportedWritten += chunkBytesRemaining; + transferred += chunkBytesRemaining; + currentHeader.release(); + currentHeader = null; + currentChunk = null; + currentChunkSize = 0; + currentReportedBytes = 0; + } + } while (currentChunk == null && transferred() + reportedWritten < count()); + + // Returning 0 triggers a backoff mechanism in netty which may harm performance. Instead, + // we return 1 until we can (i.e. until the reported count would actually match the size + // of the current chunk), at which point we resort to returning 0 so that the counts still + // match, at the cost of some performance. That situation should be rare, though. + if (reportedWritten != 0L) { + return reportedWritten; + } + + if (actuallyWritten > 0 && currentReportedBytes < currentChunkSize - 1) { + transferred += 1L; + currentReportedBytes += 1L; + return 1L; + } + + return 0L; + } + + private void nextChunk() throws IOException { + if (byteChannel == null) { + byteChannel = new ByteArrayWritableChannel(maxOutboundBlockSize); + } + byteChannel.reset(); + if (isByteBuf) { + int copied = byteChannel.write(buf.nioBuffer()); + buf.skipBytes(copied); + } else { + region.transferTo(byteChannel, region.transferred()); + } + + byte[] encrypted = backend.wrap(byteChannel.getData(), 0, byteChannel.length()); + this.currentChunk = ByteBuffer.wrap(encrypted); + this.currentChunkSize = encrypted.length; + this.currentHeader = Unpooled.copyLong(8 + currentChunkSize); + this.unencryptedChunkSize = byteChannel.length(); + } + + @Override + protected void deallocate() { + if (currentHeader != null) { + currentHeader.release(); + } + if (buf != null) { + buf.release(); + } + if (region != null) { + region.release(); + } + } + + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/sasl/SaslEncryptionBackend.java b/network-common/src/main/java/com/bytedance/css/network/sasl/SaslEncryptionBackend.java new file mode 100644 index 0000000..b385efa --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/sasl/SaslEncryptionBackend.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.sasl; + +import javax.security.sasl.SaslException; + +interface SaslEncryptionBackend { + + /** Disposes of resources used by the backend. */ + void dispose(); + + /** Encrypt data. */ + byte[] wrap(byte[] data, int offset, int len) throws SaslException; + + /** Decrypt data. */ + byte[] unwrap(byte[] data, int offset, int len) throws SaslException; + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/sasl/SaslMessage.java b/network-common/src/main/java/com/bytedance/css/network/sasl/SaslMessage.java new file mode 100644 index 0000000..45bbbf3 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/sasl/SaslMessage.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.sasl; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; + +import com.bytedance.css.network.buffer.NettyManagedBuffer; +import com.bytedance.css.network.protocol.Encoders; +import com.bytedance.css.network.protocol.AbstractMessage; + +/** + * Encodes a Sasl-related message which is attempting to authenticate using some credentials tagged + * with the given appId. This appId allows a single SaslRpcHandler to multiplex different + * applications which may be using different sets of credentials. + */ +class SaslMessage extends AbstractMessage { + + /** Serialization tag used to catch incorrect payloads. */ + private static final byte TAG_BYTE = (byte) 0xEA; + + public final String appId; + + SaslMessage(String appId, byte[] message) { + this(appId, Unpooled.wrappedBuffer(message)); + } + + SaslMessage(String appId, ByteBuf message) { + super(new NettyManagedBuffer(message), true); + this.appId = appId; + } + + @Override + public Type type() { return Type.User; } + + @Override + public int encodedLength() { + // The integer (a.k.a. the body size) is not really used, since that information is already + // encoded in the frame length. But this maintains backwards compatibility with versions of + // RpcRequest that use Encoders.ByteArrays. + return 1 + Encoders.Strings.encodedLength(appId) + 4; + } + + @Override + public void encode(ByteBuf buf) { + buf.writeByte(TAG_BYTE); + Encoders.Strings.encode(buf, appId); + // See comment in encodedLength(). + buf.writeInt((int) body().size()); + } + + public static SaslMessage decode(ByteBuf buf) { + if (buf.readByte() != TAG_BYTE) { + throw new IllegalStateException("Expected SaslMessage, received something else" + + " (maybe your client does not have SASL enabled?)"); + } + + String appId = Encoders.Strings.decode(buf); + // See comment in encodedLength(). + buf.readInt(); + return new SaslMessage(appId, buf.retain()); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/sasl/SaslRpcHandler.java b/network-common/src/main/java/com/bytedance/css/network/sasl/SaslRpcHandler.java new file mode 100644 index 0000000..efa2ee8 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/sasl/SaslRpcHandler.java @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.sasl; + +import java.io.IOException; +import java.nio.ByteBuffer; +import javax.security.sasl.Sasl; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.Channel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.bytedance.css.network.client.RpcResponseCallback; +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.server.AbstractAuthRpcHandler; +import com.bytedance.css.network.server.RpcHandler; +import com.bytedance.css.network.util.JavaUtils; +import com.bytedance.css.network.util.TransportConf; + +/** + * RPC Handler which performs SASL authentication before delegating to a child RPC handler. + * The delegate will only receive messages if the given connection has been successfully + * authenticated. A connection may be authenticated at most once. + * + * Note that the authentication process consists of multiple challenge-response pairs, each of + * which are individual RPCs. + */ +public class SaslRpcHandler extends AbstractAuthRpcHandler { + private static final Logger logger = LoggerFactory.getLogger(SaslRpcHandler.class); + + /** Transport configuration. */ + private final TransportConf conf; + + /** The client channel. */ + private final Channel channel; + + /** Class which provides secret keys which are shared by server and client on a per-app basis. */ + private final SecretKeyHolder secretKeyHolder; + + private SparkSaslServer saslServer; + + public SaslRpcHandler( + TransportConf conf, + Channel channel, + RpcHandler delegate, + SecretKeyHolder secretKeyHolder) { + super(delegate); + this.conf = conf; + this.channel = channel; + this.secretKeyHolder = secretKeyHolder; + this.saslServer = null; + } + + @Override + public boolean doAuthChallenge( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { + if (saslServer == null || !saslServer.isComplete()) { + ByteBuf nettyBuf = Unpooled.wrappedBuffer(message); + SaslMessage saslMessage; + try { + saslMessage = SaslMessage.decode(nettyBuf); + } finally { + nettyBuf.release(); + } + + if (saslServer == null) { + // First message in the handshake, setup the necessary state. + client.setClientId(saslMessage.appId); + saslServer = new SparkSaslServer(saslMessage.appId, secretKeyHolder, + conf.saslServerAlwaysEncrypt()); + } + + byte[] response; + try { + response = saslServer.response(JavaUtils.bufferToArray( + saslMessage.body().nioByteBuffer())); + } catch (IOException ioe) { + throw new RuntimeException(ioe); + } + callback.onSuccess(ByteBuffer.wrap(response)); + } + + // Setup encryption after the SASL response is sent, otherwise the client can't parse the + // response. It's ok to change the channel pipeline here since we are processing an incoming + // message, so the pipeline is busy and no new incoming messages will be fed to it before this + // method returns. This assumes that the code ensures, through other means, that no outbound + // messages are being written to the channel while negotiation is still going on. + if (saslServer.isComplete()) { + if (!SparkSaslServer.QOP_AUTH_CONF.equals(saslServer.getNegotiatedProperty(Sasl.QOP))) { + logger.debug("SASL authentication successful for channel {}", client); + complete(true); + return true; + } + + logger.debug("Enabling encryption for channel {}", client); + SaslEncryption.addToChannel(channel, saslServer, conf.maxSaslEncryptedBlockSize()); + complete(false); + return true; + } + return false; + } + + @Override + public void channelInactive(TransportClient client) { + try { + super.channelInactive(client); + } finally { + if (saslServer != null) { + saslServer.dispose(); + } + } + } + + private void complete(boolean dispose) { + if (dispose) { + try { + saslServer.dispose(); + } catch (RuntimeException e) { + logger.error("Error while disposing SASL server", e); + } + } + + saslServer = null; + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/sasl/SaslServerBootstrap.java b/network-common/src/main/java/com/bytedance/css/network/sasl/SaslServerBootstrap.java new file mode 100644 index 0000000..364ff0d --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/sasl/SaslServerBootstrap.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.sasl; + +import io.netty.channel.Channel; + +import com.bytedance.css.network.server.RpcHandler; +import com.bytedance.css.network.server.TransportServerBootstrap; +import com.bytedance.css.network.util.TransportConf; + +/** + * A bootstrap which is executed on a TransportServer's client channel once a client connects + * to the server. This allows customizing the client channel to allow for things such as SASL + * authentication. + */ +public class SaslServerBootstrap implements TransportServerBootstrap { + + private final TransportConf conf; + private final SecretKeyHolder secretKeyHolder; + + public SaslServerBootstrap(TransportConf conf, SecretKeyHolder secretKeyHolder) { + this.conf = conf; + this.secretKeyHolder = secretKeyHolder; + } + + /** + * Wrap the given application handler in a SaslRpcHandler that will handle the initial SASL + * negotiation. + */ + public RpcHandler doBootstrap(Channel channel, RpcHandler rpcHandler) { + return new SaslRpcHandler(conf, channel, rpcHandler, secretKeyHolder); + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/sasl/SecretKeyHolder.java b/network-common/src/main/java/com/bytedance/css/network/sasl/SecretKeyHolder.java new file mode 100644 index 0000000..462a475 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/sasl/SecretKeyHolder.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.sasl; + +/** + * Interface for getting a secret key associated with some application. + */ +public interface SecretKeyHolder { + /** + * Gets an appropriate SASL User for the given appId. + * @throws IllegalArgumentException if the given appId is not associated with a SASL user. + */ + String getSaslUser(String appId); + + /** + * Gets an appropriate SASL secret key for the given appId. + * @throws IllegalArgumentException if the given appId is not associated with a SASL secret key. + */ + String getSecretKey(String appId); +} diff --git a/network-common/src/main/java/com/bytedance/css/network/sasl/SparkSaslClient.java b/network-common/src/main/java/com/bytedance/css/network/sasl/SparkSaslClient.java new file mode 100644 index 0000000..9640875 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/sasl/SparkSaslClient.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.sasl; + +import java.util.Map; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.RealmCallback; +import javax.security.sasl.RealmChoiceCallback; +import javax.security.sasl.Sasl; +import javax.security.sasl.SaslClient; +import javax.security.sasl.SaslException; + +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static com.bytedance.css.network.sasl.SparkSaslServer.*; + +/** + * A SASL Client for Spark which simply keeps track of the state of a single SASL session, from the + * initial state to the "authenticated" state. This client initializes the protocol via a + * firstToken, which is then followed by a set of challenges and responses. + */ +public class SparkSaslClient implements SaslEncryptionBackend { + private static final Logger logger = LoggerFactory.getLogger(SparkSaslClient.class); + + private final String secretKeyId; + private final SecretKeyHolder secretKeyHolder; + private final String expectedQop; + private SaslClient saslClient; + + public SparkSaslClient(String secretKeyId, SecretKeyHolder secretKeyHolder, boolean encrypt) { + this.secretKeyId = secretKeyId; + this.secretKeyHolder = secretKeyHolder; + this.expectedQop = encrypt ? QOP_AUTH_CONF : QOP_AUTH; + + Map saslProps = ImmutableMap.builder() + .put(Sasl.QOP, expectedQop) + .build(); + try { + this.saslClient = Sasl.createSaslClient(new String[] { DIGEST }, null, null, DEFAULT_REALM, + saslProps, new ClientCallbackHandler()); + } catch (SaslException e) { + throw Throwables.propagate(e); + } + } + + /** Used to initiate SASL handshake with server. */ + public synchronized byte[] firstToken() { + if (saslClient != null && saslClient.hasInitialResponse()) { + try { + return saslClient.evaluateChallenge(new byte[0]); + } catch (SaslException e) { + throw Throwables.propagate(e); + } + } else { + return new byte[0]; + } + } + + /** Determines whether the authentication exchange has completed. */ + public synchronized boolean isComplete() { + return saslClient != null && saslClient.isComplete(); + } + + /** Returns the value of a negotiated property. */ + public Object getNegotiatedProperty(String name) { + return saslClient.getNegotiatedProperty(name); + } + + /** + * Respond to server's SASL token. + * @param token contains server's SASL token + * @return client's response SASL token + */ + public synchronized byte[] response(byte[] token) { + try { + return saslClient != null ? saslClient.evaluateChallenge(token) : new byte[0]; + } catch (SaslException e) { + throw Throwables.propagate(e); + } + } + + /** + * Disposes of any system resources or security-sensitive information the + * SaslClient might be using. + */ + @Override + public synchronized void dispose() { + if (saslClient != null) { + try { + saslClient.dispose(); + } catch (SaslException e) { + // ignore + } finally { + saslClient = null; + } + } + } + + /** + * Implementation of javax.security.auth.callback.CallbackHandler + * that works with share secrets. + */ + private class ClientCallbackHandler implements CallbackHandler { + @Override + public void handle(Callback[] callbacks) throws UnsupportedCallbackException { + + for (Callback callback : callbacks) { + if (callback instanceof NameCallback) { + logger.trace("SASL client callback: setting username"); + NameCallback nc = (NameCallback) callback; + nc.setName(encodeIdentifier(secretKeyHolder.getSaslUser(secretKeyId))); + } else if (callback instanceof PasswordCallback) { + logger.trace("SASL client callback: setting password"); + PasswordCallback pc = (PasswordCallback) callback; + pc.setPassword(encodePassword(secretKeyHolder.getSecretKey(secretKeyId))); + } else if (callback instanceof RealmCallback) { + logger.trace("SASL client callback: setting realm"); + RealmCallback rc = (RealmCallback) callback; + rc.setText(rc.getDefaultText()); + } else if (callback instanceof RealmChoiceCallback) { + // ignore (?) + } else { + throw new UnsupportedCallbackException(callback, "Unrecognized SASL DIGEST-MD5 Callback"); + } + } + } + } + + @Override + public byte[] wrap(byte[] data, int offset, int len) throws SaslException { + return saslClient.wrap(data, offset, len); + } + + @Override + public byte[] unwrap(byte[] data, int offset, int len) throws SaslException { + return saslClient.unwrap(data, offset, len); + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/sasl/SparkSaslServer.java b/network-common/src/main/java/com/bytedance/css/network/sasl/SparkSaslServer.java new file mode 100644 index 0000000..170d8ad --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/sasl/SparkSaslServer.java @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.sasl; + +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; +import javax.security.sasl.Sasl; +import javax.security.sasl.SaslException; +import javax.security.sasl.SaslServer; +import java.nio.charset.StandardCharsets; +import java.util.Map; + +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.handler.codec.base64.Base64; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A SASL Server for Spark which simply keeps track of the state of a single SASL session, from the + * initial state to the "authenticated" state. (It is not a server in the sense of accepting + * connections on some socket.) + */ +public class SparkSaslServer implements SaslEncryptionBackend { + private static final Logger logger = LoggerFactory.getLogger(SparkSaslServer.class); + + /** + * This is passed as the server name when creating the sasl client/server. + * This could be changed to be configurable in the future. + */ + static final String DEFAULT_REALM = "default"; + + /** + * The authentication mechanism used here is DIGEST-MD5. This could be changed to be + * configurable in the future. + */ + static final String DIGEST = "DIGEST-MD5"; + + /** + * Quality of protection value that includes encryption. + */ + static final String QOP_AUTH_CONF = "auth-conf"; + + /** + * Quality of protection value that does not include encryption. + */ + static final String QOP_AUTH = "auth"; + + /** Identifier for a certain secret key within the secretKeyHolder. */ + private final String secretKeyId; + private final SecretKeyHolder secretKeyHolder; + private SaslServer saslServer; + + public SparkSaslServer( + String secretKeyId, + SecretKeyHolder secretKeyHolder, + boolean alwaysEncrypt) { + this.secretKeyId = secretKeyId; + this.secretKeyHolder = secretKeyHolder; + + // Sasl.QOP is a comma-separated list of supported values. The value that allows encryption + // is listed first since it's preferred over the non-encrypted one (if the client also + // lists both in the request). + String qop = alwaysEncrypt ? QOP_AUTH_CONF : String.format("%s,%s", QOP_AUTH_CONF, QOP_AUTH); + Map saslProps = ImmutableMap.builder() + .put(Sasl.SERVER_AUTH, "true") + .put(Sasl.QOP, qop) + .build(); + try { + this.saslServer = Sasl.createSaslServer(DIGEST, null, DEFAULT_REALM, saslProps, + new DigestCallbackHandler()); + } catch (SaslException e) { + throw Throwables.propagate(e); + } + } + + /** + * Determines whether the authentication exchange has completed successfully. + */ + public synchronized boolean isComplete() { + return saslServer != null && saslServer.isComplete(); + } + + /** Returns the value of a negotiated property. */ + public Object getNegotiatedProperty(String name) { + return saslServer.getNegotiatedProperty(name); + } + + /** + * Used to respond to server SASL tokens. + * @param token Server's SASL token + * @return response to send back to the server. + */ + public synchronized byte[] response(byte[] token) { + try { + return saslServer != null ? saslServer.evaluateResponse(token) : new byte[0]; + } catch (SaslException e) { + throw Throwables.propagate(e); + } + } + + /** + * Disposes of any system resources or security-sensitive information the + * SaslServer might be using. + */ + @Override + public synchronized void dispose() { + if (saslServer != null) { + try { + saslServer.dispose(); + } catch (SaslException e) { + // ignore + } finally { + saslServer = null; + } + } + } + + @Override + public byte[] wrap(byte[] data, int offset, int len) throws SaslException { + return saslServer.wrap(data, offset, len); + } + + @Override + public byte[] unwrap(byte[] data, int offset, int len) throws SaslException { + return saslServer.unwrap(data, offset, len); + } + + /** + * Implementation of javax.security.auth.callback.CallbackHandler for SASL DIGEST-MD5 mechanism. + */ + private class DigestCallbackHandler implements CallbackHandler { + @Override + public void handle(Callback[] callbacks) throws UnsupportedCallbackException { + for (Callback callback : callbacks) { + if (callback instanceof NameCallback) { + logger.trace("SASL server callback: setting username"); + NameCallback nc = (NameCallback) callback; + nc.setName(encodeIdentifier(secretKeyHolder.getSaslUser(secretKeyId))); + } else if (callback instanceof PasswordCallback) { + logger.trace("SASL server callback: setting password"); + PasswordCallback pc = (PasswordCallback) callback; + pc.setPassword(encodePassword(secretKeyHolder.getSecretKey(secretKeyId))); + } else if (callback instanceof RealmCallback) { + logger.trace("SASL server callback: setting realm"); + RealmCallback rc = (RealmCallback) callback; + rc.setText(rc.getDefaultText()); + } else if (callback instanceof AuthorizeCallback) { + AuthorizeCallback ac = (AuthorizeCallback) callback; + String authId = ac.getAuthenticationID(); + String authzId = ac.getAuthorizationID(); + ac.setAuthorized(authId.equals(authzId)); + if (ac.isAuthorized()) { + ac.setAuthorizedID(authzId); + } + logger.debug("SASL Authorization complete, authorized set to {}", ac.isAuthorized()); + } else { + throw new UnsupportedCallbackException(callback, "Unrecognized SASL DIGEST-MD5 Callback"); + } + } + } + } + + /* Encode a byte[] identifier as a Base64-encoded string. */ + public static String encodeIdentifier(String identifier) { + Preconditions.checkNotNull(identifier, "User cannot be null if SASL is enabled"); + return getBase64EncodedString(identifier); + } + + /** Encode a password as a base64-encoded char[] array. */ + public static char[] encodePassword(String password) { + Preconditions.checkNotNull(password, "Password cannot be null if SASL is enabled"); + return getBase64EncodedString(password).toCharArray(); + } + + /** Return a Base64-encoded string. */ + private static String getBase64EncodedString(String str) { + ByteBuf byteBuf = null; + ByteBuf encodedByteBuf = null; + try { + byteBuf = Unpooled.wrappedBuffer(str.getBytes(StandardCharsets.UTF_8)); + encodedByteBuf = Base64.encode(byteBuf); + return encodedByteBuf.toString(StandardCharsets.UTF_8); + } finally { + // The release is called to suppress the memory leak error messages raised by netty. + if (byteBuf != null) { + byteBuf.release(); + if (encodedByteBuf != null) { + encodedByteBuf.release(); + } + } + } + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/server/AbstractAuthRpcHandler.java b/network-common/src/main/java/com/bytedance/css/network/server/AbstractAuthRpcHandler.java new file mode 100644 index 0000000..e60c516 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/server/AbstractAuthRpcHandler.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.server; + +import java.nio.ByteBuffer; + +import com.bytedance.css.network.client.RpcResponseCallback; +import com.bytedance.css.network.client.StreamCallbackWithID; +import com.bytedance.css.network.client.TransportClient; + +/** + * RPC Handler which performs authentication, and when it's successful, delegates further + * calls to another RPC handler. The authentication handshake itself should be implemented + * by subclasses. + */ +public abstract class AbstractAuthRpcHandler extends RpcHandler { + /** RpcHandler we will delegate to for authenticated connections. */ + private final RpcHandler delegate; + + private boolean isAuthenticated; + + protected AbstractAuthRpcHandler(RpcHandler delegate) { + this.delegate = delegate; + } + + /** + * Responds to an authentication challenge. + * + * @return Whether the client is authenticated. + */ + protected abstract boolean doAuthChallenge( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback); + + @Override + public final void receive( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { + if (isAuthenticated) { + delegate.receive(client, message, callback); + } else { + isAuthenticated = doAuthChallenge(client, message, callback); + } + } + + @Override + public final void receive(TransportClient client, ByteBuffer message) { + if (isAuthenticated) { + delegate.receive(client, message); + } else { + throw new SecurityException("Unauthenticated call to receive()."); + } + } + + @Override + public final StreamCallbackWithID receiveStream( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { + if (isAuthenticated) { + return delegate.receiveStream(client, message, callback); + } else { + throw new SecurityException("Unauthenticated call to receiveStream()."); + } + } + + @Override + public StreamManager getStreamManager() { + return delegate.getStreamManager(); + } + + @Override + public void channelActive(TransportClient client) { + delegate.channelActive(client); + } + + @Override + public void channelInactive(TransportClient client) { + delegate.channelInactive(client); + } + + @Override + public void exceptionCaught(Throwable cause, TransportClient client) { + delegate.exceptionCaught(cause, client); + } + + public boolean isAuthenticated() { + return isAuthenticated; + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/server/ChunkFetchMetrics.java b/network-common/src/main/java/com/bytedance/css/network/server/ChunkFetchMetrics.java new file mode 100644 index 0000000..f4de878 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/server/ChunkFetchMetrics.java @@ -0,0 +1,42 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.network.server; + +import com.codahale.metrics.Meter; +import com.codahale.metrics.Timer; + +public class ChunkFetchMetrics { + + public Meter getChunkFetchQps() { + return new Meter(); + } + + public Timer getChunkFetchLatency() { + return new Timer(); + } + + public Meter getChunkFetchFailedQPS() { + return new Meter(); + } + + public Meter getChunkFetchThroughput() { + return new Meter(); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/server/ChunkFetchRequestHandler.java b/network-common/src/main/java/com/bytedance/css/network/server/ChunkFetchRequestHandler.java new file mode 100644 index 0000000..5b70d57 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/server/ChunkFetchRequestHandler.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.server; + +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.protocol.ChunkFetchFailure; +import com.bytedance.css.network.protocol.ChunkFetchRequest; +import com.bytedance.css.network.protocol.ChunkFetchSuccess; +import com.bytedance.css.network.protocol.Encodable; +import com.bytedance.css.network.util.NettyUtils; +import com.codahale.metrics.Timer; +import com.google.common.base.Throwables; +import io.netty.channel.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.SocketAddress; + +/** + * A dedicated ChannelHandler for processing ChunkFetchRequest messages. When sending response + * of ChunkFetchRequest messages to the clients, the thread performing the I/O on the underlying + * channel could potentially be blocked due to disk contentions. If several hundreds of clients + * send ChunkFetchRequest to the server at the same time, it could potentially occupying all + * threads from TransportServer's default EventLoopGroup for waiting for disk reads before it + * can send the block data back to the client as part of the ChunkFetchSuccess messages. As a + * result, it would leave no threads left to process other RPC messages, which takes much less + * time to process, and could lead to client timing out on either performing SASL authentication, + * registering executors, or waiting for response for an OpenBlocks messages. + */ +public class ChunkFetchRequestHandler extends SimpleChannelInboundHandler { + private static final Logger logger = LoggerFactory.getLogger(ChunkFetchRequestHandler.class); + + private final TransportClient client; + private final StreamManager streamManager; + /** The max number of chunks being transferred and not finished yet. */ + private final long maxChunksBeingTransferred; + private final boolean syncModeEnabled; + + public ChunkFetchRequestHandler( + TransportClient client, + StreamManager streamManager, + Long maxChunksBeingTransferred, + boolean syncModeEnabled) { + this.client = client; + this.streamManager = streamManager; + this.maxChunksBeingTransferred = maxChunksBeingTransferred; + this.syncModeEnabled = syncModeEnabled; + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + logger.warn("Exception in connection from " + NettyUtils.getRemoteAddress(ctx.channel()), cause); + ctx.close(); + } + + @Override + protected void channelRead0( + ChannelHandlerContext ctx, + final ChunkFetchRequest msg) throws Exception { + Channel channel = ctx.channel(); + processFetchRequest(channel, msg); + } + + public void processFetchRequest( + final Channel channel, final ChunkFetchRequest msg) throws Exception { + if (logger.isTraceEnabled()) { + logger.trace("Received req from {} to fetch block {}", NettyUtils.getRemoteAddress(channel), + msg.streamChunkId); + } + streamManager.chunkFetchMetrics.getChunkFetchQps().mark(); + Timer.Context latencyTimer = streamManager.chunkFetchMetrics.getChunkFetchLatency().time(); + if (maxChunksBeingTransferred < Long.MAX_VALUE) { + long chunksBeingTransferred = streamManager.chunksBeingTransferred(); + if (chunksBeingTransferred >= maxChunksBeingTransferred) { + logger.warn("The number of chunks being transferred {} is above {}, close the connection.", + chunksBeingTransferred, maxChunksBeingTransferred); + channel.close(); + return; + } + } + ManagedBuffer buf; + try { + streamManager.checkAuthorization(client, msg.streamChunkId.streamId); + buf = streamManager.getChunk(msg.streamChunkId.streamId, msg.streamChunkId.chunkIndex); + if (buf == null) { + throw new IllegalStateException("Chunk was not found"); + } + } catch (Exception e) { + logger.error(String.format("Error opening block %s for request from %s", + msg.streamChunkId, NettyUtils.getRemoteAddress(channel)), e); + streamManager.chunkFetchMetrics.getChunkFetchFailedQPS().mark(); + respond(channel, new ChunkFetchFailure(msg.streamChunkId, + Throwables.getStackTraceAsString(e))); + return; + } + + streamManager.chunkBeingSent(msg.streamChunkId.streamId); + long bufSize = buf.size(); + respond(channel, new ChunkFetchSuccess(msg.streamChunkId, buf)).addListener(future -> { + latencyTimer.stop(); + streamManager.chunkFetchMetrics.getChunkFetchThroughput().mark(bufSize); + streamManager.chunkSent(msg.streamChunkId.streamId); + }); + } + + /** + * The invocation to channel.writeAndFlush is async, and the actual I/O on the + * channel will be handled by the EventLoop the channel is registered to. So even + * though we are processing the ChunkFetchRequest in a separate thread pool, the actual I/O, + * which is the potentially blocking call that could deplete server handler threads, is still + * being processed by TransportServer's default EventLoopGroup. + * + * When syncModeEnabled is true, Spark will throttle the max number of threads that channel I/O + * for sending response to ChunkFetchRequest, the thread calling channel.writeAndFlush will wait + * for the completion of sending response back to client by invoking await(). This will throttle + * the rate at which threads from ChunkFetchRequest dedicated EventLoopGroup submit channel I/O + * requests to TransportServer's default EventLoopGroup, thus making sure that we can reserve + * some threads in TransportServer's default EventLoopGroup for handling other RPC messages. + */ + private ChannelFuture respond( + final Channel channel, + final Encodable result) throws InterruptedException { + final SocketAddress remoteAddress = channel.remoteAddress(); + ChannelFuture channelFuture; + if (syncModeEnabled) { + channelFuture = channel.writeAndFlush(result).await(); + } else { + channelFuture = channel.writeAndFlush(result); + } + return channelFuture.addListener((ChannelFutureListener) future -> { + if (future.isSuccess()) { + logger.trace("Sent result {} to client {}", result, remoteAddress); + } else { + logger.error(String.format("Error sending result %s to %s; closing connection", + result, remoteAddress), future.cause()); + channel.close(); + } + }); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/server/CssFileInfo.java b/network-common/src/main/java/com/bytedance/css/network/server/CssFileInfo.java new file mode 100644 index 0000000..e164db0 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/server/CssFileInfo.java @@ -0,0 +1,37 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.network.server; + +import java.io.File; +import java.util.ArrayList; + +public class CssFileInfo { + public final File file; + public final ArrayList chunkOffsets; + public final long fileLength; + public final int numChunks; + + public CssFileInfo(File file, ArrayList chunkOffsets, long fileLength) { + this.file = file; + this.chunkOffsets = chunkOffsets; + this.fileLength = fileLength; + this.numChunks = chunkOffsets.size() - 1; + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/server/CssManagedBufferIterator.java b/network-common/src/main/java/com/bytedance/css/network/server/CssManagedBufferIterator.java new file mode 100644 index 0000000..144d189 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/server/CssManagedBufferIterator.java @@ -0,0 +1,69 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.network.server; + +import com.bytedance.css.network.buffer.FileSegmentManagedBuffer; +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.util.TransportConf; + +import java.io.File; +import java.io.IOException; +import java.util.Iterator; + +public final class CssManagedBufferIterator implements Iterator { + private final File file; + private final long[] offsets; + private final int numChunks; + private final TransportConf conf; + + private int index = 0; + + public CssManagedBufferIterator(CssFileInfo fileInfo, TransportConf conf) throws IOException { + this.file = fileInfo.file; + this.conf = conf; + this.numChunks = fileInfo.numChunks; + this.offsets = new long[numChunks + 1]; + for (int i = 0; i <= numChunks; i++) { + offsets[i] = fileInfo.chunkOffsets.get(i); + } + if (offsets[numChunks] != fileInfo.fileLength) { + throw new IOException( + String.format("The last chunk offset %d should be equals to file length %d!", + offsets[numChunks], fileInfo.fileLength)); + } + } + + public void setInitIndex(int index) { + this.index = index; + } + + @Override + public boolean hasNext() { + return index < numChunks; + } + + @Override + public ManagedBuffer next() { + final long offset = offsets[index]; + final long length = offsets[index + 1] - offset; + index++; + return new FileSegmentManagedBuffer(conf, file, offset, length); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/server/MessageHandler.java b/network-common/src/main/java/com/bytedance/css/network/server/MessageHandler.java new file mode 100644 index 0000000..c7d3739 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/server/MessageHandler.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.server; + +import com.bytedance.css.network.protocol.Message; + +/** + * Handles either request or response messages coming off of Netty. A MessageHandler instance + * is associated with a single Netty Channel (though it may have multiple clients on the same + * Channel.) + */ +public abstract class MessageHandler { + /** Handles the receipt of a single message. */ + public abstract void handle(T message) throws Exception; + + /** Invoked when the channel this MessageHandler is on is active. */ + public abstract void channelActive(); + + /** Invoked when an exception was caught on the Channel. */ + public abstract void exceptionCaught(Throwable cause); + + /** Invoked when the channel this MessageHandler is on is inactive. */ + public abstract void channelInactive(); +} diff --git a/network-common/src/main/java/com/bytedance/css/network/server/NoOpRpcHandler.java b/network-common/src/main/java/com/bytedance/css/network/server/NoOpRpcHandler.java new file mode 100644 index 0000000..d55d51a --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/server/NoOpRpcHandler.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.server; + +import java.nio.ByteBuffer; + +import com.bytedance.css.network.client.RpcResponseCallback; +import com.bytedance.css.network.client.TransportClient; + +/** An RpcHandler suitable for a client-only TransportContext, which cannot receive RPCs. */ +public class NoOpRpcHandler extends RpcHandler { + private final StreamManager streamManager; + + public NoOpRpcHandler() { + streamManager = new OneForOneStreamManager(); + } + + @Override + public void receive(TransportClient client, ByteBuffer message, RpcResponseCallback callback) { + throw new UnsupportedOperationException("Cannot handle messages"); + } + + @Override + public StreamManager getStreamManager() { return streamManager; } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/server/OneForOneStreamManager.java b/network-common/src/main/java/com/bytedance/css/network/server/OneForOneStreamManager.java new file mode 100644 index 0000000..6a3a7ae --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/server/OneForOneStreamManager.java @@ -0,0 +1,233 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.server; + +import java.util.Iterator; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import io.netty.channel.Channel; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.client.TransportClient; + +/** + * StreamManager which allows registration of an Iterator<ManagedBuffer>, which are + * individually fetched as chunks by the client. Each registered buffer is one chunk. + */ +public class OneForOneStreamManager extends StreamManager { + private static final Logger logger = LoggerFactory.getLogger(OneForOneStreamManager.class); + + private final AtomicLong nextStreamId; + private final ConcurrentHashMap streams; + + /** State of a single stream. */ + private static class StreamState { + final String appId; + final Iterator buffers; + + // The channel associated to the stream + final Channel associatedChannel; + + // Used to keep track of the index of the buffer that the user has retrieved, just to ensure + // that the caller only requests each chunk one at a time, in order. + int curChunk = 0; + + // Used to keep track of the number of chunks being transferred and not finished yet. + final AtomicLong chunksBeingTransferred = new AtomicLong(0L); + + StreamState(String appId, Iterator buffers, Channel channel) { + this.appId = appId; + this.buffers = Preconditions.checkNotNull(buffers); + this.associatedChannel = channel; + } + } + + public OneForOneStreamManager() { + // For debugging purposes, start with a random stream id to help identifying different streams. + // This does not need to be globally unique, only unique to this class. + nextStreamId = new AtomicLong((long) new Random().nextInt(Integer.MAX_VALUE) * 1000); + streams = new ConcurrentHashMap<>(); + } + + @Override + public ManagedBuffer getChunk(long streamId, int chunkIndex) { + StreamState state = streams.get(streamId); + if (chunkIndex != state.curChunk) { + throw new IllegalStateException(String.format( + "Received out-of-order chunk index %s (expected %s)", chunkIndex, state.curChunk)); + } else if (!state.buffers.hasNext()) { + throw new IllegalStateException(String.format( + "Requested chunk index beyond end %s", chunkIndex)); + } + state.curChunk += 1; + ManagedBuffer nextChunk = state.buffers.next(); + + if (!state.buffers.hasNext()) { + logger.trace("Removing stream id {}", streamId); + streams.remove(streamId); + } + + return nextChunk; + } + + @Override + public ManagedBuffer openStream(String streamChunkId) { + Pair streamChunkIdPair = parseStreamChunkId(streamChunkId); + return getChunk(streamChunkIdPair.getLeft(), streamChunkIdPair.getRight()); + } + + public static String genStreamChunkId(long streamId, int chunkId) { + return String.format("%d_%d", streamId, chunkId); + } + + // Parse streamChunkId to be stream id and chunk id. This is used when fetch remote chunk as a + // stream. + public static Pair parseStreamChunkId(String streamChunkId) { + String[] array = streamChunkId.split("_"); + assert array.length == 2: + "Stream id and chunk index should be specified."; + long streamId = Long.valueOf(array[0]); + int chunkIndex = Integer.valueOf(array[1]); + return ImmutablePair.of(streamId, chunkIndex); + } + + @Override + public void connectionTerminated(Channel channel) { + RuntimeException failedToReleaseBufferException = null; + + // Close all streams which have been associated with the channel. + for (Map.Entry entry: streams.entrySet()) { + StreamState state = entry.getValue(); + if (state.associatedChannel == channel) { + streams.remove(entry.getKey()); + + if (!(state.buffers instanceof CssManagedBufferIterator)) { + try { + // Release all remaining buffers. + while (state.buffers.hasNext()) { + ManagedBuffer buffer = state.buffers.next(); + if (buffer != null) { + buffer.release(); + } + } + } catch (RuntimeException e) { + if (failedToReleaseBufferException == null) { + failedToReleaseBufferException = e; + } else { + logger.error("Exception trying to release remaining StreamState buffers", e); + } + } + } + } + } + + if (failedToReleaseBufferException != null) { + throw failedToReleaseBufferException; + } + } + + @Override + public void checkAuthorization(TransportClient client, long streamId) { + if (client.getClientId() != null) { + StreamState state = streams.get(streamId); + Preconditions.checkArgument(state != null, "Unknown stream ID."); + if (!client.getClientId().equals(state.appId)) { + throw new SecurityException(String.format( + "Client %s not authorized to read stream %d (app %s).", + client.getClientId(), + streamId, + state.appId)); + } + } + } + + @Override + public void chunkBeingSent(long streamId) { + StreamState streamState = streams.get(streamId); + if (streamState != null) { + streamState.chunksBeingTransferred.incrementAndGet(); + } + + } + + @Override + public void streamBeingSent(String streamId) { + chunkBeingSent(parseStreamChunkId(streamId).getLeft()); + } + + @Override + public void chunkSent(long streamId) { + StreamState streamState = streams.get(streamId); + if (streamState != null) { + streamState.chunksBeingTransferred.decrementAndGet(); + } + } + + @Override + public void streamSent(String streamId) { + chunkSent(OneForOneStreamManager.parseStreamChunkId(streamId).getLeft()); + } + + @Override + public long chunksBeingTransferred() { + long sum = 0L; + for (StreamState streamState: streams.values()) { + sum += streamState.chunksBeingTransferred.get(); + } + return sum; + } + + /** + * Registers a stream of ManagedBuffers which are served as individual chunks one at a time to + * callers. Each ManagedBuffer will be release()'d after it is transferred on the wire. If a + * client connection is closed before the iterator is fully drained, then the remaining buffers + * will all be release()'d. + * + * If an app ID is provided, only callers who've authenticated with the given app ID will be + * allowed to fetch from this stream. + * + * This method also associates the stream with a single client connection, which is guaranteed + * to be the only reader of the stream. Once the connection is closed, the stream will never + * be used again, enabling cleanup by `connectionTerminated`. + */ + public long registerStream(String appId, Iterator buffers, Channel channel) { + long myStreamId = nextStreamId.getAndIncrement(); + streams.put(myStreamId, new StreamState(appId, buffers, channel)); + return myStreamId; + } + + public void setStreamStateCurIndex(long streamId, int chunkIndex) { + streams.get(streamId).curChunk = chunkIndex; + } + + @VisibleForTesting + public int numStreamStates() { + return streams.size(); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/server/RpcHandler.java b/network-common/src/main/java/com/bytedance/css/network/server/RpcHandler.java new file mode 100644 index 0000000..b0a369c --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/server/RpcHandler.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.server; + +import java.nio.ByteBuffer; + +import com.bytedance.css.network.protocol.BatchPushDataRequest; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.bytedance.css.network.client.RpcResponseCallback; +import com.bytedance.css.network.client.StreamCallbackWithID; +import com.bytedance.css.network.client.TransportClient; + +/** + * Handler for sendRPC() messages sent by {@link com.bytedance.css.network.client.TransportClient}s. + */ +public abstract class RpcHandler { + + private static final RpcResponseCallback ONE_WAY_CALLBACK = new OneWayRpcCallback(); + + /** + * Receive a single RPC message. Any exception thrown while in this method will be sent back to + * the client in string form as a standard RPC failure. + * + * Neither this method nor #receiveStream will be called in parallel for a single + * TransportClient (i.e., channel). + * + * @param client A channel client which enables the handler to make requests back to the sender + * of this RPC. This will always be the exact same object for a particular channel. + * @param message The serialized bytes of the RPC. + * @param callback Callback which should be invoked exactly once upon success or failure of the + * RPC. + */ + public abstract void receive( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback); + + /** + * Receive a single RPC message which includes data that is to be received as a stream. Any + * exception thrown while in this method will be sent back to the client in string form as a + * standard RPC failure. + * + * Neither this method nor #receive will be called in parallel for a single TransportClient + * (i.e., channel). + * + * An error while reading data from the stream + * ({@link com.bytedance.css.network.client.StreamCallback#onData(String, ByteBuffer)}) + * will fail the entire channel. A failure in "post-processing" the stream in + * {@link com.bytedance.css.network.client.StreamCallback#onComplete(String)} will result in an + * rpcFailure, but the channel will remain active. + * + * @param client A channel client which enables the handler to make requests back to the sender + * of this RPC. This will always be the exact same object for a particular channel. + * @param messageHeader The serialized bytes of the header portion of the RPC. This is in meant + * to be relatively small, and will be buffered entirely in memory, to + * facilitate how the streaming portion should be received. + * @param callback Callback which should be invoked exactly once upon success or failure of the + * RPC. + * @return a StreamCallback for handling the accompanying streaming data + */ + public StreamCallbackWithID receiveStream( + TransportClient client, + ByteBuffer messageHeader, + RpcResponseCallback callback) { + throw new UnsupportedOperationException(); + } + + /** + * Returns the StreamManager which contains the state about which streams are currently being + * fetched by a TransportClient. + */ + public abstract StreamManager getStreamManager(); + + public void receiveBatchPushDataRequest( + TransportClient client, + BatchPushDataRequest req, + RpcResponseCallback callback) { + throw new UnsupportedOperationException(); + } + + /** + * Receives an RPC message that does not expect a reply. The default implementation will + * call "{@link #receive(TransportClient, ByteBuffer, RpcResponseCallback)}" and log a warning if + * any of the callback methods are called. + * + * @param client A channel client which enables the handler to make requests back to the sender + * of this RPC. This will always be the exact same object for a particular channel. + * @param message The serialized bytes of the RPC. + */ + public void receive(TransportClient client, ByteBuffer message) { + receive(client, message, ONE_WAY_CALLBACK); + } + + /** + * Invoked when the channel associated with the given client is active. + */ + public void channelActive(TransportClient client) { } + + /** + * Invoked when the channel associated with the given client is inactive. + * No further requests will come from this client. + */ + public void channelInactive(TransportClient client) { } + + public void exceptionCaught(Throwable cause, TransportClient client) { } + + private static class OneWayRpcCallback implements RpcResponseCallback { + + private static final Logger logger = LoggerFactory.getLogger(OneWayRpcCallback.class); + + @Override + public void onSuccess(ByteBuffer response) { + logger.warn("Response provided for one-way RPC."); + } + + @Override + public void onFailure(Throwable e) { + logger.error("Error response provided for one-way RPC.", e); + } + + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/server/StreamManager.java b/network-common/src/main/java/com/bytedance/css/network/server/StreamManager.java new file mode 100644 index 0000000..24d267b --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/server/StreamManager.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.server; + +import io.netty.channel.Channel; + +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.client.TransportClient; + +/** + * The StreamManager is used to fetch individual chunks from a stream. This is used in + * {@link TransportRequestHandler} in order to respond to fetchChunk() requests. Creation of the + * stream is outside the scope of the transport layer, but a given stream is guaranteed to be read + * by only one client connection, meaning that getChunk() for a particular stream will be called + * serially and that once the connection associated with the stream is closed, that stream will + * never be used again. + */ +public abstract class StreamManager { + /** + * Called in response to a fetchChunk() request. The returned buffer will be passed as-is to the + * client. A single stream will be associated with a single TCP connection, so this method + * will not be called in parallel for a particular stream. + * + * Chunks may be requested in any order, and requests may be repeated, but it is not required + * that implementations support this behavior. + * + * The returned ManagedBuffer will be release()'d after being written to the network. + * + * @param streamId id of a stream that has been previously registered with the StreamManager. + * @param chunkIndex 0-indexed chunk of the stream that's requested + */ + public abstract ManagedBuffer getChunk(long streamId, int chunkIndex); + + /** + * Called in response to a stream() request. The returned data is streamed to the client + * through a single TCP connection. + * + * Note the streamId argument is not related to the similarly named argument in the + * {@link #getChunk(long, int)} method. + * + * @param streamId id of a stream that has been previously registered with the StreamManager. + * @return A managed buffer for the stream, or null if the stream was not found. + */ + public ManagedBuffer openStream(String streamId) { + throw new UnsupportedOperationException(); + } + + /** + * Indicates that the given channel has been terminated. After this occurs, we are guaranteed not + * to read from the associated streams again, so any state can be cleaned up. + */ + public void connectionTerminated(Channel channel) { } + + /** + * Verify that the client is authorized to read from the given stream. + * + * @throws SecurityException If client is not authorized. + */ + public void checkAuthorization(TransportClient client, long streamId) { } + + /** + * Return the number of chunks being transferred and not finished yet in this StreamManager. + */ + public long chunksBeingTransferred() { + return 0; + } + + /** + * Called when start sending a chunk. + */ + public void chunkBeingSent(long streamId) { } + + /** + * Called when start sending a stream. + */ + public void streamBeingSent(String streamId) { } + + /** + * Called when a chunk is successfully sent. + */ + public void chunkSent(long streamId) { } + + /** + * Called when a stream is successfully sent. + */ + public void streamSent(String streamId) { } + + /** + * Chunk fetch metrics. + */ + public ChunkFetchMetrics chunkFetchMetrics = new ChunkFetchMetrics(); +} diff --git a/network-common/src/main/java/com/bytedance/css/network/server/TransportChannelHandler.java b/network-common/src/main/java/com/bytedance/css/network/server/TransportChannelHandler.java new file mode 100644 index 0000000..6d91c5b --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/server/TransportChannelHandler.java @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.server; + +import com.bytedance.css.network.TransportContext; +import com.bytedance.css.network.protocol.ChunkFetchRequest; +import com.bytedance.css.network.protocol.Message; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.handler.timeout.IdleState; +import io.netty.handler.timeout.IdleStateEvent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.client.TransportResponseHandler; +import com.bytedance.css.network.protocol.RequestMessage; +import com.bytedance.css.network.protocol.ResponseMessage; +import static com.bytedance.css.network.util.NettyUtils.getRemoteAddress; + +/** + * The single Transport-level Channel handler which is used for delegating requests to the + * {@link TransportRequestHandler} and responses to the {@link TransportResponseHandler}. + * + * All channels created in the transport layer are bidirectional. When the Client initiates a Netty + * Channel with a RequestMessage (which gets handled by the Server's RequestHandler), the Server + * will produce a ResponseMessage (handled by the Client's ResponseHandler). However, the Server + * also gets a handle on the same Channel, so it may then begin to send RequestMessages to the + * Client. + * This means that the Client also needs a RequestHandler and the Server needs a ResponseHandler, + * for the Client's responses to the Server's requests. + * + * This class also handles timeouts from a {@link io.netty.handler.timeout.IdleStateHandler}. + * We consider a connection timed out if there are outstanding fetch or RPC requests but no traffic + * on the channel for at least `requestTimeoutMs`. Note that this is duplex traffic; we will not + * timeout if the client is continuously sending but getting no responses, for simplicity. + */ +public class TransportChannelHandler extends SimpleChannelInboundHandler { + private static final Logger logger = LoggerFactory.getLogger(TransportChannelHandler.class); + + private final TransportClient client; + private final TransportResponseHandler responseHandler; + private final TransportRequestHandler requestHandler; + private final long requestTimeoutNs; + private final boolean closeIdleConnections; + private final boolean skipChunkFetchRequest; + private final TransportContext transportContext; + + public TransportChannelHandler( + TransportClient client, + TransportResponseHandler responseHandler, + TransportRequestHandler requestHandler, + long requestTimeoutMs, + boolean skipChunkFetchRequest, + boolean closeIdleConnections, + TransportContext transportContext) { + this.client = client; + this.responseHandler = responseHandler; + this.requestHandler = requestHandler; + this.requestTimeoutNs = requestTimeoutMs * 1000L * 1000; + this.skipChunkFetchRequest = skipChunkFetchRequest; + this.closeIdleConnections = closeIdleConnections; + this.transportContext = transportContext; + } + + public TransportClient getClient() { + return client; + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + logger.warn("Exception in connection from " + getRemoteAddress(ctx.channel()), + cause); + requestHandler.exceptionCaught(cause); + responseHandler.exceptionCaught(cause); + ctx.close(); + } + + @Override + public void channelActive(ChannelHandlerContext ctx) throws Exception { + try { + requestHandler.channelActive(); + } catch (RuntimeException e) { + logger.error("Exception from request handler while channel is active", e); + } + try { + responseHandler.channelActive(); + } catch (RuntimeException e) { + logger.error("Exception from response handler while channel is active", e); + } + super.channelActive(ctx); + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + try { + requestHandler.channelInactive(); + } catch (RuntimeException e) { + logger.error("Exception from request handler while channel is inactive", e); + } + try { + responseHandler.channelInactive(); + } catch (RuntimeException e) { + logger.error("Exception from response handler while channel is inactive", e); + } + super.channelInactive(ctx); + } + + /** + * Overwrite acceptInboundMessage to properly delegate ChunkFetchRequest messages + * to ChunkFetchRequestHandler. + */ + @Override + public boolean acceptInboundMessage(Object msg) throws Exception { + if (skipChunkFetchRequest && msg instanceof ChunkFetchRequest) { + return false; + } else { + return super.acceptInboundMessage(msg); + } + } + + @Override + public void channelRead0(ChannelHandlerContext ctx, Message request) throws Exception { + if (request instanceof RequestMessage) { + requestHandler.handle((RequestMessage) request); + } else if (request instanceof ResponseMessage) { + responseHandler.handle((ResponseMessage) request); + } else { + ctx.fireChannelRead(request); + } + } + + /** Triggered based on events from an {@link io.netty.handler.timeout.IdleStateHandler}. */ + @Override + public void userEventTriggered(ChannelHandlerContext ctx, Object evt) throws Exception { + if (evt instanceof IdleStateEvent) { + IdleStateEvent e = (IdleStateEvent) evt; + // See class comment for timeout semantics. In addition to ensuring we only timeout while + // there are outstanding requests, we also do a secondary consistency check to ensure + // there's no race between the idle timeout and incrementing the numOutstandingRequests + // (see SPARK-7003). + // + // To avoid a race between TransportClientFactory.createClient() and this code which could + // result in an inactive client being returned, this needs to run in a synchronized block. + synchronized (this) { + boolean isActuallyOverdue = + System.nanoTime() - responseHandler.getTimeOfLastRequestNs() > requestTimeoutNs; + if (e.state() == IdleState.ALL_IDLE && isActuallyOverdue) { + if (responseHandler.numOutstandingRequests() > 0) { + String address = getRemoteAddress(ctx.channel()); + logger.error("Connection to {} has been quiet for {} ms while there are outstanding " + + "requests. Assuming connection is dead; please adjust css.network.timeout if " + + "this is wrong.", address, requestTimeoutNs / 1000 / 1000); + client.timeOut(); + ctx.close(); + } else if (closeIdleConnections) { + // While CloseIdleConnections is enable, we also close idle connection + client.timeOut(); + ctx.close(); + } + } + } + } + ctx.fireUserEventTriggered(evt); + } + + public TransportResponseHandler getResponseHandler() { + return responseHandler; + } + + @Override + public void channelRegistered(ChannelHandlerContext ctx) throws Exception { + transportContext.getRegisteredConnections().inc(); + super.channelRegistered(ctx); + } + + @Override + public void channelUnregistered(ChannelHandlerContext ctx) throws Exception { + transportContext.getRegisteredConnections().dec(); + super.channelUnregistered(ctx); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/server/TransportRequestHandler.java b/network-common/src/main/java/com/bytedance/css/network/server/TransportRequestHandler.java new file mode 100644 index 0000000..6135635 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/server/TransportRequestHandler.java @@ -0,0 +1,300 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.server; + +import java.io.IOException; +import java.net.SocketAddress; +import java.nio.ByteBuffer; + +import com.google.common.base.Throwables; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.buffer.NioManagedBuffer; +import com.bytedance.css.network.client.*; +import com.bytedance.css.network.protocol.*; +import com.bytedance.css.network.util.TransportFrameDecoder; + +import static com.bytedance.css.network.util.NettyUtils.getRemoteAddress; + +/** + * A handler that processes requests from clients and writes chunk data back. Each handler is + * attached to a single Netty channel, and keeps track of which streams have been fetched via this + * channel, in order to clean them up if the channel is terminated (see #channelUnregistered). + * + * The messages should have been processed by the pipeline setup by {@link TransportServer}. + */ +public class TransportRequestHandler extends MessageHandler { + + private static final Logger logger = LoggerFactory.getLogger(TransportRequestHandler.class); + + /** The Netty channel that this handler is associated with. */ + private final Channel channel; + + /** Client on the same channel allowing us to talk back to the requester. */ + private final TransportClient reverseClient; + + /** Handles all RPC messages. */ + private final RpcHandler rpcHandler; + + /** Returns each chunk part of a stream. */ + private final StreamManager streamManager; + + /** The max number of chunks being transferred and not finished yet. */ + private final long maxChunksBeingTransferred; + + /** The dedicated ChannelHandler for ChunkFetchRequest messages. */ + private final ChunkFetchRequestHandler chunkFetchRequestHandler; + + public TransportRequestHandler( + Channel channel, + TransportClient reverseClient, + RpcHandler rpcHandler, + Long maxChunksBeingTransferred, + ChunkFetchRequestHandler chunkFetchRequestHandler) { + this.channel = channel; + this.reverseClient = reverseClient; + this.rpcHandler = rpcHandler; + this.streamManager = rpcHandler.getStreamManager(); + this.maxChunksBeingTransferred = maxChunksBeingTransferred; + this.chunkFetchRequestHandler = chunkFetchRequestHandler; + } + + @Override + public void exceptionCaught(Throwable cause) { + rpcHandler.exceptionCaught(cause, reverseClient); + } + + @Override + public void channelActive() { + rpcHandler.channelActive(reverseClient); + } + + @Override + public void channelInactive() { + if (streamManager != null) { + try { + streamManager.connectionTerminated(channel); + } catch (RuntimeException e) { + logger.error("StreamManager connectionTerminated() callback failed.", e); + } + } + rpcHandler.channelInactive(reverseClient); + } + + @Override + public void handle(RequestMessage request) throws Exception { + if (request instanceof ChunkFetchRequest) { + chunkFetchRequestHandler.processFetchRequest(channel, (ChunkFetchRequest) request); + } else if (request instanceof RpcRequest) { + processRpcRequest((RpcRequest) request); + } else if (request instanceof OneWayMessage) { + processOneWayMessage((OneWayMessage) request); + } else if (request instanceof StreamRequest) { + processStreamRequest((StreamRequest) request); + } else if (request instanceof UploadStream) { + processStreamUpload((UploadStream) request); + } else if (request instanceof BatchPushDataRequest) { + processBatchPushDataRequest((BatchPushDataRequest) request); + } else { + throw new IllegalArgumentException("Unknown request type: " + request); + } + } + + private void processStreamRequest(final StreamRequest req) { + if (logger.isTraceEnabled()) { + logger.trace("Received req from {} to fetch stream {}", getRemoteAddress(channel), + req.streamId); + } + + long chunksBeingTransferred = streamManager.chunksBeingTransferred(); + if (chunksBeingTransferred >= maxChunksBeingTransferred) { + logger.warn("The number of chunks being transferred {} is above {}, close the connection.", + chunksBeingTransferred, maxChunksBeingTransferred); + channel.close(); + return; + } + ManagedBuffer buf; + try { + buf = streamManager.openStream(req.streamId); + } catch (Exception e) { + logger.error(String.format( + "Error opening stream %s for request from %s", req.streamId, getRemoteAddress(channel)), e); + respond(new StreamFailure(req.streamId, Throwables.getStackTraceAsString(e))); + return; + } + + if (buf != null) { + streamManager.streamBeingSent(req.streamId); + respond(new StreamResponse(req.streamId, buf.size(), buf)).addListener(future -> { + streamManager.streamSent(req.streamId); + }); + } else { + respond(new StreamFailure(req.streamId, String.format( + "Stream '%s' was not found.", req.streamId))); + } + } + + private void processRpcRequest(final RpcRequest req) { + try { + rpcHandler.receive(reverseClient, req.body().nioByteBuffer(), new RpcResponseCallback() { + @Override + public void onSuccess(ByteBuffer response) { + respond(new RpcResponse(req.requestId, new NioManagedBuffer(response))); + } + + @Override + public void onFailure(Throwable e) { + respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); + } + }); + } catch (Exception e) { + logger.error("Error while invoking RpcHandler#receive() on RPC id " + req.requestId, e); + respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); + } finally { + req.body().release(); + } + } + + /** + * Handle a request from the client to upload a stream of data. + */ + private void processStreamUpload(final UploadStream req) { + assert (req.body() == null); + try { + RpcResponseCallback callback = new RpcResponseCallback() { + @Override + public void onSuccess(ByteBuffer response) { + respond(new RpcResponse(req.requestId, new NioManagedBuffer(response))); + } + + @Override + public void onFailure(Throwable e) { + respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); + } + }; + TransportFrameDecoder frameDecoder = (TransportFrameDecoder) + channel.pipeline().get(TransportFrameDecoder.HANDLER_NAME); + ByteBuffer meta = req.meta.nioByteBuffer(); + StreamCallbackWithID streamHandler = rpcHandler.receiveStream(reverseClient, meta, callback); + if (streamHandler == null) { + throw new NullPointerException("rpcHandler returned a null streamHandler"); + } + StreamCallbackWithID wrappedCallback = new StreamCallbackWithID() { + @Override + public void onData(String streamId, ByteBuffer buf) throws IOException { + streamHandler.onData(streamId, buf); + } + + @Override + public void onComplete(String streamId) throws IOException { + try { + streamHandler.onComplete(streamId); + callback.onSuccess(ByteBuffer.allocate(0)); + } catch (Exception ex) { + IOException ioExc = new IOException("Failure post-processing complete stream;" + + " failing this rpc and leaving channel active", ex); + callback.onFailure(ioExc); + streamHandler.onFailure(streamId, ioExc); + } + } + + @Override + public void onFailure(String streamId, Throwable cause) throws IOException { + callback.onFailure(new IOException("Destination failed while reading stream", cause)); + streamHandler.onFailure(streamId, cause); + } + + @Override + public String getID() { + return streamHandler.getID(); + } + }; + if (req.bodyByteCount > 0) { + StreamInterceptor interceptor = new StreamInterceptor<>( + this, wrappedCallback.getID(), req.bodyByteCount, wrappedCallback); + frameDecoder.setInterceptor(interceptor); + } else { + wrappedCallback.onComplete(wrappedCallback.getID()); + } + } catch (Exception e) { + logger.error("Error while invoking RpcHandler#receive() on RPC id " + req.requestId, e); + respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); + // We choose to totally fail the channel, rather than trying to recover as we do in other + // cases. We don't know how many bytes of the stream the client has already sent for the + // stream, it's not worth trying to recover. + channel.pipeline().fireExceptionCaught(e); + } finally { + req.meta.release(); + } + } + + private void processOneWayMessage(OneWayMessage req) { + try { + rpcHandler.receive(reverseClient, req.body().nioByteBuffer()); + } catch (Exception e) { + logger.error("Error while invoking RpcHandler#receive() for one-way message.", e); + } finally { + req.body().release(); + } + } + + private void processBatchPushDataRequest(BatchPushDataRequest req) { + try { + rpcHandler.receiveBatchPushDataRequest(reverseClient, req, new RpcResponseCallback() { + @Override + public void onSuccess(ByteBuffer response) { + respond(new RpcResponse(req.requestId, new NioManagedBuffer(response))); + } + + @Override + public void onFailure(Throwable e) { + logger.error("Process batchPushData onFailure!", e); + respond(new RpcFailure(req.requestId, e.toString())); + } + }); + } catch (Exception e) { + logger.error("Error while invoking RpcHandler#receive() on PushDataRequest " + req, e); + channel.writeAndFlush(e); + } finally { + req.body().release(); + } + } + + /** + * Responds to a single message with some Encodable object. If a failure occurs while sending, + * it will be logged and the channel closed. + */ + private ChannelFuture respond(Encodable result) { + SocketAddress remoteAddress = channel.remoteAddress(); + return channel.writeAndFlush(result).addListener(future -> { + if (future.isSuccess()) { + logger.trace("Sent result {} to client {}", result, remoteAddress); + } else { + logger.error(String.format("Error sending result %s to %s; closing connection", + result, remoteAddress), future.cause()); + channel.close(); + } + }); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/server/TransportServer.java b/network-common/src/main/java/com/bytedance/css/network/server/TransportServer.java new file mode 100644 index 0000000..928228f --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/server/TransportServer.java @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.server; + +import java.io.Closeable; +import java.net.InetSocketAddress; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import com.codahale.metrics.MetricSet; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import io.netty.bootstrap.ServerBootstrap; +import io.netty.buffer.NettyMemoryMetrics; +import io.netty.buffer.PooledByteBufAllocator; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelOption; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.socket.SocketChannel; +import org.apache.commons.lang3.SystemUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.bytedance.css.network.TransportContext; +import com.bytedance.css.network.util.*; + +/** + * Server for the efficient, low-level streaming service. + */ +public class TransportServer implements Closeable { + private static final Logger logger = LoggerFactory.getLogger(TransportServer.class); + + private final TransportContext context; + private final TransportConf conf; + private final RpcHandler appRpcHandler; + private final List bootstraps; + + private ServerBootstrap bootstrap; + private ChannelFuture channelFuture; + private int port = -1; + private NettyMemoryMetrics metrics; + private NettyMemoryMetrics cssMetrics; + private PooledByteBufAllocator allocator; + + /** + * Creates a TransportServer that binds to the given host and the given port, or to any available + * if 0. If you don't want to bind to any special host, set "hostToBind" to null. + * */ + public TransportServer( + TransportContext context, + String hostToBind, + int portToBind, + RpcHandler appRpcHandler, + List bootstraps) { + this.context = context; + this.conf = context.getConf(); + this.appRpcHandler = appRpcHandler; + if (conf.sharedByteBufAllocators()) { + this.allocator = NettyUtils.getSharedPooledByteBufAllocator( + conf.preferDirectBufsForSharedByteBufAllocators(), true /* allowCache */); + } else { + this.allocator = NettyUtils.createPooledByteBufAllocator( + conf.preferDirectBufs(), true /* allowCache */, conf.serverThreads()); + } + this.bootstraps = Lists.newArrayList(Preconditions.checkNotNull(bootstraps)); + + boolean shouldClose = true; + try { + init(hostToBind, portToBind); + shouldClose = false; + } finally { + if (shouldClose) { + JavaUtils.closeQuietly(this); + } + } + } + + public void registerCssMetrics(String cssMetricsPrefix) { + synchronized (this) { + if (cssMetrics == null) { + String cssNettyPrefix = String.format("%s#%s", cssMetricsPrefix, conf.getModuleName() + "-server"); + cssMetrics = new NettyMemoryMetrics(allocator, cssNettyPrefix, conf); + } + } + } + + public int getPort() { + if (port == -1) { + throw new IllegalStateException("Server not initialized"); + } + return port; + } + + private void init(String hostToBind, int portToBind) { + + IOMode ioMode = IOMode.valueOf(conf.ioMode()); + EventLoopGroup bossGroup = NettyUtils.createEventLoop(ioMode, 1, + conf.getModuleName() + "-boss"); + EventLoopGroup workerGroup = NettyUtils.createEventLoop(ioMode, conf.serverThreads(), + conf.getModuleName() + "-server"); + + allocator = NettyUtils.createPooledByteBufAllocator( + conf.preferDirectBufs(), true /* allowCache */, conf.serverThreads()); + + bootstrap = new ServerBootstrap() + .group(bossGroup, workerGroup) + .channel(NettyUtils.getServerChannelClass(ioMode)) + .option(ChannelOption.ALLOCATOR, allocator) + .option(ChannelOption.SO_REUSEADDR, !SystemUtils.IS_OS_WINDOWS) + .childOption(ChannelOption.ALLOCATOR, allocator); + + this.metrics = new NettyMemoryMetrics( + allocator, conf.getModuleName() + "-server", conf); + + if (conf.backLog() > 0) { + bootstrap.option(ChannelOption.SO_BACKLOG, conf.backLog()); + } + + if (conf.receiveBuf() > 0) { + bootstrap.childOption(ChannelOption.SO_RCVBUF, conf.receiveBuf()); + } + + if (conf.sendBuf() > 0) { + bootstrap.childOption(ChannelOption.SO_SNDBUF, conf.sendBuf()); + } + + if (conf.enableTcpKeepAlive()) { + bootstrap.childOption(ChannelOption.SO_KEEPALIVE, true); + } + + bootstrap.childHandler(new ChannelInitializer() { + @Override + protected void initChannel(SocketChannel ch) { + logger.debug("New connection accepted for remote address {}.", ch.remoteAddress()); + + RpcHandler rpcHandler = appRpcHandler; + for (TransportServerBootstrap bootstrap : bootstraps) { + rpcHandler = bootstrap.doBootstrap(ch, rpcHandler); + } + context.initializePipeline(ch, rpcHandler); + } + }); + + InetSocketAddress address = hostToBind == null ? + new InetSocketAddress(portToBind): new InetSocketAddress(hostToBind, portToBind); + channelFuture = bootstrap.bind(address); + channelFuture.syncUninterruptibly(); + + port = ((InetSocketAddress) channelFuture.channel().localAddress()).getPort(); + logger.debug("Shuffle server started on port: {}", port); + } + + public MetricSet getAllMetrics() { + return metrics; + } + + public MetricSet getAllCssMetrics() { + return cssMetrics; + } + + @Override + public void close() { + if (channelFuture != null) { + // close is a local operation and should finish within milliseconds; timeout just to be safe + channelFuture.channel().close().awaitUninterruptibly(10, TimeUnit.SECONDS); + channelFuture = null; + } + if (bootstrap != null && bootstrap.config().group() != null) { + bootstrap.config().group().shutdownGracefully(); + } + if (bootstrap != null && bootstrap.config().childGroup() != null) { + bootstrap.config().childGroup().shutdownGracefully(); + } + bootstrap = null; + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/server/TransportServerBootstrap.java b/network-common/src/main/java/com/bytedance/css/network/server/TransportServerBootstrap.java new file mode 100644 index 0000000..b556891 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/server/TransportServerBootstrap.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.server; + +import io.netty.channel.Channel; + +/** + * A bootstrap which is executed on a TransportServer's client channel once a client connects + * to the server. This allows customizing the client channel to allow for things such as SASL + * authentication. + */ +public interface TransportServerBootstrap { + /** + * Customizes the channel to include new features, if needed. + * + * @param channel The connected channel opened by the client. + * @param rpcHandler The RPC handler for the server. + * @return The RPC handler to use for the channel. + */ + RpcHandler doBootstrap(Channel channel, RpcHandler rpcHandler); +} diff --git a/network-common/src/main/java/com/bytedance/css/network/util/AbstractFileRegion.java b/network-common/src/main/java/com/bytedance/css/network/util/AbstractFileRegion.java new file mode 100644 index 0000000..64071bf --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/util/AbstractFileRegion.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.util; + +import io.netty.channel.FileRegion; +import io.netty.util.AbstractReferenceCounted; + +public abstract class AbstractFileRegion extends AbstractReferenceCounted implements FileRegion { + + @Override + @SuppressWarnings("deprecation") + public final long transfered() { + return transferred(); + } + + @Override + public AbstractFileRegion retain() { + super.retain(); + return this; + } + + @Override + public AbstractFileRegion retain(int increment) { + super.retain(increment); + return this; + } + + @Override + public AbstractFileRegion touch() { + super.touch(); + return this; + } + + @Override + public AbstractFileRegion touch(Object o) { + return this; + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/util/ByteArrayReadableChannel.java b/network-common/src/main/java/com/bytedance/css/network/util/ByteArrayReadableChannel.java new file mode 100644 index 0000000..6bc7ecb --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/util/ByteArrayReadableChannel.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.util; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ClosedChannelException; +import java.nio.channels.ReadableByteChannel; + +import io.netty.buffer.ByteBuf; + +public class ByteArrayReadableChannel implements ReadableByteChannel { + private ByteBuf data; + private boolean closed; + + public void feedData(ByteBuf buf) throws ClosedChannelException { + if (closed) { + throw new ClosedChannelException(); + } + data = buf; + } + + @Override + public int read(ByteBuffer dst) throws IOException { + if (closed) { + throw new ClosedChannelException(); + } + int totalRead = 0; + while (data.readableBytes() > 0 && dst.remaining() > 0) { + int bytesToRead = Math.min(data.readableBytes(), dst.remaining()); + dst.put(data.readSlice(bytesToRead).nioBuffer()); + totalRead += bytesToRead; + } + + return totalRead; + } + + @Override + public void close() { + closed = true; + } + + @Override + public boolean isOpen() { + return !closed; + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/util/ByteArrayWritableChannel.java b/network-common/src/main/java/com/bytedance/css/network/util/ByteArrayWritableChannel.java new file mode 100644 index 0000000..20171d9 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/util/ByteArrayWritableChannel.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.util; + +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +/** + * A writable channel that stores the written data in a byte array in memory. + */ +public class ByteArrayWritableChannel implements WritableByteChannel { + + private final byte[] data; + private int offset; + + public ByteArrayWritableChannel(int size) { + this.data = new byte[size]; + } + + public byte[] getData() { + return data; + } + + public int length() { + return offset; + } + + /** Resets the channel so that writing to it will overwrite the existing buffer. */ + public void reset() { + offset = 0; + } + + /** + * Reads from the given buffer into the internal byte array. + */ + @Override + public int write(ByteBuffer src) { + int toTransfer = Math.min(src.remaining(), data.length - offset); + src.get(data, offset, toTransfer); + offset += toTransfer; + return toTransfer; + } + + @Override + public void close() { + + } + + @Override + public boolean isOpen() { + return true; + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/util/ByteUnit.java b/network-common/src/main/java/com/bytedance/css/network/util/ByteUnit.java new file mode 100644 index 0000000..106fc7f --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/util/ByteUnit.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.util; + +public enum ByteUnit { + BYTE(1), + KiB(1024L), + MiB((long) Math.pow(1024L, 2L)), + GiB((long) Math.pow(1024L, 3L)), + TiB((long) Math.pow(1024L, 4L)), + PiB((long) Math.pow(1024L, 5L)); + + ByteUnit(long multiplier) { + this.multiplier = multiplier; + } + + // Interpret the provided number (d) with suffix (u) as this unit type. + // E.g. KiB.interpret(1, MiB) interprets 1MiB as its KiB representation = 1024k + public long convertFrom(long d, ByteUnit u) { + return u.convertTo(d, this); + } + + // Convert the provided number (d) interpreted as this unit type to unit type (u). + public long convertTo(long d, ByteUnit u) { + if (multiplier > u.multiplier) { + long ratio = multiplier / u.multiplier; + if (Long.MAX_VALUE / ratio < d) { + throw new IllegalArgumentException("Conversion of " + d + " exceeds Long.MAX_VALUE in " + + name() + ". Try a larger unit (e.g. MiB instead of KiB)"); + } + return d * ratio; + } else { + // Perform operations in this order to avoid potential overflow + // when computing d * multiplier + return d / (u.multiplier / multiplier); + } + } + + public long toBytes(long d) { + if (d < 0) { + throw new IllegalArgumentException("Negative size value. Size must be positive: " + d); + } + return d * multiplier; + } + + public long toKiB(long d) { return convertTo(d, KiB); } + public long toMiB(long d) { return convertTo(d, MiB); } + public long toGiB(long d) { return convertTo(d, GiB); } + public long toTiB(long d) { return convertTo(d, TiB); } + public long toPiB(long d) { return convertTo(d, PiB); } + + private final long multiplier; +} diff --git a/network-common/src/main/java/com/bytedance/css/network/util/ConfigProvider.java b/network-common/src/main/java/com/bytedance/css/network/util/ConfigProvider.java new file mode 100644 index 0000000..2bf5554 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/util/ConfigProvider.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.util; + +import java.util.Map; +import java.util.NoSuchElementException; + +/** + * Provides a mechanism for constructing a {@link TransportConf} using some sort of configuration. + */ +public abstract class ConfigProvider { + /** Obtains the value of the given config, throws NoSuchElementException if it doesn't exist. */ + public abstract String get(String name); + + /** Returns all the config values in the provider. */ + public abstract Iterable> getAll(); + + public String get(String name, String defaultValue) { + try { + return get(name); + } catch (NoSuchElementException e) { + return defaultValue; + } + } + + public int getInt(String name, int defaultValue) { + return Integer.parseInt(get(name, Integer.toString(defaultValue))); + } + + public long getLong(String name, long defaultValue) { + return Long.parseLong(get(name, Long.toString(defaultValue))); + } + + public double getDouble(String name, double defaultValue) { + return Double.parseDouble(get(name, Double.toString(defaultValue))); + } + + public boolean getBoolean(String name, boolean defaultValue) { + return Boolean.parseBoolean(get(name, Boolean.toString(defaultValue))); + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/util/CryptoUtils.java b/network-common/src/main/java/com/bytedance/css/network/util/CryptoUtils.java new file mode 100644 index 0000000..c3687d6 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/util/CryptoUtils.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.util; + +import java.util.Map; +import java.util.Properties; + +/** + * Utility methods related to the commons-crypto library. + */ +public class CryptoUtils { + + // The prefix for the configurations passing to Apache Commons Crypto library. + public static final String COMMONS_CRYPTO_CONFIG_PREFIX = "commons.crypto."; + + /** + * Extract the commons-crypto configuration embedded in a list of config values. + * + * @param prefix Prefix in the given configuration that identifies the commons-crypto configs. + * @param conf List of configuration values. + */ + public static Properties toCryptoConf(String prefix, Iterable> conf) { + Properties props = new Properties(); + for (Map.Entry e : conf) { + String key = e.getKey(); + if (key.startsWith(prefix)) { + props.setProperty(COMMONS_CRYPTO_CONFIG_PREFIX + key.substring(prefix.length()), + e.getValue()); + } + } + return props; + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/util/IOMode.java b/network-common/src/main/java/com/bytedance/css/network/util/IOMode.java new file mode 100644 index 0000000..f6ff8b7 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/util/IOMode.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.util; + +/** + * Selector for which form of low-level IO we should use. + * NIO is always available, while EPOLL is only available on Linux. + * AUTO is used to select EPOLL if it's available, or NIO otherwise. + */ +public enum IOMode { + NIO, EPOLL +} diff --git a/network-common/src/main/java/com/bytedance/css/network/util/JavaUtils.java b/network-common/src/main/java/com/bytedance/css/network/util/JavaUtils.java new file mode 100644 index 0000000..972fab0 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/util/JavaUtils.java @@ -0,0 +1,380 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.util; + +import java.io.*; +import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; +import java.nio.charset.StandardCharsets; +import java.util.Locale; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; +import io.netty.buffer.Unpooled; +import org.apache.commons.lang3.SystemUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * General utilities available in the network package. Many of these are sourced from Spark's + * own Utils, just accessible within this package. + */ +public class JavaUtils { + private static final Logger logger = LoggerFactory.getLogger(JavaUtils.class); + + /** + * Define a default value for driver memory here since this value is referenced across the code + * base and nearly all files already use Utils.scala + */ + public static final long DEFAULT_DRIVER_MEM_MB = 1024; + + /** Closes the given object, ignoring IOExceptions. */ + public static void closeQuietly(Closeable closeable) { + try { + if (closeable != null) { + closeable.close(); + } + } catch (IOException e) { + logger.error("IOException should not have been thrown.", e); + } + } + + /** Returns a hash consistent with Spark's Utils.nonNegativeHash(). */ + public static int nonNegativeHash(Object obj) { + if (obj == null) { return 0; } + int hash = obj.hashCode(); + return hash != Integer.MIN_VALUE ? Math.abs(hash) : 0; + } + + /** + * Convert the given string to a byte buffer. The resulting buffer can be + * converted back to the same string through {@link #bytesToString(ByteBuffer)}. + */ + public static ByteBuffer stringToBytes(String s) { + return Unpooled.wrappedBuffer(s.getBytes(StandardCharsets.UTF_8)).nioBuffer(); + } + + /** + * Convert the given byte buffer to a string. The resulting string can be + * converted back to the same byte buffer through {@link #stringToBytes(String)}. + */ + public static String bytesToString(ByteBuffer b) { + return Unpooled.wrappedBuffer(b).toString(StandardCharsets.UTF_8); + } + + /** + * Delete a file or directory and its contents recursively. + * Don't follow directories if they are symlinks. + * + * @param file Input file / dir to be deleted + * @throws IOException if deletion is unsuccessful + */ + public static void deleteRecursively(File file) throws IOException { + deleteRecursively(file, null); + } + + /** + * Delete a file or directory and its contents recursively. + * Don't follow directories if they are symlinks. + * + * @param file Input file / dir to be deleted + * @param filter A filename filter that make sure only files / dirs with the satisfied filenames + * are deleted. + * @throws IOException if deletion is unsuccessful + */ + public static void deleteRecursively(File file, FilenameFilter filter) throws IOException { + if (file == null) { return; } + + // On Unix systems, use operating system command to run faster + // If that does not work out, fallback to the Java IO way + if (SystemUtils.IS_OS_UNIX && filter == null) { + try { + deleteRecursivelyUsingUnixNative(file); + return; + } catch (IOException e) { + logger.warn("Attempt to delete using native Unix OS command failed for path = {}. " + + "Falling back to Java IO way", file.getAbsolutePath(), e); + } + } + + deleteRecursivelyUsingJavaIO(file, filter); + } + + private static void deleteRecursivelyUsingJavaIO( + File file, + FilenameFilter filter) throws IOException { + if (file.isDirectory() && !isSymlink(file)) { + IOException savedIOException = null; + for (File child : listFilesSafely(file, filter)) { + try { + deleteRecursively(child, filter); + } catch (IOException e) { + // In case of multiple exceptions, only last one will be thrown + savedIOException = e; + } + } + if (savedIOException != null) { + throw savedIOException; + } + } + + // Delete file only when it's a normal file or an empty directory. + if (file.isFile() || (file.isDirectory() && listFilesSafely(file, null).length == 0)) { + boolean deleted = file.delete(); + // Delete can also fail if the file simply did not exist. + if (!deleted && file.exists()) { + throw new IOException("Failed to delete: " + file.getAbsolutePath()); + } + } + } + + private static void deleteRecursivelyUsingUnixNative(File file) throws IOException { + ProcessBuilder builder = new ProcessBuilder("rm", "-rf", file.getAbsolutePath()); + Process process = null; + int exitCode = -1; + + try { + // In order to avoid deadlocks, consume the stdout (and stderr) of the process + builder.redirectErrorStream(true); + builder.redirectOutput(new File("/dev/null")); + + process = builder.start(); + + exitCode = process.waitFor(); + } catch (Exception e) { + throw new IOException("Failed to delete: " + file.getAbsolutePath(), e); + } finally { + if (process != null) { + process.destroy(); + } + } + + if (exitCode != 0 || file.exists()) { + throw new IOException("Failed to delete: " + file.getAbsolutePath()); + } + } + + private static File[] listFilesSafely(File file, FilenameFilter filter) throws IOException { + if (file.exists()) { + File[] files = file.listFiles(filter); + if (files == null) { + throw new IOException("Failed to list files for dir: " + file); + } + return files; + } else { + return new File[0]; + } + } + + private static boolean isSymlink(File file) throws IOException { + Preconditions.checkNotNull(file); + File fileInCanonicalDir = null; + if (file.getParent() == null) { + fileInCanonicalDir = file; + } else { + fileInCanonicalDir = new File(file.getParentFile().getCanonicalFile(), file.getName()); + } + return !fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile()); + } + + private static final ImmutableMap timeSuffixes = + ImmutableMap.builder() + .put("us", TimeUnit.MICROSECONDS) + .put("ms", TimeUnit.MILLISECONDS) + .put("s", TimeUnit.SECONDS) + .put("m", TimeUnit.MINUTES) + .put("min", TimeUnit.MINUTES) + .put("h", TimeUnit.HOURS) + .put("d", TimeUnit.DAYS) + .build(); + + private static final ImmutableMap byteSuffixes = + ImmutableMap.builder() + .put("b", ByteUnit.BYTE) + .put("k", ByteUnit.KiB) + .put("kb", ByteUnit.KiB) + .put("m", ByteUnit.MiB) + .put("mb", ByteUnit.MiB) + .put("g", ByteUnit.GiB) + .put("gb", ByteUnit.GiB) + .put("t", ByteUnit.TiB) + .put("tb", ByteUnit.TiB) + .put("p", ByteUnit.PiB) + .put("pb", ByteUnit.PiB) + .build(); + + /** + * Convert a passed time string (e.g. 50s, 100ms, or 250us) to a time count in the given unit. + * The unit is also considered the default if the given string does not specify a unit. + */ + public static long timeStringAs(String str, TimeUnit unit) { + String lower = str.toLowerCase(Locale.ROOT).trim(); + + try { + Matcher m = Pattern.compile("(-?[0-9]+)([a-z]+)?").matcher(lower); + if (!m.matches()) { + throw new NumberFormatException("Failed to parse time string: " + str); + } + + long val = Long.parseLong(m.group(1)); + String suffix = m.group(2); + + // Check for invalid suffixes + if (suffix != null && !timeSuffixes.containsKey(suffix)) { + throw new NumberFormatException("Invalid suffix: \"" + suffix + "\""); + } + + // If suffix is valid use that, otherwise none was provided and use the default passed + return unit.convert(val, suffix != null ? timeSuffixes.get(suffix) : unit); + } catch (NumberFormatException e) { + String timeError = "Time must be specified as seconds (s), " + + "milliseconds (ms), microseconds (us), minutes (m or min), hour (h), or day (d). " + + "E.g. 50s, 100ms, or 250us."; + + throw new NumberFormatException(timeError + "\n" + e.getMessage()); + } + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to milliseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in ms. + */ + public static long timeStringAsMs(String str) { + return timeStringAs(str, TimeUnit.MILLISECONDS); + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to seconds for internal use. If + * no suffix is provided, the passed number is assumed to be in seconds. + */ + public static long timeStringAsSec(String str) { + return timeStringAs(str, TimeUnit.SECONDS); + } + + /** + * Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to the given. If no suffix is + * provided, a direct conversion to the provided unit is attempted. + */ + public static long byteStringAs(String str, ByteUnit unit) { + String lower = str.toLowerCase(Locale.ROOT).trim(); + + try { + Matcher m = Pattern.compile("([0-9]+)([a-z]+)?").matcher(lower); + Matcher fractionMatcher = Pattern.compile("([0-9]+\\.[0-9]+)([a-z]+)?").matcher(lower); + + if (m.matches()) { + long val = Long.parseLong(m.group(1)); + String suffix = m.group(2); + + // Check for invalid suffixes + if (suffix != null && !byteSuffixes.containsKey(suffix)) { + throw new NumberFormatException("Invalid suffix: \"" + suffix + "\""); + } + + // If suffix is valid use that, otherwise none was provided and use the default passed + return unit.convertFrom(val, suffix != null ? byteSuffixes.get(suffix) : unit); + } else if (fractionMatcher.matches()) { + throw new NumberFormatException("Fractional values are not supported. Input was: " + + fractionMatcher.group(1)); + } else { + throw new NumberFormatException("Failed to parse byte string: " + str); + } + + } catch (NumberFormatException e) { + String byteError = "Size must be specified as bytes (b), " + + "kibibytes (k), mebibytes (m), gibibytes (g), tebibytes (t), or pebibytes(p). " + + "E.g. 50b, 100k, or 250m."; + + throw new NumberFormatException(byteError + "\n" + e.getMessage()); + } + } + + /** + * Convert a passed byte string (e.g. 50b, 100k, or 250m) to bytes for + * internal use. + * + * If no suffix is provided, the passed number is assumed to be in bytes. + */ + public static long byteStringAsBytes(String str) { + return byteStringAs(str, ByteUnit.BYTE); + } + + /** + * Convert a passed byte string (e.g. 50b, 100k, or 250m) to kibibytes for + * internal use. + * + * If no suffix is provided, the passed number is assumed to be in kibibytes. + */ + public static long byteStringAsKb(String str) { + return byteStringAs(str, ByteUnit.KiB); + } + + /** + * Convert a passed byte string (e.g. 50b, 100k, or 250m) to mebibytes for + * internal use. + * + * If no suffix is provided, the passed number is assumed to be in mebibytes. + */ + public static long byteStringAsMb(String str) { + return byteStringAs(str, ByteUnit.MiB); + } + + /** + * Convert a passed byte string (e.g. 50b, 100k, or 250m) to gibibytes for + * internal use. + * + * If no suffix is provided, the passed number is assumed to be in gibibytes. + */ + public static long byteStringAsGb(String str) { + return byteStringAs(str, ByteUnit.GiB); + } + + /** + * Returns a byte array with the buffer's contents, trying to avoid copying the data if + * possible. + */ + public static byte[] bufferToArray(ByteBuffer buffer) { + if (buffer.hasArray() && buffer.arrayOffset() == 0 && + buffer.array().length == buffer.remaining()) { + return buffer.array(); + } else { + byte[] bytes = new byte[buffer.remaining()]; + buffer.get(bytes); + return bytes; + } + } + + /** + * Fills a buffer with data read from the channel. + */ + public static void readFully(ReadableByteChannel channel, ByteBuffer dst) throws IOException { + int expected = dst.remaining(); + while (dst.hasRemaining()) { + if (channel.read(dst) < 0) { + throw new EOFException(String.format("Not enough bytes in channel (expected %d).", + expected)); + } + } + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/util/LevelDBProvider.java b/network-common/src/main/java/com/bytedance/css/network/util/LevelDBProvider.java new file mode 100644 index 0000000..0df4f1b --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/util/LevelDBProvider.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.util; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.fusesource.leveldbjni.JniDBFactory; +import org.fusesource.leveldbjni.internal.NativeDB; +import org.iq80.leveldb.DB; +import org.iq80.leveldb.Options; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * LevelDB utility class available in the network package. + */ +public class LevelDBProvider { + private static final Logger logger = LoggerFactory.getLogger(LevelDBProvider.class); + + public static DB initLevelDB(File dbFile, StoreVersion version, ObjectMapper mapper) throws + IOException { + DB tmpDb = null; + if (dbFile != null) { + Options options = new Options(); + options.createIfMissing(false); + options.logger(new LevelDBLogger()); + try { + tmpDb = JniDBFactory.factory.open(dbFile, options); + } catch (NativeDB.DBException e) { + if (e.isNotFound() || e.getMessage().contains(" does not exist ")) { + logger.info("Creating state database at " + dbFile); + options.createIfMissing(true); + try { + tmpDb = JniDBFactory.factory.open(dbFile, options); + } catch (NativeDB.DBException dbExc) { + throw new IOException("Unable to create state store", dbExc); + } + } else { + // the leveldb file seems to be corrupt somehow. Lets just blow it away and create a new + // one, so we can keep processing new apps + logger.error("error opening leveldb file {}. Creating new file, will not be able to " + + "recover state for existing applications", dbFile, e); + if (dbFile.isDirectory()) { + for (File f : dbFile.listFiles()) { + if (!f.delete()) { + logger.warn("error deleting {}", f.getPath()); + } + } + } + if (!dbFile.delete()) { + logger.warn("error deleting {}", dbFile.getPath()); + } + options.createIfMissing(true); + try { + tmpDb = JniDBFactory.factory.open(dbFile, options); + } catch (NativeDB.DBException dbExc) { + throw new IOException("Unable to create state store", dbExc); + } + + } + } + // if there is a version mismatch, we throw an exception, which means the service is unusable + checkVersion(tmpDb, version, mapper); + } + return tmpDb; + } + + private static class LevelDBLogger implements org.iq80.leveldb.Logger { + private static final Logger LOG = LoggerFactory.getLogger(LevelDBLogger.class); + + @Override + public void log(String message) { + LOG.info(message); + } + } + + /** + * Simple major.minor versioning scheme. Any incompatible changes should be across major + * versions. Minor version differences are allowed -- meaning we should be able to read + * dbs that are either earlier *or* later on the minor version. + */ + public static void checkVersion(DB db, StoreVersion newversion, ObjectMapper mapper) throws + IOException { + byte[] bytes = db.get(StoreVersion.KEY); + if (bytes == null) { + storeVersion(db, newversion, mapper); + } else { + StoreVersion version = mapper.readValue(bytes, StoreVersion.class); + if (version.major != newversion.major) { + throw new IOException("cannot read state DB with version " + version + ", incompatible " + + "with current version " + newversion); + } + storeVersion(db, newversion, mapper); + } + } + + public static void storeVersion(DB db, StoreVersion version, ObjectMapper mapper) + throws IOException { + db.put(StoreVersion.KEY, mapper.writeValueAsBytes(version)); + } + + public static class StoreVersion { + + static final byte[] KEY = "StoreVersion".getBytes(StandardCharsets.UTF_8); + + public final int major; + public final int minor; + + @JsonCreator + public StoreVersion(@JsonProperty("major") int major, @JsonProperty("minor") int minor) { + this.major = major; + this.minor = minor; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + StoreVersion that = (StoreVersion) o; + + return major == that.major && minor == that.minor; + } + + @Override + public int hashCode() { + int result = major; + result = 31 * result + minor; + return result; + } + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/util/LimitedInputStream.java b/network-common/src/main/java/com/bytedance/css/network/util/LimitedInputStream.java new file mode 100644 index 0000000..23977b9 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/util/LimitedInputStream.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * Based on LimitedInputStream.java from Google Guava + * + * Copyright (C) 2007 The Guava Authors + * + * 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. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.util; + +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; + +import com.google.common.base.Preconditions; + +/** + * Wraps a {@link InputStream}, limiting the number of bytes which can be read. + * + * This code is from Guava's 14.0 source code, because there is no compatible way to + * use this functionality in both a Guava 11 environment and a Guava >14 environment. + */ +public final class LimitedInputStream extends FilterInputStream { + private final boolean closeWrappedStream; + private long left; + private long mark = -1; + + public LimitedInputStream(InputStream in, long limit) { + this(in, limit, true); + } + + /** + * Create a LimitedInputStream that will read {@code limit} bytes from {@code in}. + *

    + * If {@code closeWrappedStream} is true, this will close {@code in} when it is closed. + * Otherwise, the stream is left open for reading its remaining content. + * + * @param in a {@link InputStream} to read from + * @param limit the number of bytes to read + * @param closeWrappedStream whether to close {@code in} when {@link #close} is called + */ + public LimitedInputStream(InputStream in, long limit, boolean closeWrappedStream) { + super(in); + this.closeWrappedStream = closeWrappedStream; + Preconditions.checkNotNull(in); + Preconditions.checkArgument(limit >= 0, "limit must be non-negative"); + left = limit; + } + @Override public int available() throws IOException { + return (int) Math.min(in.available(), left); + } + // it's okay to mark even if mark isn't supported, as reset won't work + @Override public synchronized void mark(int readLimit) { + in.mark(readLimit); + mark = left; + } + @Override public int read() throws IOException { + if (left == 0) { + return -1; + } + int result = in.read(); + if (result != -1) { + --left; + } + return result; + } + @Override public int read(byte[] b, int off, int len) throws IOException { + if (left == 0) { + return -1; + } + len = (int) Math.min(len, left); + int result = in.read(b, off, len); + if (result != -1) { + left -= result; + } + return result; + } + @Override public synchronized void reset() throws IOException { + if (!in.markSupported()) { + throw new IOException("Mark not supported"); + } + if (mark == -1) { + throw new IOException("Mark not set"); + } + in.reset(); + left = mark; + } + @Override public long skip(long n) throws IOException { + n = Math.min(n, left); + long skipped = in.skip(n); + left -= skipped; + return skipped; + } + + @Override + public void close() throws IOException { + if (closeWrappedStream) { + super.close(); + } + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/util/MapConfigProvider.java b/network-common/src/main/java/com/bytedance/css/network/util/MapConfigProvider.java new file mode 100644 index 0000000..0198c8f --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/util/MapConfigProvider.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.util; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.NoSuchElementException; + +/** ConfigProvider based on a Map (copied in the constructor). */ +public class MapConfigProvider extends ConfigProvider { + + public static final MapConfigProvider EMPTY = new MapConfigProvider(Collections.emptyMap()); + + private final Map config; + + public MapConfigProvider(Map config) { + this.config = new HashMap<>(config); + } + + @Override + public String get(String name) { + String value = config.get(name); + if (value == null) { + throw new NoSuchElementException(name); + } + return value; + } + + @Override + public String get(String name, String defaultValue) { + String value = config.get(name); + return value == null ? defaultValue : value; + } + + @Override + public Iterable> getAll() { + return config.entrySet(); + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/util/NettyUtils.java b/network-common/src/main/java/com/bytedance/css/network/util/NettyUtils.java new file mode 100644 index 0000000..51f2805 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/util/NettyUtils.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.util; + +import java.util.concurrent.ThreadFactory; + +import io.netty.buffer.PooledByteBufAllocator; +import io.netty.channel.Channel; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.ServerChannel; +import io.netty.channel.epoll.EpollEventLoopGroup; +import io.netty.channel.epoll.EpollServerSocketChannel; +import io.netty.channel.epoll.EpollSocketChannel; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.nio.NioServerSocketChannel; +import io.netty.channel.socket.nio.NioSocketChannel; +import io.netty.util.concurrent.DefaultThreadFactory; +import io.netty.util.internal.PlatformDependent; + +/** + * Utilities for creating various Netty constructs based on whether we're using EPOLL or NIO. + */ +public class NettyUtils { + + /** + * Specifies an upper bound on the number of Netty threads that Spark requires by default. + * In practice, only 2-4 cores should be required to transfer roughly 10 Gb/s, and each core + * that we use will have an initial overhead of roughly 32 MB of off-heap memory, which comes + * at a premium. + * + * Thus, this value should still retain maximum throughput and reduce wasted off-heap memory + * allocation. It can be overridden by setting the number of serverThreads and clientThreads + * manually in Spark's configuration. + */ + private static int MAX_DEFAULT_NETTY_THREADS = 8; + + private static final PooledByteBufAllocator[] _sharedPooledByteBufAllocator = + new PooledByteBufAllocator[2]; + + /** Creates a new ThreadFactory which prefixes each thread with the given name. */ + public static ThreadFactory createThreadFactory(String threadPoolPrefix) { + return new DefaultThreadFactory(threadPoolPrefix, true); + } + + /** Creates a Netty EventLoopGroup based on the IOMode. */ + public static EventLoopGroup createEventLoop(IOMode mode, int numThreads, String threadPrefix) { + ThreadFactory threadFactory = createThreadFactory(threadPrefix); + + switch (mode) { + case NIO: + return new NioEventLoopGroup(numThreads, threadFactory); + case EPOLL: + return new EpollEventLoopGroup(numThreads, threadFactory); + default: + throw new IllegalArgumentException("Unknown io mode: " + mode); + } + } + + /** Returns the correct (client) SocketChannel class based on IOMode. */ + public static Class getClientChannelClass(IOMode mode) { + switch (mode) { + case NIO: + return NioSocketChannel.class; + case EPOLL: + return EpollSocketChannel.class; + default: + throw new IllegalArgumentException("Unknown io mode: " + mode); + } + } + + /** Returns the correct ServerSocketChannel class based on IOMode. */ + public static Class getServerChannelClass(IOMode mode) { + switch (mode) { + case NIO: + return NioServerSocketChannel.class; + case EPOLL: + return EpollServerSocketChannel.class; + default: + throw new IllegalArgumentException("Unknown io mode: " + mode); + } + } + + /** + * Creates a LengthFieldBasedFrameDecoder where the first 8 bytes are the length of the frame. + * This is used before all decoders. + */ + public static TransportFrameDecoder createFrameDecoder() { + return new TransportFrameDecoder(); + } + + /** Returns the remote address on the channel or "<unknown remote>" if none exists. */ + public static String getRemoteAddress(Channel channel) { + if (channel != null && channel.remoteAddress() != null) { + return channel.remoteAddress().toString(); + } + return ""; + } + + /** + * Returns the default number of threads for both the Netty client and server thread pools. + * If numUsableCores is 0, we will use Runtime get an approximate number of available cores. + */ + public static int defaultNumThreads(int numUsableCores) { + final int availableCores; + if (numUsableCores > 0) { + availableCores = numUsableCores; + } else { + availableCores = Runtime.getRuntime().availableProcessors(); + } + return Math.min(availableCores, MAX_DEFAULT_NETTY_THREADS); + } + + /** + * Returns the lazily created shared pooled ByteBuf allocator for the specified allowCache + * parameter value. + */ + public static synchronized PooledByteBufAllocator getSharedPooledByteBufAllocator( + boolean allowDirectBufs, + boolean allowCache) { + final int index = allowCache ? 0 : 1; + if (_sharedPooledByteBufAllocator[index] == null) { + _sharedPooledByteBufAllocator[index] = + createPooledByteBufAllocator( + allowDirectBufs, + allowCache, + defaultNumThreads(0)); + } + return _sharedPooledByteBufAllocator[index]; + } + + /** + * Create a pooled ByteBuf allocator but disables the thread-local cache. Thread-local caches + * are disabled for TransportClients because the ByteBufs are allocated by the event loop thread, + * but released by the executor thread rather than the event loop thread. Those thread-local + * caches actually delay the recycling of buffers, leading to larger memory usage. + */ + public static PooledByteBufAllocator createPooledByteBufAllocator( + boolean allowDirectBufs, + boolean allowCache, + int numCores) { + if (numCores == 0) { + numCores = Runtime.getRuntime().availableProcessors(); + } + return new PooledByteBufAllocator( + allowDirectBufs && PlatformDependent.directBufferPreferred(), + Math.min(PooledByteBufAllocator.defaultNumHeapArena(), numCores), + Math.min(PooledByteBufAllocator.defaultNumDirectArena(), allowDirectBufs ? numCores : 0), + PooledByteBufAllocator.defaultPageSize(), + PooledByteBufAllocator.defaultMaxOrder(), + allowCache ? PooledByteBufAllocator.defaultTinyCacheSize() : 0, + allowCache ? PooledByteBufAllocator.defaultSmallCacheSize() : 0, + allowCache ? PooledByteBufAllocator.defaultNormalCacheSize() : 0, + allowCache ? PooledByteBufAllocator.defaultUseCacheForAllThreads() : false + ); + } +} diff --git a/network-common/src/main/java/com/bytedance/css/network/util/TransportConf.java b/network-common/src/main/java/com/bytedance/css/network/util/TransportConf.java new file mode 100644 index 0000000..7a70682 --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/util/TransportConf.java @@ -0,0 +1,355 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.util; + +import java.util.Locale; +import java.util.Properties; + +import com.google.common.primitives.Ints; +import io.netty.util.NettyRuntime; + +/** + * A central location that tracks all the settings we expose to users. + */ +public class TransportConf { + + private final String CSS_NETWORK_IO_MODE_KEY; + private final String CSS_NETWORK_IO_PREFERDIRECTBUFS_KEY; + private final String CSS_NETWORK_IO_CONNECTIONTIMEOUT_KEY; + private final String CSS_NETWORK_IO_BACKLOG_KEY; + private final String CSS_NETWORK_IO_NUMCONNECTIONSPERPEER_KEY; + private final String CSS_NETWORK_IO_SERVERTHREADS_KEY; + private final String CSS_NETWORK_IO_CLIENTTHREADS_KEY; + private final String CSS_NETWORK_IO_RECEIVEBUFFER_KEY; + private final String CSS_NETWORK_IO_SENDBUFFER_KEY; + private final String CSS_NETWORK_SASL_TIMEOUT_KEY; + private final String CSS_NETWORK_IO_MAXRETRIES_KEY; + private final String CSS_NETWORK_IO_RETRYWAIT_KEY; + private final String CSS_NETWORK_IO_LAZYFD_KEY; + private final String CSS_NETWORK_VERBOSE_METRICS; + private final String CSS_NETWORK_IO_ENABLETCPKEEPALIVE_KEY; + + private final ConfigProvider conf; + + private final String module; + + public TransportConf(String module, ConfigProvider conf) { + this.module = module; + this.conf = conf; + CSS_NETWORK_IO_MODE_KEY = getConfKey("io.mode"); + CSS_NETWORK_IO_PREFERDIRECTBUFS_KEY = getConfKey("io.preferDirectBufs"); + CSS_NETWORK_IO_CONNECTIONTIMEOUT_KEY = getConfKey("io.connectionTimeout"); + CSS_NETWORK_IO_BACKLOG_KEY = getConfKey("io.backLog"); + CSS_NETWORK_IO_NUMCONNECTIONSPERPEER_KEY = getConfKey("io.numConnectionsPerPeer"); + CSS_NETWORK_IO_SERVERTHREADS_KEY = getConfKey("io.serverThreads"); + CSS_NETWORK_IO_CLIENTTHREADS_KEY = getConfKey("io.clientThreads"); + CSS_NETWORK_IO_RECEIVEBUFFER_KEY = getConfKey("io.receiveBuffer"); + CSS_NETWORK_IO_SENDBUFFER_KEY = getConfKey("io.sendBuffer"); + CSS_NETWORK_SASL_TIMEOUT_KEY = getConfKey("sasl.timeout"); + CSS_NETWORK_IO_MAXRETRIES_KEY = getConfKey("io.maxRetries"); + CSS_NETWORK_IO_RETRYWAIT_KEY = getConfKey("io.retryWait"); + CSS_NETWORK_IO_LAZYFD_KEY = getConfKey("io.lazyFD"); + CSS_NETWORK_VERBOSE_METRICS = getConfKey("io.enableVerboseMetrics"); + CSS_NETWORK_IO_ENABLETCPKEEPALIVE_KEY = getConfKey("io.enableTcpKeepAlive"); + } + + public int getInt(String name, int defaultValue) { + return conf.getInt(name, defaultValue); + } + + public String get(String name, String defaultValue) { + return conf.get(name, defaultValue); + } + + private String getConfKey(String suffix) { + return "css." + module + "." + suffix; + } + + public String getModuleName() { + return module; + } + + /** IO mode: nio or epoll */ + public String ioMode() { + return conf.get(CSS_NETWORK_IO_MODE_KEY, "NIO").toUpperCase(Locale.ROOT); + } + + /** If true, we will prefer allocating off-heap byte buffers within Netty. */ + public boolean preferDirectBufs() { + return conf.getBoolean(CSS_NETWORK_IO_PREFERDIRECTBUFS_KEY, true); + } + + /** Connect timeout in milliseconds. Default 120 secs. */ + public int connectionTimeoutMs() { + long defaultNetworkTimeoutS = JavaUtils.timeStringAsSec( + conf.get("css.network.timeout", "120s")); + long defaultTimeoutMs = JavaUtils.timeStringAsSec( + conf.get(CSS_NETWORK_IO_CONNECTIONTIMEOUT_KEY, defaultNetworkTimeoutS + "s")) * 1000; + return (int) defaultTimeoutMs; + } + + /** Number of concurrent connections between two nodes for fetching data. */ + public int numConnectionsPerPeer() { + return conf.getInt(CSS_NETWORK_IO_NUMCONNECTIONSPERPEER_KEY, 1); + } + + /** Requested maximum length of the queue of incoming connections. Default -1 for no backlog. */ + public int backLog() { return conf.getInt(CSS_NETWORK_IO_BACKLOG_KEY, -1); } + + /** Number of threads used in the server thread pool. Default to 0, which is 2x#cores. */ + public int serverThreads() { return conf.getInt(CSS_NETWORK_IO_SERVERTHREADS_KEY, 0); } + + /** Number of threads used in the client thread pool. Default to 0, which is 2x#cores. */ + public int clientThreads() { return conf.getInt(CSS_NETWORK_IO_CLIENTTHREADS_KEY, 0); } + + /** + * Receive buffer size (SO_RCVBUF). + * Note: the optimal size for receive buffer and send buffer should be + * latency * network_bandwidth. + * Assuming latency = 1ms, network_bandwidth = 10Gbps + * buffer size should be ~ 1.25MB + */ + public int receiveBuf() { return conf.getInt(CSS_NETWORK_IO_RECEIVEBUFFER_KEY, -1); } + + /** Send buffer size (SO_SNDBUF). */ + public int sendBuf() { return conf.getInt(CSS_NETWORK_IO_SENDBUFFER_KEY, -1); } + + /** Timeout for a single round trip of auth message exchange, in milliseconds. */ + public int authRTTimeoutMs() { + return (int) JavaUtils.timeStringAsSec(conf.get("css.network.auth.rpcTimeout", + conf.get(CSS_NETWORK_SASL_TIMEOUT_KEY, "30s"))) * 1000; + } + + /** + * Max number of times we will try IO exceptions (such as connection timeouts) per request. + * If set to 0, we will not do any retries. + */ + public int maxIORetries() { return conf.getInt(CSS_NETWORK_IO_MAXRETRIES_KEY, 3); } + + /** + * Time (in milliseconds) that we will wait in order to perform a retry after an IOException. + * Only relevant if maxIORetries > 0. + */ + public int ioRetryWaitTimeMs() { + return (int) JavaUtils.timeStringAsSec(conf.get(CSS_NETWORK_IO_RETRYWAIT_KEY, "5s")) * 1000; + } + + /** + * Minimum size of a block that we should start using memory map rather than reading in through + * normal IO operations. This prevents Spark from memory mapping very small blocks. In general, + * memory mapping has high overhead for blocks close to or below the page size of the OS. + */ + public int memoryMapBytes() { + return Ints.checkedCast(JavaUtils.byteStringAsBytes( + conf.get("css.storage.memoryMapThreshold", "2m"))); + } + + /** + * Whether to initialize FileDescriptor lazily or not. If true, file descriptors are + * created only when data is going to be transferred. This can reduce the number of open files. + */ + public boolean lazyFileDescriptor() { + return conf.getBoolean(CSS_NETWORK_IO_LAZYFD_KEY, true); + } + + /** + * Whether to track Netty memory detailed metrics. If true, the detailed metrics of Netty + * PoolByteBufAllocator will be gotten, otherwise only general memory usage will be tracked. + */ + public boolean verboseMetrics() { + return conf.getBoolean(CSS_NETWORK_VERBOSE_METRICS, false); + } + + /** + * Whether to enable TCP keep-alive. If true, the TCP keep-alives are enabled, which removes + * connections that are idle for too long. + */ + public boolean enableTcpKeepAlive() { + return conf.getBoolean(CSS_NETWORK_IO_ENABLETCPKEEPALIVE_KEY, false); + } + + /** + * Maximum number of retries when binding to a port before giving up. + */ + public int portMaxRetries() { + return conf.getInt("css.port.maxRetries", 16); + } + + /** + * Enables strong encryption. Also enables the new auth protocol, used to negotiate keys. + */ + public boolean encryptionEnabled() { + return conf.getBoolean("css.network.crypto.enabled", false); + } + + /** + * The cipher transformation to use for encrypting session data. + */ + public String cipherTransformation() { + return conf.get("css.network.crypto.cipher", "AES/CTR/NoPadding"); + } + + /** + * The key generation algorithm. This should be an algorithm that accepts a "PBEKeySpec" + * as input. The default value (PBKDF2WithHmacSHA1) is available in Java 7. + */ + public String keyFactoryAlgorithm() { + return conf.get("css.network.crypto.keyFactoryAlgorithm", "PBKDF2WithHmacSHA1"); + } + + /** + * How many iterations to run when generating keys. + * + * See some discussion about this at: http://security.stackexchange.com/q/3959 + * The default value was picked for speed, since it assumes that the secret has good entropy + * (128 bits by default), which is not generally the case with user passwords. + */ + public int keyFactoryIterations() { + return conf.getInt("css.network.crypto.keyFactoryIterations", 1024); + } + + /** + * Encryption key length, in bits. + */ + public int encryptionKeyLength() { + return conf.getInt("css.network.crypto.keyLength", 128); + } + + /** + * Initial vector length, in bytes. + */ + public int ivLength() { + return conf.getInt("css.network.crypto.ivLength", 16); + } + + /** + * The algorithm for generated secret keys. Nobody should really need to change this, + * but configurable just in case. + */ + public String keyAlgorithm() { + return conf.get("css.network.crypto.keyAlgorithm", "AES"); + } + + /** + * Whether to fall back to SASL if the new auth protocol fails. Enabled by default for + * backwards compatibility. + */ + public boolean saslFallback() { + return conf.getBoolean("css.network.crypto.saslFallback", true); + } + + /** + * Whether to enable SASL-based encryption when authenticating using SASL. + */ + public boolean saslEncryption() { + return conf.getBoolean("css.authenticate.enableSaslEncryption", false); + } + + /** + * Maximum number of bytes to be encrypted at a time when SASL encryption is used. + */ + public int maxSaslEncryptedBlockSize() { + return Ints.checkedCast(JavaUtils.byteStringAsBytes( + conf.get("css.network.sasl.maxEncryptedBlockSize", "64k"))); + } + + /** + * Whether the server should enforce encryption on SASL-authenticated connections. + */ + public boolean saslServerAlwaysEncrypt() { + return conf.getBoolean("css.network.sasl.serverAlwaysEncrypt", false); + } + + /** + * Flag indicating whether to share the pooled ByteBuf allocators between the different Netty + * channels. If enabled then only two pooled ByteBuf allocators are created: one where caching + * is allowed (for transport servers) and one where not (for transport clients). + * When disabled a new allocator is created for each transport servers and clients. + */ + public boolean sharedByteBufAllocators() { + return conf.getBoolean("css.network.sharedByteBufAllocators.enabled", true); + } + + /** + * If enabled then off-heap byte buffers will be preferred for the shared ByteBuf allocators. + */ + public boolean preferDirectBufsForSharedByteBufAllocators() { + return conf.getBoolean("css.network.io.preferDirectBufs", true); + } + + /** + * The commons-crypto configuration for the module. + */ + public Properties cryptoConf() { + return CryptoUtils.toCryptoConf("css.network.crypto.config.", conf.getAll()); + } + + /** + * The max number of chunks allowed to be transferred at the same time on shuffle service. + * Note that new incoming connections will be closed when the max number is hit. The client will + * retry according to the shuffle retry configs (see `css.shuffle.io.maxRetries` and + * `css.shuffle.io.retryWait`), if those limits are reached the task will fail with fetch + * failure. + */ + public long maxChunksBeingTransferred() { + return conf.getLong("css.shuffle.maxChunksBeingTransferred", Long.MAX_VALUE); + } + + /** + * Percentage of io.serverThreads used by netty to process ChunkFetchRequest. + * When the config `spark.shuffle.server.chunkFetchHandlerThreadsPercent` is set, + * shuffle server will use a separate EventLoopGroup to process ChunkFetchRequest messages. + * Although when calling the async writeAndFlush on the underlying channel to send + * response back to client, the I/O on the channel is still being handled by + * {@link com.bytedance.css.network.server.TransportServer}'s default EventLoopGroup + * that's registered with the Channel, by waiting inside the ChunkFetchRequest handler + * threads for the completion of sending back responses, we are able to put a limit on + * the max number of threads from TransportServer's default EventLoopGroup that are + * going to be consumed by writing response to ChunkFetchRequest, which are I/O intensive + * and could take long time to process due to disk contentions. By configuring a slightly + * higher number of shuffler server threads, we are able to reserve some threads for + * handling other RPC messages, thus making the Client less likely to experience timeout + * when sending RPC messages to the shuffle server. The number of threads used for handling + * chunked fetch requests are percentage of io.serverThreads (if defined) else it is a percentage + * of 2 * #cores. However, a percentage of 0 means netty default number of threads which + * is 2 * #cores ignoring io.serverThreads. The percentage here is configured via + * spark.shuffle.server.chunkFetchHandlerThreadsPercent. The returned value is rounded off to + * ceiling of the nearest integer. + */ + public int chunkFetchHandlerThreads() { + if (!this.getModuleName().equalsIgnoreCase("fetch")) { + return 0; + } + int chunkFetchHandlerThreadsPercent = + Integer.parseInt(conf.get("css.shuffle.server.chunkFetchHandlerThreadsPercent")); + int threads = + this.serverThreads() > 0 ? this.serverThreads() : 2 * NettyRuntime.availableProcessors(); + return (int) Math.ceil(threads * (chunkFetchHandlerThreadsPercent / 100.0)); + } + + /** + * Whether to use a separate EventLoopGroup to process ChunkFetchRequest messages, it is decided + * by the config `spark.shuffle.server.chunkFetchHandlerThreadsPercent` is set or not. + */ + public boolean separateChunkFetchRequest() { + return conf.getInt("css.shuffle.server.chunkFetchHandlerThreadsPercent", 0) > 0; + } + +} diff --git a/network-common/src/main/java/com/bytedance/css/network/util/TransportFrameDecoder.java b/network-common/src/main/java/com/bytedance/css/network/util/TransportFrameDecoder.java new file mode 100644 index 0000000..d72d26f --- /dev/null +++ b/network-common/src/main/java/com/bytedance/css/network/util/TransportFrameDecoder.java @@ -0,0 +1,289 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.util; + +import java.util.LinkedList; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.CompositeByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; + +/** + * A customized frame decoder that allows intercepting raw data. + *

    + * This behaves like Netty's frame decoder (with hard coded parameters that match this library's + * needs), except it allows an interceptor to be installed to read data directly before it's + * framed. + *

    + * Unlike Netty's frame decoder, each frame is dispatched to child handlers as soon as it's + * decoded, instead of building as many frames as the current buffer allows and dispatching + * all of them. This allows a child handler to install an interceptor if needed. + *

    + * If an interceptor is installed, framing stops, and data is instead fed directly to the + * interceptor. When the interceptor indicates that it doesn't need to read any more data, + * framing resumes. Interceptors should not hold references to the data buffers provided + * to their handle() method. + */ +public class TransportFrameDecoder extends ChannelInboundHandlerAdapter { + + public static final String HANDLER_NAME = "frameDecoder"; + private static final int LENGTH_SIZE = 8; + private static final int MAX_FRAME_SIZE = Integer.MAX_VALUE; + private static final int UNKNOWN_FRAME_SIZE = -1; + private static final long CONSOLIDATE_THRESHOLD = 20 * 1024 * 1024; + + private final LinkedList buffers = new LinkedList<>(); + private final ByteBuf frameLenBuf = Unpooled.buffer(LENGTH_SIZE, LENGTH_SIZE); + private final long consolidateThreshold; + + private CompositeByteBuf frameBuf = null; + private long consolidatedFrameBufSize = 0; + private int consolidatedNumComponents = 0; + + private long totalSize = 0; + private long nextFrameSize = UNKNOWN_FRAME_SIZE; + private int frameRemainingBytes = UNKNOWN_FRAME_SIZE; + private volatile Interceptor interceptor; + + public TransportFrameDecoder() { + this(CONSOLIDATE_THRESHOLD); + } + + @VisibleForTesting + TransportFrameDecoder(long consolidateThreshold) { + this.consolidateThreshold = consolidateThreshold; + } + + @Override + public void channelRead(ChannelHandlerContext ctx, Object data) throws Exception { + ByteBuf in = (ByteBuf) data; + buffers.add(in); + totalSize += in.readableBytes(); + + while (!buffers.isEmpty()) { + // First, feed the interceptor, and if it's still, active, try again. + if (interceptor != null) { + ByteBuf first = buffers.getFirst(); + int available = first.readableBytes(); + if (feedInterceptor(first)) { + assert !first.isReadable() : "Interceptor still active but buffer has data."; + } + + int read = available - first.readableBytes(); + if (read == available) { + buffers.removeFirst().release(); + } + totalSize -= read; + } else { + // Interceptor is not active, so try to decode one frame. + ByteBuf frame = decodeNext(); + if (frame == null) { + break; + } + ctx.fireChannelRead(frame); + } + } + } + + private long decodeFrameSize() { + if (nextFrameSize != UNKNOWN_FRAME_SIZE || totalSize < LENGTH_SIZE) { + return nextFrameSize; + } + + // We know there's enough data. If the first buffer contains all the data, great. Otherwise, + // hold the bytes for the frame length in a composite buffer until we have enough data to read + // the frame size. Normally, it should be rare to need more than one buffer to read the frame + // size. + ByteBuf first = buffers.getFirst(); + if (first.readableBytes() >= LENGTH_SIZE) { + nextFrameSize = first.readLong() - LENGTH_SIZE; + totalSize -= LENGTH_SIZE; + if (!first.isReadable()) { + buffers.removeFirst().release(); + } + return nextFrameSize; + } + + while (frameLenBuf.readableBytes() < LENGTH_SIZE) { + ByteBuf next = buffers.getFirst(); + int toRead = Math.min(next.readableBytes(), LENGTH_SIZE - frameLenBuf.readableBytes()); + frameLenBuf.writeBytes(next, toRead); + if (!next.isReadable()) { + buffers.removeFirst().release(); + } + } + + nextFrameSize = frameLenBuf.readLong() - LENGTH_SIZE; + totalSize -= LENGTH_SIZE; + frameLenBuf.clear(); + return nextFrameSize; + } + + private ByteBuf decodeNext() { + long frameSize = decodeFrameSize(); + if (frameSize == UNKNOWN_FRAME_SIZE) { + return null; + } + + if (frameBuf == null) { + Preconditions.checkArgument(frameSize < MAX_FRAME_SIZE, + "Too large frame: %s", frameSize); + Preconditions.checkArgument(frameSize > 0, + "Frame length should be positive: %s", frameSize); + frameRemainingBytes = (int) frameSize; + + // If buffers is empty, then return immediately for more input data. + if (buffers.isEmpty()) { + return null; + } + // Otherwise, if the first buffer holds the entire frame, we attempt to + // build frame with it and return. + if (buffers.getFirst().readableBytes() >= frameRemainingBytes) { + // Reset buf and size for next frame. + frameBuf = null; + nextFrameSize = UNKNOWN_FRAME_SIZE; + return nextBufferForFrame(frameRemainingBytes); + } + // Other cases, create a composite buffer to manage all the buffers. + frameBuf = buffers.getFirst().alloc().compositeBuffer(Integer.MAX_VALUE); + } + + while (frameRemainingBytes > 0 && !buffers.isEmpty()) { + ByteBuf next = nextBufferForFrame(frameRemainingBytes); + frameRemainingBytes -= next.readableBytes(); + frameBuf.addComponent(true, next); + } + // If the delta size of frameBuf exceeds the threshold, then we do consolidation + // to reduce memory consumption. + if (frameBuf.capacity() - consolidatedFrameBufSize > consolidateThreshold) { + int newNumComponents = frameBuf.numComponents() - consolidatedNumComponents; + frameBuf.consolidate(consolidatedNumComponents, newNumComponents); + consolidatedFrameBufSize = frameBuf.capacity(); + consolidatedNumComponents = frameBuf.numComponents(); + } + if (frameRemainingBytes > 0) { + return null; + } + + return consumeCurrentFrameBuf(); + } + + private ByteBuf consumeCurrentFrameBuf() { + ByteBuf frame = frameBuf; + // Reset buf and size for next frame. + frameBuf = null; + consolidatedFrameBufSize = 0; + consolidatedNumComponents = 0; + nextFrameSize = UNKNOWN_FRAME_SIZE; + return frame; + } + + /** + * Takes the first buffer in the internal list, and either adjust it to fit in the frame + * (by taking a slice out of it) or remove it from the internal list. + */ + private ByteBuf nextBufferForFrame(int bytesToRead) { + ByteBuf buf = buffers.getFirst(); + ByteBuf frame; + + if (buf.readableBytes() > bytesToRead) { + frame = buf.retain().readSlice(bytesToRead); + totalSize -= bytesToRead; + } else { + frame = buf; + buffers.removeFirst(); + totalSize -= frame.readableBytes(); + } + + return frame; + } + + @Override + public void channelInactive(ChannelHandlerContext ctx) throws Exception { + if (interceptor != null) { + interceptor.channelInactive(); + } + super.channelInactive(ctx); + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + if (interceptor != null) { + interceptor.exceptionCaught(cause); + } + super.exceptionCaught(ctx, cause); + } + + @Override + public void handlerRemoved(ChannelHandlerContext ctx) throws Exception { + // Release all buffers that are still in our ownership. + // Doing this in handlerRemoved(...) guarantees that this will happen in all cases: + // - When the Channel becomes inactive + // - When the decoder is removed from the ChannelPipeline + for (ByteBuf b : buffers) { + b.release(); + } + buffers.clear(); + frameLenBuf.release(); + ByteBuf frame = consumeCurrentFrameBuf(); + if (frame != null) { + frame.release(); + } + super.handlerRemoved(ctx); + } + + public void setInterceptor(Interceptor interceptor) { + Preconditions.checkState(this.interceptor == null, "Already have an interceptor."); + this.interceptor = interceptor; + } + + /** + * @return Whether the interceptor is still active after processing the data. + */ + private boolean feedInterceptor(ByteBuf buf) throws Exception { + if (interceptor != null && !interceptor.handle(buf)) { + interceptor = null; + } + return interceptor != null; + } + + public interface Interceptor { + + /** + * Handles data received from the remote end. + * + * @param data Buffer containing data. + * @return "true" if the interceptor expects more data, "false" to uninstall the interceptor. + */ + boolean handle(ByteBuf data) throws Exception; + + /** Called if an exception is thrown in the channel pipeline. */ + void exceptionCaught(Throwable cause) throws Exception; + + /** Called if the channel is closed and the interceptor is still installed. */ + void channelInactive() throws Exception; + + } + +} diff --git a/network-common/src/main/java/io/netty/buffer/NettyMemoryMetrics.java b/network-common/src/main/java/io/netty/buffer/NettyMemoryMetrics.java new file mode 100644 index 0000000..5f3afbf --- /dev/null +++ b/network-common/src/main/java/io/netty/buffer/NettyMemoryMetrics.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package io.netty.buffer; + +import com.bytedance.css.network.util.TransportConf; +import com.codahale.metrics.Gauge; +import com.codahale.metrics.Metric; +import com.codahale.metrics.MetricRegistry; +import com.codahale.metrics.MetricSet; +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.nio.ByteBuffer; +import java.util.*; + +/** + * A Netty memory metrics class to collect metrics from Netty PooledByteBufAllocator. + */ +public class NettyMemoryMetrics implements MetricSet { + + private static final Logger logger = LoggerFactory.getLogger(NettyMemoryMetrics.class); + + private final PooledByteBufAllocator pooledAllocator; + + private final boolean verboseMetricsEnabled; + + private final Map allMetrics; + + private final String metricPrefix; + + @VisibleForTesting + public static final Set VERBOSE_METRICS = new HashSet<>(); + static { + VERBOSE_METRICS.addAll(Arrays.asList( + "numAllocations", + "numTinyAllocations", + "numSmallAllocations", + "numNormalAllocations", + "numHugeAllocations", + "numDeallocations", + "numTinyDeallocations", + "numSmallDeallocations", + "numNormalDeallocations", + "numHugeDeallocations", + "numActiveAllocations", + "numActiveTinyAllocations", + "numActiveSmallAllocations", + "numActiveNormalAllocations", + "numActiveHugeAllocations", + "numActiveBytes")); + } + + public NettyMemoryMetrics(PooledByteBufAllocator pooledAllocator, + String metricPrefix, + TransportConf conf) { + this.pooledAllocator = pooledAllocator; + this.allMetrics = new HashMap<>(); + this.metricPrefix = metricPrefix; + this.verboseMetricsEnabled = conf.verboseMetrics(); + + registerMetrics(this.pooledAllocator); + } + + private void registerMetrics(PooledByteBufAllocator allocator) { + PooledByteBufAllocatorMetric pooledAllocatorMetric = allocator.metric(); + + // Register general metrics. + allMetrics.put(MetricRegistry.name(metricPrefix, "usedHeapMemory"), + (Gauge) () -> pooledAllocatorMetric.usedHeapMemory()); + allMetrics.put(MetricRegistry.name(metricPrefix, "usedDirectMemory"), + (Gauge) () -> pooledAllocatorMetric.usedDirectMemory()); + allMetrics.put(MetricRegistry.name(metricPrefix, "realDirectMemory"), + (Gauge) () -> { + try { + Field field = PooledByteBufAllocator.class.getDeclaredField("directArenas"); + field.setAccessible(true); + PoolArena[] directArenas = (PoolArena[]) field.get(allocator); + long free = 0L; + for (PoolArena arena : directArenas) { + for (int i = 0; i < arena.chunkLists().size(); i++) { + for (PoolChunkMetric m: arena.chunkLists().get(i)) { + free += m.freeBytes(); + } + } + } + return pooledAllocatorMetric.usedDirectMemory() - free; + } catch (Exception ex) { + logger.error("realDirectMemory metrics reflect failed.", ex); + return -1L; + } + }); + + if (verboseMetricsEnabled) { + int directArenaIndex = 0; + for (PoolArenaMetric metric : pooledAllocatorMetric.directArenas()) { + registerArenaMetric(metric, "directArena" + directArenaIndex); + directArenaIndex++; + } + + int heapArenaIndex = 0; + for (PoolArenaMetric metric : pooledAllocatorMetric.heapArenas()) { + registerArenaMetric(metric, "heapArena" + heapArenaIndex); + heapArenaIndex++; + } + } + } + + private void registerArenaMetric(PoolArenaMetric arenaMetric, String arenaName) { + for (String methodName : VERBOSE_METRICS) { + Method m; + try { + m = PoolArenaMetric.class.getMethod(methodName); + } catch (Exception e) { + // Failed to find metric related method, ignore this metric. + continue; + } + + if (!Modifier.isPublic(m.getModifiers())) { + // Ignore non-public methods. + continue; + } + + Class returnType = m.getReturnType(); + String metricName = MetricRegistry.name(metricPrefix, arenaName, m.getName()); + if (returnType.equals(int.class)) { + allMetrics.put(metricName, (Gauge) () -> { + try { + return (Integer) m.invoke(arenaMetric); + } catch (Exception e) { + return -1; // Swallow the exceptions. + } + }); + + } else if (returnType.equals(long.class)) { + allMetrics.put(metricName, (Gauge) () -> { + try { + return (Long) m.invoke(arenaMetric); + } catch (Exception e) { + return -1L; // Swallow the exceptions. + } + }); + } + } + } + + @Override + public Map getMetrics() { + return Collections.unmodifiableMap(allMetrics); + } +} diff --git a/network-common/src/test/java/com/bytedance/css/network/ChunkFetchIntegrationSuite.java b/network-common/src/test/java/com/bytedance/css/network/ChunkFetchIntegrationSuite.java new file mode 100644 index 0000000..1b0cb16 --- /dev/null +++ b/network-common/src/test/java/com/bytedance/css/network/ChunkFetchIntegrationSuite.java @@ -0,0 +1,238 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network; + +import com.google.common.collect.Sets; +import com.google.common.io.Closeables; +import com.bytedance.css.network.buffer.FileSegmentManagedBuffer; +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.buffer.NioManagedBuffer; +import com.bytedance.css.network.client.ChunkReceivedCallback; +import com.bytedance.css.network.client.RpcResponseCallback; +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.client.TransportClientFactory; +import com.bytedance.css.network.server.RpcHandler; +import com.bytedance.css.network.server.StreamManager; +import com.bytedance.css.network.server.TransportServer; +import com.bytedance.css.network.util.MapConfigProvider; +import com.bytedance.css.network.util.TransportConf; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.File; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.util.*; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.*; + +public class ChunkFetchIntegrationSuite { + static final long STREAM_ID = 1; + static final int BUFFER_CHUNK_INDEX = 0; + static final int FILE_CHUNK_INDEX = 1; + + static TransportServer server; + static TransportClientFactory clientFactory; + static StreamManager streamManager; + static File testFile; + + static ManagedBuffer bufferChunk; + static ManagedBuffer fileChunk; + + @BeforeClass + public static void setUp() throws Exception { + int bufSize = 100000; + final ByteBuffer buf = ByteBuffer.allocate(bufSize); + for (int i = 0; i < bufSize; i ++) { + buf.put((byte) i); + } + buf.flip(); + bufferChunk = new NioManagedBuffer(buf); + + testFile = File.createTempFile("shuffle-test-file", "txt"); + testFile.deleteOnExit(); + RandomAccessFile fp = new RandomAccessFile(testFile, "rw"); + boolean shouldSuppressIOException = true; + try { + byte[] fileContent = new byte[1024]; + new Random().nextBytes(fileContent); + fp.write(fileContent); + shouldSuppressIOException = false; + } finally { + Closeables.close(fp, shouldSuppressIOException); + } + + final TransportConf conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); + fileChunk = new FileSegmentManagedBuffer(conf, testFile, 10, testFile.length() - 25); + + streamManager = new StreamManager() { + @Override + public ManagedBuffer getChunk(long streamId, int chunkIndex) { + assertEquals(STREAM_ID, streamId); + if (chunkIndex == BUFFER_CHUNK_INDEX) { + return new NioManagedBuffer(buf); + } else if (chunkIndex == FILE_CHUNK_INDEX) { + return new FileSegmentManagedBuffer(conf, testFile, 10, testFile.length() - 25); + } else { + throw new IllegalArgumentException("Invalid chunk index: " + chunkIndex); + } + } + }; + RpcHandler handler = new RpcHandler() { + @Override + public void receive( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { + throw new UnsupportedOperationException(); + } + + @Override + public StreamManager getStreamManager() { + return streamManager; + } + }; + TransportContext context = new TransportContext(conf, handler); + server = context.createServer(); + clientFactory = context.createClientFactory(); + } + + @AfterClass + public static void tearDown() { + bufferChunk.release(); + server.close(); + clientFactory.close(); + testFile.delete(); + } + + static class FetchResult { + public Set successChunks; + public Set failedChunks; + public List buffers; + + public void releaseBuffers() { + for (ManagedBuffer buffer : buffers) { + buffer.release(); + } + } + } + + private FetchResult fetchChunks(List chunkIndices) throws Exception { + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + final Semaphore sem = new Semaphore(0); + + final FetchResult res = new FetchResult(); + res.successChunks = Collections.synchronizedSet(new HashSet()); + res.failedChunks = Collections.synchronizedSet(new HashSet()); + res.buffers = Collections.synchronizedList(new LinkedList()); + + ChunkReceivedCallback callback = new ChunkReceivedCallback() { + @Override + public void onSuccess(int chunkIndex, ManagedBuffer buffer) { + buffer.retain(); + res.successChunks.add(chunkIndex); + res.buffers.add(buffer); + sem.release(); + } + + @Override + public void onFailure(int chunkIndex, Throwable e) { + res.failedChunks.add(chunkIndex); + sem.release(); + } + }; + + for (int chunkIndex : chunkIndices) { + client.fetchChunk(STREAM_ID, chunkIndex, callback); + } + if (!sem.tryAcquire(chunkIndices.size(), 5, TimeUnit.SECONDS)) { + fail("Timeout getting response from the server"); + } + client.close(); + return res; + } + + @Test + public void fetchBufferChunk() throws Exception { + FetchResult res = fetchChunks(Arrays.asList(BUFFER_CHUNK_INDEX)); + assertEquals(Sets.newHashSet(BUFFER_CHUNK_INDEX), res.successChunks); + assertTrue(res.failedChunks.isEmpty()); + assertBufferListsEqual(Arrays.asList(bufferChunk), res.buffers); + res.releaseBuffers(); + } + + @Test + public void fetchFileChunk() throws Exception { + FetchResult res = fetchChunks(Arrays.asList(FILE_CHUNK_INDEX)); + assertEquals(Sets.newHashSet(FILE_CHUNK_INDEX), res.successChunks); + assertTrue(res.failedChunks.isEmpty()); + assertBufferListsEqual(Arrays.asList(fileChunk), res.buffers); + res.releaseBuffers(); + } + + @Test + public void fetchNonExistentChunk() throws Exception { + FetchResult res = fetchChunks(Arrays.asList(12345)); + assertTrue(res.successChunks.isEmpty()); + assertEquals(Sets.newHashSet(12345), res.failedChunks); + assertTrue(res.buffers.isEmpty()); + } + + @Test + public void fetchBothChunks() throws Exception { + FetchResult res = fetchChunks(Arrays.asList(BUFFER_CHUNK_INDEX, FILE_CHUNK_INDEX)); + assertEquals(Sets.newHashSet(BUFFER_CHUNK_INDEX, FILE_CHUNK_INDEX), res.successChunks); + assertTrue(res.failedChunks.isEmpty()); + assertBufferListsEqual(Arrays.asList(bufferChunk, fileChunk), res.buffers); + res.releaseBuffers(); + } + + @Test + public void fetchChunkAndNonExistent() throws Exception { + FetchResult res = fetchChunks(Arrays.asList(BUFFER_CHUNK_INDEX, 12345)); + assertEquals(Sets.newHashSet(BUFFER_CHUNK_INDEX), res.successChunks); + assertEquals(Sets.newHashSet(12345), res.failedChunks); + assertBufferListsEqual(Arrays.asList(bufferChunk), res.buffers); + res.releaseBuffers(); + } + + private static void assertBufferListsEqual(List list0, List list1) + throws Exception { + assertEquals(list0.size(), list1.size()); + for (int i = 0; i < list0.size(); i ++) { + assertBuffersEqual(list0.get(i), list1.get(i)); + } + } + + private static void assertBuffersEqual(ManagedBuffer buffer0, ManagedBuffer buffer1) + throws Exception { + ByteBuffer nio0 = buffer0.nioByteBuffer(); + ByteBuffer nio1 = buffer1.nioByteBuffer(); + + int len = nio0.remaining(); + assertEquals(nio0.remaining(), nio1.remaining()); + for (int i = 0; i < len; i ++) { + assertEquals(nio0.get(), nio1.get()); + } + } +} diff --git a/network-common/src/test/java/com/bytedance/css/network/ProtocolSuite.java b/network-common/src/test/java/com/bytedance/css/network/ProtocolSuite.java new file mode 100644 index 0000000..ff6bfb1 --- /dev/null +++ b/network-common/src/test/java/com/bytedance/css/network/ProtocolSuite.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network; + +import com.google.common.primitives.Ints; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.FileRegion; +import io.netty.channel.embedded.EmbeddedChannel; +import io.netty.handler.codec.MessageToMessageEncoder; +import com.bytedance.css.network.protocol.*; +import com.bytedance.css.network.util.ByteArrayWritableChannel; +import com.bytedance.css.network.util.NettyUtils; +import org.junit.Test; + +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class ProtocolSuite { + private void testServerToClient(Message msg) { + EmbeddedChannel serverChannel = new EmbeddedChannel(new FileRegionEncoder(), + MessageEncoder.INSTANCE); + serverChannel.writeOutbound(msg); + + EmbeddedChannel clientChannel = new EmbeddedChannel( + NettyUtils.createFrameDecoder(), MessageDecoder.INSTANCE); + + while (!serverChannel.outboundMessages().isEmpty()) { + clientChannel.writeOneInbound(serverChannel.readOutbound()); + } + + assertEquals(1, clientChannel.inboundMessages().size()); + assertEquals(msg, clientChannel.readInbound()); + } + + private void testClientToServer(Message msg) { + EmbeddedChannel clientChannel = new EmbeddedChannel(new FileRegionEncoder(), + MessageEncoder.INSTANCE); + clientChannel.writeOutbound(msg); + + EmbeddedChannel serverChannel = new EmbeddedChannel( + NettyUtils.createFrameDecoder(), MessageDecoder.INSTANCE); + + while (!clientChannel.outboundMessages().isEmpty()) { + serverChannel.writeOneInbound(clientChannel.readOutbound()); + } + + assertEquals(1, serverChannel.inboundMessages().size()); + assertEquals(msg, serverChannel.readInbound()); + } + + @Test + public void requests() { + testClientToServer(new ChunkFetchRequest(new StreamChunkId(1, 2))); + testClientToServer(new RpcRequest(12345, new TestManagedBuffer(0))); + testClientToServer(new RpcRequest(12345, new TestManagedBuffer(10))); + testClientToServer(new StreamRequest("abcde")); + testClientToServer(new OneWayMessage(new TestManagedBuffer(10))); + } + + @Test + public void responses() { + testServerToClient(new ChunkFetchSuccess(new StreamChunkId(1, 2), new TestManagedBuffer(10))); + testServerToClient(new ChunkFetchSuccess(new StreamChunkId(1, 2), new TestManagedBuffer(0))); + testServerToClient(new ChunkFetchFailure(new StreamChunkId(1, 2), "this is an error")); + testServerToClient(new ChunkFetchFailure(new StreamChunkId(1, 2), "")); + testServerToClient(new RpcResponse(12345, new TestManagedBuffer(0))); + testServerToClient(new RpcResponse(12345, new TestManagedBuffer(100))); + testServerToClient(new RpcFailure(0, "this is an error")); + testServerToClient(new RpcFailure(0, "")); + // Note: buffer size must be "0" since StreamResponse's buffer is written differently to the + // channel and cannot be tested like this. + testServerToClient(new StreamResponse("anId", 12345L, new TestManagedBuffer(0))); + testServerToClient(new StreamFailure("anId", "this is an error")); + } + + /** + * Handler to transform a FileRegion into a byte buffer. EmbeddedChannel doesn't actually transfer + * bytes, but messages, so this is needed so that the frame decoder on the receiving side can + * understand what MessageWithHeader actually contains. + */ + private static class FileRegionEncoder extends MessageToMessageEncoder { + + @Override + public void encode(ChannelHandlerContext ctx, FileRegion in, List out) + throws Exception { + + ByteArrayWritableChannel channel = new ByteArrayWritableChannel(Ints.checkedCast(in.count())); + while (in.transferred() < in.count()) { + in.transferTo(channel, in.transferred()); + } + out.add(Unpooled.wrappedBuffer(channel.getData())); + } + + } + +} diff --git a/network-common/src/test/java/com/bytedance/css/network/RequestTimeoutIntegrationSuite.java b/network-common/src/test/java/com/bytedance/css/network/RequestTimeoutIntegrationSuite.java new file mode 100644 index 0000000..75b90ff --- /dev/null +++ b/network-common/src/test/java/com/bytedance/css/network/RequestTimeoutIntegrationSuite.java @@ -0,0 +1,279 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network; + +import com.google.common.util.concurrent.Uninterruptibles; +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.buffer.NioManagedBuffer; +import com.bytedance.css.network.client.ChunkReceivedCallback; +import com.bytedance.css.network.client.RpcResponseCallback; +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.client.TransportClientFactory; +import com.bytedance.css.network.server.RpcHandler; +import com.bytedance.css.network.server.StreamManager; +import com.bytedance.css.network.server.TransportServer; +import com.bytedance.css.network.util.MapConfigProvider; +import com.bytedance.css.network.util.TransportConf; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.*; + +/** + * Suite which ensures that requests that go without a response for the network timeout period are + * failed, and the connection closed. + * + * In this suite, we use 10 seconds as the connection timeout, with some slack given in the tests, + * to ensure stability in different test environments. + */ +public class RequestTimeoutIntegrationSuite { + + private TransportServer server; + private TransportClientFactory clientFactory; + + private StreamManager defaultManager; + private TransportConf conf; + + // A large timeout that "shouldn't happen", for the sake of faulty tests not hanging forever. + private static final int FOREVER = 60 * 1000; + + @Before + public void setUp() throws Exception { + Map configMap = new HashMap<>(); + configMap.put("css.shuffle.io.connectionTimeout", "10s"); + conf = new TransportConf("shuffle", new MapConfigProvider(configMap)); + + defaultManager = new StreamManager() { + @Override + public ManagedBuffer getChunk(long streamId, int chunkIndex) { + throw new UnsupportedOperationException(); + } + }; + } + + @After + public void tearDown() { + if (server != null) { + server.close(); + } + if (clientFactory != null) { + clientFactory.close(); + } + } + + // Basic suite: First request completes quickly, and second waits for longer than network timeout. + @Test + public void timeoutInactiveRequests() throws Exception { + final Semaphore semaphore = new Semaphore(1); + final int responseSize = 16; + RpcHandler handler = new RpcHandler() { + @Override + public void receive( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { + try { + semaphore.acquire(); + callback.onSuccess(ByteBuffer.allocate(responseSize)); + } catch (InterruptedException e) { + // do nothing + } + } + + @Override + public StreamManager getStreamManager() { + return defaultManager; + } + }; + + TransportContext context = new TransportContext(conf, handler); + server = context.createServer(); + clientFactory = context.createClientFactory(); + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + + // First completes quickly (semaphore starts at 1). + TestCallback callback0 = new TestCallback(); + client.sendRpc(ByteBuffer.allocate(0), callback0); + callback0.latch.await(); + assertEquals(responseSize, callback0.successLength); + + // Second times out after 10 seconds, with slack. Must be IOException. + TestCallback callback1 = new TestCallback(); + client.sendRpc(ByteBuffer.allocate(0), callback1); + callback1.latch.await(60, TimeUnit.SECONDS); + assertNotNull(callback1.failure); + assertTrue(callback1.failure instanceof IOException); + + semaphore.release(); + } + + // A timeout will cause the connection to be closed, invalidating the current TransportClient. + // It should be the case that requesting a client from the factory produces a new, valid one. + @Test + public void timeoutCleanlyClosesClient() throws Exception { + final Semaphore semaphore = new Semaphore(0); + final int responseSize = 16; + RpcHandler handler = new RpcHandler() { + @Override + public void receive( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { + try { + semaphore.acquire(); + callback.onSuccess(ByteBuffer.allocate(responseSize)); + } catch (InterruptedException e) { + // do nothing + } + } + + @Override + public StreamManager getStreamManager() { + return defaultManager; + } + }; + + TransportContext context = new TransportContext(conf, handler); + server = context.createServer(); + clientFactory = context.createClientFactory(); + + // First request should eventually fail. + TransportClient client0 = + clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + TestCallback callback0 = new TestCallback(); + client0.sendRpc(ByteBuffer.allocate(0), callback0); + callback0.latch.await(); + assertTrue(callback0.failure instanceof IOException); + assertFalse(client0.isActive()); + + // Increment the semaphore and the second request should succeed quickly. + semaphore.release(2); + TransportClient client1 = + clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + TestCallback callback1 = new TestCallback(); + client1.sendRpc(ByteBuffer.allocate(0), callback1); + callback1.latch.await(); + assertEquals(responseSize, callback1.successLength); + assertNull(callback1.failure); + } + + // The timeout is relative to the LAST request sent, which is kinda weird, but still. + // This test also makes sure the timeout works for Fetch requests as well as RPCs. + @Test + public void furtherRequestsDelay() throws Exception { + final byte[] response = new byte[16]; + final StreamManager manager = new StreamManager() { + @Override + public ManagedBuffer getChunk(long streamId, int chunkIndex) { + Uninterruptibles.sleepUninterruptibly(FOREVER, TimeUnit.MILLISECONDS); + return new NioManagedBuffer(ByteBuffer.wrap(response)); + } + }; + RpcHandler handler = new RpcHandler() { + @Override + public void receive( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { + throw new UnsupportedOperationException(); + } + + @Override + public StreamManager getStreamManager() { + return manager; + } + }; + + TransportContext context = new TransportContext(conf, handler); + server = context.createServer(); + clientFactory = context.createClientFactory(); + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + + // Send one request, which will eventually fail. + TestCallback callback0 = new TestCallback(); + client.fetchChunk(0, 0, callback0); + Uninterruptibles.sleepUninterruptibly(1200, TimeUnit.MILLISECONDS); + + // Send a second request before the first has failed. + TestCallback callback1 = new TestCallback(); + client.fetchChunk(0, 1, callback1); + Uninterruptibles.sleepUninterruptibly(1200, TimeUnit.MILLISECONDS); + + // not complete yet, but should complete soon + assertEquals(-1, callback0.successLength); + assertNull(callback0.failure); + callback0.latch.await(60, TimeUnit.SECONDS); + assertTrue(callback0.failure instanceof IOException); + + // make sure callback1 is called. + callback1.latch.await(60, TimeUnit.SECONDS); + // failed at same time as previous + assertTrue(callback1.failure instanceof IOException); + } + + /** + * Callback which sets 'success' or 'failure' on completion. + * Additionally notifies all waiters on this callback when invoked. + */ + static class TestCallback implements RpcResponseCallback, ChunkReceivedCallback { + + int successLength = -1; + Throwable failure; + final CountDownLatch latch = new CountDownLatch(1); + + @Override + public void onSuccess(ByteBuffer response) { + successLength = response.remaining(); + latch.countDown(); + } + + @Override + public void onFailure(Throwable e) { + failure = e; + latch.countDown(); + } + + @Override + public void onSuccess(int chunkIndex, ManagedBuffer buffer) { + try { + successLength = buffer.nioByteBuffer().remaining(); + } catch (IOException e) { + // weird + } finally { + latch.countDown(); + } + } + + @Override + public void onFailure(int chunkIndex, Throwable e) { + failure = e; + latch.countDown(); + } + } +} diff --git a/network-common/src/test/java/com/bytedance/css/network/RpcIntegrationSuite.java b/network-common/src/test/java/com/bytedance/css/network/RpcIntegrationSuite.java new file mode 100644 index 0000000..96a4ec5 --- /dev/null +++ b/network-common/src/test/java/com/bytedance/css/network/RpcIntegrationSuite.java @@ -0,0 +1,487 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network; + +import com.bytedance.css.network.client.RpcResponseCallback; +import com.bytedance.css.network.client.StreamCallbackWithID; +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.client.TransportClientFactory; +import com.bytedance.css.network.server.OneForOneStreamManager; +import com.bytedance.css.network.server.RpcHandler; +import com.bytedance.css.network.server.StreamManager; +import com.bytedance.css.network.server.TransportServer; +import com.google.common.collect.Sets; +import com.google.common.io.Files; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.buffer.NioManagedBuffer; +import com.bytedance.css.network.util.JavaUtils; +import com.bytedance.css.network.util.MapConfigProvider; +import com.bytedance.css.network.util.TransportConf; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.*; +import java.nio.ByteBuffer; +import java.util.*; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.*; + +public class RpcIntegrationSuite { + static TransportConf conf; + static TransportServer server; + static TransportClientFactory clientFactory; + static RpcHandler rpcHandler; + static List oneWayMsgs; + static StreamTestHelper testData; + + static ConcurrentHashMap streamCallbacks = + new ConcurrentHashMap<>(); + + @BeforeClass + public static void setUp() throws Exception { + conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); + testData = new StreamTestHelper(); + rpcHandler = new RpcHandler() { + @Override + public void receive( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { + String msg = JavaUtils.bytesToString(message); + String[] parts = msg.split("/"); + if (parts[0].equals("hello")) { + callback.onSuccess(JavaUtils.stringToBytes("Hello, " + parts[1] + "!")); + } else if (parts[0].equals("return error")) { + callback.onFailure(new RuntimeException("Returned: " + parts[1])); + } else if (parts[0].equals("throw error")) { + throw new RuntimeException("Thrown: " + parts[1]); + } + } + + @Override + public StreamCallbackWithID receiveStream( + TransportClient client, + ByteBuffer messageHeader, + RpcResponseCallback callback) { + return receiveStreamHelper(JavaUtils.bytesToString(messageHeader)); + } + + @Override + public void receive(TransportClient client, ByteBuffer message) { + oneWayMsgs.add(JavaUtils.bytesToString(message)); + } + + @Override + public StreamManager getStreamManager() { return new OneForOneStreamManager(); } + }; + TransportContext context = new TransportContext(conf, rpcHandler); + server = context.createServer(); + clientFactory = context.createClientFactory(); + oneWayMsgs = new ArrayList<>(); + } + + private static StreamCallbackWithID receiveStreamHelper(String msg) { + try { + if (msg.startsWith("fail/")) { + String[] parts = msg.split("/"); + switch (parts[1]) { + case "exception-ondata": + return new StreamCallbackWithID() { + @Override + public void onData(String streamId, ByteBuffer buf) throws IOException { + throw new IOException("failed to read stream data!"); + } + + @Override + public void onComplete(String streamId) throws IOException { + } + + @Override + public void onFailure(String streamId, Throwable cause) throws IOException { + } + + @Override + public String getID() { + return msg; + } + }; + case "exception-oncomplete": + return new StreamCallbackWithID() { + @Override + public void onData(String streamId, ByteBuffer buf) throws IOException { + } + + @Override + public void onComplete(String streamId) throws IOException { + throw new IOException("exception in onComplete"); + } + + @Override + public void onFailure(String streamId, Throwable cause) throws IOException { + } + + @Override + public String getID() { + return msg; + } + }; + case "null": + return null; + default: + throw new IllegalArgumentException("unexpected msg: " + msg); + } + } else { + VerifyingStreamCallback streamCallback = new VerifyingStreamCallback(msg); + streamCallbacks.put(msg, streamCallback); + return streamCallback; + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @AfterClass + public static void tearDown() { + server.close(); + clientFactory.close(); + testData.cleanup(); + } + + static class RpcResult { + public Set successMessages; + public Set errorMessages; + } + + private RpcResult sendRPC(String ... commands) throws Exception { + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + final Semaphore sem = new Semaphore(0); + + final RpcResult res = new RpcResult(); + res.successMessages = Collections.synchronizedSet(new HashSet()); + res.errorMessages = Collections.synchronizedSet(new HashSet()); + + RpcResponseCallback callback = new RpcResponseCallback() { + @Override + public void onSuccess(ByteBuffer message) { + String response = JavaUtils.bytesToString(message); + res.successMessages.add(response); + sem.release(); + } + + @Override + public void onFailure(Throwable e) { + res.errorMessages.add(e.getMessage()); + sem.release(); + } + }; + + for (String command : commands) { + client.sendRpc(JavaUtils.stringToBytes(command), callback); + } + + if (!sem.tryAcquire(commands.length, 5, TimeUnit.SECONDS)) { + fail("Timeout getting response from the server"); + } + client.close(); + return res; + } + + private RpcResult sendRpcWithStream(String... streams) throws Exception { + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + final Semaphore sem = new Semaphore(0); + RpcResult res = new RpcResult(); + res.successMessages = Collections.synchronizedSet(new HashSet()); + res.errorMessages = Collections.synchronizedSet(new HashSet()); + + for (String stream : streams) { + int idx = stream.lastIndexOf('/'); + ManagedBuffer meta = new NioManagedBuffer(JavaUtils.stringToBytes(stream)); + String streamName = (idx == -1) ? stream : stream.substring(idx + 1); + ManagedBuffer data = testData.openStream(conf, streamName); + client.uploadStream(meta, data, new RpcStreamCallback(stream, res, sem)); + } + + if (!sem.tryAcquire(streams.length, 5, TimeUnit.SECONDS)) { + fail("Timeout getting response from the server"); + } + streamCallbacks.values().forEach(streamCallback -> { + try { + streamCallback.verify(); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + client.close(); + return res; + } + + private static class RpcStreamCallback implements RpcResponseCallback { + final String streamId; + final RpcResult res; + final Semaphore sem; + + RpcStreamCallback(String streamId, RpcResult res, Semaphore sem) { + this.streamId = streamId; + this.res = res; + this.sem = sem; + } + + @Override + public void onSuccess(ByteBuffer message) { + res.successMessages.add(streamId); + sem.release(); + } + + @Override + public void onFailure(Throwable e) { + res.errorMessages.add(e.getMessage()); + sem.release(); + } + } + + @Test + public void singleRPC() throws Exception { + RpcResult res = sendRPC("hello/Aaron"); + assertEquals(res.successMessages, Sets.newHashSet("Hello, Aaron!")); + assertTrue(res.errorMessages.isEmpty()); + } + + @Test + public void doubleRPC() throws Exception { + RpcResult res = sendRPC("hello/Aaron", "hello/Reynold"); + assertEquals(res.successMessages, Sets.newHashSet("Hello, Aaron!", "Hello, Reynold!")); + assertTrue(res.errorMessages.isEmpty()); + } + + @Test + public void returnErrorRPC() throws Exception { + RpcResult res = sendRPC("return error/OK"); + assertTrue(res.successMessages.isEmpty()); + assertErrorsContain(res.errorMessages, Sets.newHashSet("Returned: OK")); + } + + @Test + public void throwErrorRPC() throws Exception { + RpcResult res = sendRPC("throw error/uh-oh"); + assertTrue(res.successMessages.isEmpty()); + assertErrorsContain(res.errorMessages, Sets.newHashSet("Thrown: uh-oh")); + } + + @Test + public void doubleTrouble() throws Exception { + RpcResult res = sendRPC("return error/OK", "throw error/uh-oh"); + assertTrue(res.successMessages.isEmpty()); + assertErrorsContain(res.errorMessages, Sets.newHashSet("Returned: OK", "Thrown: uh-oh")); + } + + @Test + public void sendSuccessAndFailure() throws Exception { + RpcResult res = sendRPC("hello/Bob", "throw error/the", "hello/Builder", "return error/!"); + assertEquals(res.successMessages, Sets.newHashSet("Hello, Bob!", "Hello, Builder!")); + assertErrorsContain(res.errorMessages, Sets.newHashSet("Thrown: the", "Returned: !")); + } + + @Test + public void sendOneWayMessage() throws Exception { + final String message = "no reply"; + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + try { + client.send(JavaUtils.stringToBytes(message)); + assertEquals(0, client.getHandler().numOutstandingRequests()); + + // Make sure the message arrives. + long deadline = System.nanoTime() + TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS); + while (System.nanoTime() < deadline && oneWayMsgs.size() == 0) { + TimeUnit.MILLISECONDS.sleep(10); + } + + assertEquals(1, oneWayMsgs.size()); + assertEquals(message, oneWayMsgs.get(0)); + } finally { + client.close(); + } + } + + @Test + public void sendRpcWithStreamOneAtATime() throws Exception { + for (String stream : StreamTestHelper.STREAMS) { + RpcResult res = sendRpcWithStream(stream); + assertTrue("there were error messages!" + res.errorMessages, res.errorMessages.isEmpty()); + assertEquals(Sets.newHashSet(stream), res.successMessages); + } + } + + @Test + public void sendRpcWithStreamConcurrently() throws Exception { + String[] streams = new String[10]; + for (int i = 0; i < 10; i++) { + streams[i] = StreamTestHelper.STREAMS[i % StreamTestHelper.STREAMS.length]; + } + RpcResult res = sendRpcWithStream(streams); + assertEquals(Sets.newHashSet(StreamTestHelper.STREAMS), res.successMessages); + assertTrue(res.errorMessages.isEmpty()); + } + + @Test + public void sendRpcWithStreamFailures() throws Exception { + // when there is a failure reading stream data, we don't try to keep the channel usable, + // just send back a decent error msg. + RpcResult exceptionInCallbackResult = + sendRpcWithStream("fail/exception-ondata/smallBuffer", "smallBuffer"); + assertErrorAndClosed(exceptionInCallbackResult, "Destination failed while reading stream"); + + RpcResult nullStreamHandler = + sendRpcWithStream("fail/null/smallBuffer", "smallBuffer"); + assertErrorAndClosed(exceptionInCallbackResult, "Destination failed while reading stream"); + + // OTOH, if there is a failure during onComplete, the channel should still be fine + RpcResult exceptionInOnComplete = + sendRpcWithStream("fail/exception-oncomplete/smallBuffer", "smallBuffer"); + assertErrorsContain(exceptionInOnComplete.errorMessages, + Sets.newHashSet("Failure post-processing")); + assertEquals(Sets.newHashSet("smallBuffer"), exceptionInOnComplete.successMessages); + } + + private void assertErrorsContain(Set errors, Set contains) { + assertEquals("Expected " + contains.size() + " errors, got " + errors.size() + "errors: " + + errors, contains.size(), errors.size()); + + Pair, Set> r = checkErrorsContain(errors, contains); + assertTrue("Could not find error containing " + r.getRight() + "; errors: " + errors, + r.getRight().isEmpty()); + + assertTrue(r.getLeft().isEmpty()); + } + + private void assertErrorAndClosed(RpcResult result, String expectedError) { + assertTrue("unexpected success: " + result.successMessages, result.successMessages.isEmpty()); + Set errors = result.errorMessages; + assertEquals("Expected 2 errors, got " + errors.size() + "errors: " + + errors, 2, errors.size()); + + // We expect 1 additional error due to closed connection and here are possible keywords in the + // error message. + Set possibleClosedErrors = Sets.newHashSet( + "closed", + "Connection reset", + "java.nio.channels.ClosedChannelException", + "java.io.IOException: Broken pipe" + ); + Set containsAndClosed = Sets.newHashSet(expectedError); + containsAndClosed.addAll(possibleClosedErrors); + + Pair, Set> r = checkErrorsContain(errors, containsAndClosed); + + assertTrue("Got a non-empty set " + r.getLeft(), r.getLeft().isEmpty()); + + Set errorsNotFound = r.getRight(); + assertEquals( + "The size of " + errorsNotFound + " was not " + (possibleClosedErrors.size() - 1), + possibleClosedErrors.size() - 1, + errorsNotFound.size()); + for (String err: errorsNotFound) { + assertTrue("Found a wrong error " + err, containsAndClosed.contains(err)); + } + } + + private Pair, Set> checkErrorsContain( + Set errors, + Set contains) { + Set remainingErrors = Sets.newHashSet(errors); + Set notFound = Sets.newHashSet(); + for (String contain : contains) { + Iterator it = remainingErrors.iterator(); + boolean foundMatch = false; + while (it.hasNext()) { + if (it.next().contains(contain)) { + it.remove(); + foundMatch = true; + break; + } + } + if (!foundMatch) { + notFound.add(contain); + } + } + return new ImmutablePair<>(remainingErrors, notFound); + } + + private static class VerifyingStreamCallback implements StreamCallbackWithID { + final String streamId; + final StreamSuite.TestCallback helper; + final OutputStream out; + final File outFile; + + VerifyingStreamCallback(String streamId) throws IOException { + if (streamId.equals("file")) { + outFile = File.createTempFile("data", ".tmp", testData.tempDir); + out = new FileOutputStream(outFile); + } else { + out = new ByteArrayOutputStream(); + outFile = null; + } + this.streamId = streamId; + helper = new StreamSuite.TestCallback(out); + } + + void verify() throws IOException { + if (streamId.equals("file")) { + assertTrue("File stream did not match.", Files.equal(testData.testFile, outFile)); + } else { + byte[] result = ((ByteArrayOutputStream)out).toByteArray(); + ByteBuffer srcBuffer = testData.srcBuffer(streamId); + ByteBuffer base; + synchronized (srcBuffer) { + base = srcBuffer.duplicate(); + } + byte[] expected = new byte[base.remaining()]; + base.get(expected); + assertEquals(expected.length, result.length); + assertTrue("buffers don't match", Arrays.equals(expected, result)); + } + } + + @Override + public void onData(String streamId, ByteBuffer buf) throws IOException { + helper.onData(streamId, buf); + } + + @Override + public void onComplete(String streamId) throws IOException { + helper.onComplete(streamId); + } + + @Override + public void onFailure(String streamId, Throwable cause) throws IOException { + helper.onFailure(streamId, cause); + } + + @Override + public String getID() { + return streamId; + } + } +} diff --git a/network-common/src/test/java/com/bytedance/css/network/StreamSuite.java b/network-common/src/test/java/com/bytedance/css/network/StreamSuite.java new file mode 100644 index 0000000..f0d0d44 --- /dev/null +++ b/network-common/src/test/java/com/bytedance/css/network/StreamSuite.java @@ -0,0 +1,309 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network; + +import com.google.common.io.Files; +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.client.RpcResponseCallback; +import com.bytedance.css.network.client.StreamCallback; +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.client.TransportClientFactory; +import com.bytedance.css.network.server.RpcHandler; +import com.bytedance.css.network.server.StreamManager; +import com.bytedance.css.network.server.TransportServer; +import com.bytedance.css.network.util.MapConfigProvider; +import com.bytedance.css.network.util.TransportConf; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.*; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +import static org.junit.Assert.*; + +public class StreamSuite { + private static final String[] STREAMS = StreamTestHelper.STREAMS; + private static StreamTestHelper testData; + + private static TransportServer server; + private static TransportClientFactory clientFactory; + + private static ByteBuffer createBuffer(int bufSize) { + ByteBuffer buf = ByteBuffer.allocate(bufSize); + for (int i = 0; i < bufSize; i ++) { + buf.put((byte) i); + } + buf.flip(); + return buf; + } + + @BeforeClass + public static void setUp() throws Exception { + testData = new StreamTestHelper(); + + final TransportConf conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); + final StreamManager streamManager = new StreamManager() { + @Override + public ManagedBuffer getChunk(long streamId, int chunkIndex) { + throw new UnsupportedOperationException(); + } + + @Override + public ManagedBuffer openStream(String streamId) { + return testData.openStream(conf, streamId); + } + }; + RpcHandler handler = new RpcHandler() { + @Override + public void receive( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { + throw new UnsupportedOperationException(); + } + + @Override + public StreamManager getStreamManager() { + return streamManager; + } + }; + TransportContext context = new TransportContext(conf, handler); + server = context.createServer(); + clientFactory = context.createClientFactory(); + } + + @AfterClass + public static void tearDown() { + server.close(); + clientFactory.close(); + testData.cleanup(); + } + + @Test + public void testZeroLengthStream() throws Throwable { + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + try { + StreamTask task = new StreamTask(client, "emptyBuffer", TimeUnit.SECONDS.toMillis(5)); + task.run(); + task.check(); + } finally { + client.close(); + } + } + + @Test + public void testSingleStream() throws Throwable { + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + try { + StreamTask task = new StreamTask(client, "largeBuffer", TimeUnit.SECONDS.toMillis(5)); + task.run(); + task.check(); + } finally { + client.close(); + } + } + + @Test + public void testMultipleStreams() throws Throwable { + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + try { + for (int i = 0; i < 20; i++) { + StreamTask task = new StreamTask(client, STREAMS[i % STREAMS.length], + TimeUnit.SECONDS.toMillis(5)); + task.run(); + task.check(); + } + } finally { + client.close(); + } + } + + @Test + public void testConcurrentStreams() throws Throwable { + ExecutorService executor = Executors.newFixedThreadPool(20); + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + + try { + List tasks = new ArrayList<>(); + for (int i = 0; i < 20; i++) { + StreamTask task = new StreamTask(client, STREAMS[i % STREAMS.length], + TimeUnit.SECONDS.toMillis(20)); + tasks.add(task); + executor.submit(task); + } + + executor.shutdown(); + assertTrue("Timed out waiting for tasks.", executor.awaitTermination(30, TimeUnit.SECONDS)); + for (StreamTask task : tasks) { + task.check(); + } + } finally { + executor.shutdownNow(); + client.close(); + } + } + + private static class StreamTask implements Runnable { + + private final TransportClient client; + private final String streamId; + private final long timeoutMs; + private Throwable error; + + StreamTask(TransportClient client, String streamId, long timeoutMs) { + this.client = client; + this.streamId = streamId; + this.timeoutMs = timeoutMs; + } + + @Override + public void run() { + ByteBuffer srcBuffer = null; + OutputStream out = null; + File outFile = null; + try { + ByteArrayOutputStream baos = null; + + switch (streamId) { + case "largeBuffer": + baos = new ByteArrayOutputStream(); + out = baos; + srcBuffer = testData.largeBuffer; + break; + case "smallBuffer": + baos = new ByteArrayOutputStream(); + out = baos; + srcBuffer = testData.smallBuffer; + break; + case "file": + outFile = File.createTempFile("data", ".tmp", testData.tempDir); + out = new FileOutputStream(outFile); + break; + case "emptyBuffer": + baos = new ByteArrayOutputStream(); + out = baos; + srcBuffer = testData.emptyBuffer; + break; + default: + throw new IllegalArgumentException(streamId); + } + + TestCallback callback = new TestCallback(out); + client.stream(streamId, callback); + callback.waitForCompletion(timeoutMs); + + if (srcBuffer == null) { + assertTrue("File stream did not match.", Files.equal(testData.testFile, outFile)); + } else { + ByteBuffer base; + synchronized (srcBuffer) { + base = srcBuffer.duplicate(); + } + byte[] result = baos.toByteArray(); + byte[] expected = new byte[base.remaining()]; + base.get(expected); + assertEquals(expected.length, result.length); + assertTrue("buffers don't match", Arrays.equals(expected, result)); + } + } catch (Throwable t) { + error = t; + } finally { + if (out != null) { + try { + out.close(); + } catch (Exception e) { + // ignore. + } + } + if (outFile != null) { + outFile.delete(); + } + } + } + + public void check() throws Throwable { + if (error != null) { + throw error; + } + } + } + + static class TestCallback implements StreamCallback { + + private final OutputStream out; + public volatile boolean completed; + public volatile Throwable error; + + TestCallback(OutputStream out) { + this.out = out; + this.completed = false; + } + + @Override + public void onData(String streamId, ByteBuffer buf) throws IOException { + byte[] tmp = new byte[buf.remaining()]; + buf.get(tmp); + out.write(tmp); + } + + @Override + public void onComplete(String streamId) throws IOException { + out.close(); + synchronized (this) { + completed = true; + notifyAll(); + } + } + + @Override + public void onFailure(String streamId, Throwable cause) { + error = cause; + synchronized (this) { + completed = true; + notifyAll(); + } + } + + void waitForCompletion(long timeoutMs) { + long now = System.currentTimeMillis(); + long deadline = now + timeoutMs; + synchronized (this) { + while (!completed && now < deadline) { + try { + wait(deadline - now); + } catch (InterruptedException ie) { + throw new RuntimeException(ie); + } + now = System.currentTimeMillis(); + } + } + assertTrue("Timed out waiting for stream.", completed); + assertNull(error); + } + } + +} diff --git a/network-common/src/test/java/com/bytedance/css/network/StreamTestHelper.java b/network-common/src/test/java/com/bytedance/css/network/StreamTestHelper.java new file mode 100644 index 0000000..347b52b --- /dev/null +++ b/network-common/src/test/java/com/bytedance/css/network/StreamTestHelper.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network; + +import com.google.common.io.Files; +import com.bytedance.css.network.buffer.FileSegmentManagedBuffer; +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.buffer.NioManagedBuffer; +import com.bytedance.css.network.util.JavaUtils; +import com.bytedance.css.network.util.TransportConf; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Random; + +class StreamTestHelper { + static final String[] STREAMS = { "largeBuffer", "smallBuffer", "emptyBuffer", "file" }; + + final File testFile; + final File tempDir; + + final ByteBuffer emptyBuffer; + final ByteBuffer smallBuffer; + final ByteBuffer largeBuffer; + + private static ByteBuffer createBuffer(int bufSize) { + ByteBuffer buf = ByteBuffer.allocate(bufSize); + for (int i = 0; i < bufSize; i ++) { + buf.put((byte) i); + } + buf.flip(); + return buf; + } + + StreamTestHelper() throws Exception { + tempDir = Files.createTempDir(); + emptyBuffer = createBuffer(0); + smallBuffer = createBuffer(100); + largeBuffer = createBuffer(100000); + + testFile = File.createTempFile("stream-test-file", "txt", tempDir); + FileOutputStream fp = new FileOutputStream(testFile); + try { + Random rnd = new Random(); + for (int i = 0; i < 512; i++) { + byte[] fileContent = new byte[1024]; + rnd.nextBytes(fileContent); + fp.write(fileContent); + } + } finally { + fp.close(); + } + } + + public ByteBuffer srcBuffer(String name) { + switch (name) { + case "largeBuffer": + return largeBuffer; + case "smallBuffer": + return smallBuffer; + case "emptyBuffer": + return emptyBuffer; + default: + throw new IllegalArgumentException("Invalid stream: " + name); + } + } + + public ManagedBuffer openStream(TransportConf conf, String streamId) { + switch (streamId) { + case "file": + return new FileSegmentManagedBuffer(conf, testFile, 0, testFile.length()); + default: + return new NioManagedBuffer(srcBuffer(streamId)); + } + } + + void cleanup() { + if (tempDir != null) { + try { + JavaUtils.deleteRecursively(tempDir); + } catch (IOException io) { + throw new RuntimeException(io); + } + } + } +} diff --git a/network-common/src/test/java/com/bytedance/css/network/TestManagedBuffer.java b/network-common/src/test/java/com/bytedance/css/network/TestManagedBuffer.java new file mode 100644 index 0000000..0cc2b9d --- /dev/null +++ b/network-common/src/test/java/com/bytedance/css/network/TestManagedBuffer.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network; + +import com.google.common.base.Preconditions; +import io.netty.buffer.Unpooled; +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.buffer.NettyManagedBuffer; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; + +/** + * A ManagedBuffer implementation that contains 0, 1, 2, 3, ..., (len-1). + * + * Used for testing. + */ +public class TestManagedBuffer extends ManagedBuffer { + + private final int len; + private NettyManagedBuffer underlying; + + public TestManagedBuffer(int len) { + Preconditions.checkArgument(len <= Byte.MAX_VALUE); + this.len = len; + byte[] byteArray = new byte[len]; + for (int i = 0; i < len; i ++) { + byteArray[i] = (byte) i; + } + this.underlying = new NettyManagedBuffer(Unpooled.wrappedBuffer(byteArray)); + } + + + @Override + public long size() { + return underlying.size(); + } + + @Override + public ByteBuffer nioByteBuffer() throws IOException { + return underlying.nioByteBuffer(); + } + + @Override + public InputStream createInputStream() throws IOException { + return underlying.createInputStream(); + } + + @Override + public ManagedBuffer retain() { + underlying.retain(); + return this; + } + + @Override + public ManagedBuffer release() { + underlying.release(); + return this; + } + + @Override + public Object convertToNetty() throws IOException { + return underlying.convertToNetty(); + } + + @Override + public int hashCode() { + return underlying.hashCode(); + } + + @Override + public boolean equals(Object other) { + if (other instanceof ManagedBuffer) { + try { + ByteBuffer nioBuf = ((ManagedBuffer) other).nioByteBuffer(); + if (nioBuf.remaining() != len) { + return false; + } else { + for (int i = 0; i < len; i ++) { + if (nioBuf.get() != i) { + return false; + } + } + return true; + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return false; + } +} diff --git a/network-common/src/test/java/com/bytedance/css/network/TestUtils.java b/network-common/src/test/java/com/bytedance/css/network/TestUtils.java new file mode 100644 index 0000000..1c3ba9b --- /dev/null +++ b/network-common/src/test/java/com/bytedance/css/network/TestUtils.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network; + +import java.net.InetAddress; + +public class TestUtils { + public static String getLocalHost() { + try { + return (System.getenv().containsKey("SPARK_LOCAL_IP"))? + System.getenv("SPARK_LOCAL_IP"): + InetAddress.getLocalHost().getHostAddress(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/network-common/src/test/java/com/bytedance/css/network/TransportRequestHandlerSuite.java b/network-common/src/test/java/com/bytedance/css/network/TransportRequestHandlerSuite.java new file mode 100644 index 0000000..761717a --- /dev/null +++ b/network-common/src/test/java/com/bytedance/css/network/TransportRequestHandlerSuite.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network; + +import com.bytedance.css.network.protocol.RequestMessage; +import com.bytedance.css.network.protocol.StreamFailure; +import com.bytedance.css.network.protocol.StreamRequest; +import com.bytedance.css.network.protocol.StreamResponse; +import io.netty.channel.Channel; +import io.netty.channel.ChannelPromise; +import io.netty.channel.DefaultChannelPromise; +import io.netty.util.concurrent.Future; +import io.netty.util.concurrent.GenericFutureListener; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.apache.commons.lang3.tuple.Pair; +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.server.*; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +import static org.mockito.Mockito.*; + +public class TransportRequestHandlerSuite { + + @Test + public void handleStreamRequest() throws Exception { + RpcHandler rpcHandler = new NoOpRpcHandler(); + OneForOneStreamManager streamManager = (OneForOneStreamManager) (rpcHandler.getStreamManager()); + Channel channel = mock(Channel.class); + List> responseAndPromisePairs = + new ArrayList<>(); + when(channel.writeAndFlush(any())) + .thenAnswer(invocationOnMock0 -> { + Object response = invocationOnMock0.getArguments()[0]; + ExtendedChannelPromise channelFuture = new ExtendedChannelPromise(channel); + responseAndPromisePairs.add(ImmutablePair.of(response, channelFuture)); + return channelFuture; + }); + + // Prepare the stream. + List managedBuffers = new ArrayList<>(); + managedBuffers.add(new TestManagedBuffer(10)); + managedBuffers.add(new TestManagedBuffer(20)); + managedBuffers.add(null); + managedBuffers.add(new TestManagedBuffer(30)); + managedBuffers.add(new TestManagedBuffer(40)); + long streamId = streamManager.registerStream("test-app", managedBuffers.iterator(), channel); + + Assert.assertEquals(1, streamManager.numStreamStates()); + + TransportClient reverseClient = mock(TransportClient.class); + TransportRequestHandler requestHandler = new TransportRequestHandler(channel, reverseClient, + rpcHandler, 2L, null); + + RequestMessage request0 = new StreamRequest(String.format("%d_%d", streamId, 0)); + requestHandler.handle(request0); + Assert.assertEquals(1, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(0).getLeft() instanceof StreamResponse); + Assert.assertEquals(managedBuffers.get(0), + ((StreamResponse) (responseAndPromisePairs.get(0).getLeft())).body()); + + RequestMessage request1 = new StreamRequest(String.format("%d_%d", streamId, 1)); + requestHandler.handle(request1); + Assert.assertEquals(2, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(1).getLeft() instanceof StreamResponse); + Assert.assertEquals(managedBuffers.get(1), + ((StreamResponse) (responseAndPromisePairs.get(1).getLeft())).body()); + + // Finish flushing the response for request0. + responseAndPromisePairs.get(0).getRight().finish(true); + + StreamRequest request2 = new StreamRequest(String.format("%d_%d", streamId, 2)); + requestHandler.handle(request2); + Assert.assertEquals(3, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(2).getLeft() instanceof StreamFailure); + Assert.assertEquals(String.format("Stream '%s' was not found.", request2.streamId), + ((StreamFailure) (responseAndPromisePairs.get(2).getLeft())).error); + + RequestMessage request3 = new StreamRequest(String.format("%d_%d", streamId, 3)); + requestHandler.handle(request3); + Assert.assertEquals(4, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(3).getLeft() instanceof StreamResponse); + Assert.assertEquals(managedBuffers.get(3), + ((StreamResponse) (responseAndPromisePairs.get(3).getLeft())).body()); + + // Request4 will trigger the close of channel, because the number of max chunks being + // transferred is 2; + RequestMessage request4 = new StreamRequest(String.format("%d_%d", streamId, 4)); + requestHandler.handle(request4); + verify(channel, times(1)).close(); + Assert.assertEquals(4, responseAndPromisePairs.size()); + + streamManager.connectionTerminated(channel); + Assert.assertEquals(0, streamManager.numStreamStates()); + } + + private class ExtendedChannelPromise extends DefaultChannelPromise { + + private List>> listeners = new ArrayList<>(); + private boolean success; + + ExtendedChannelPromise(Channel channel) { + super(channel); + success = false; + } + + @Override + public ChannelPromise addListener( + GenericFutureListener> listener) { + @SuppressWarnings("unchecked") + GenericFutureListener> gfListener = + (GenericFutureListener>) listener; + listeners.add(gfListener); + return super.addListener(listener); + } + + @Override + public boolean isSuccess() { + return success; + } + + public void finish(boolean success) { + this.success = success; + listeners.forEach(listener -> { + try { + listener.operationComplete(this); + } catch (Exception e) { + // do nothing + } + }); + } + } +} diff --git a/network-common/src/test/java/com/bytedance/css/network/TransportResponseHandlerSuite.java b/network-common/src/test/java/com/bytedance/css/network/TransportResponseHandlerSuite.java new file mode 100644 index 0000000..5fd20b1 --- /dev/null +++ b/network-common/src/test/java/com/bytedance/css/network/TransportResponseHandlerSuite.java @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network; + +import io.netty.channel.Channel; +import io.netty.channel.local.LocalChannel; +import com.bytedance.css.network.buffer.NioManagedBuffer; +import com.bytedance.css.network.client.ChunkReceivedCallback; +import com.bytedance.css.network.client.RpcResponseCallback; +import com.bytedance.css.network.client.StreamCallback; +import com.bytedance.css.network.client.TransportResponseHandler; +import com.bytedance.css.network.protocol.*; +import com.bytedance.css.network.util.TransportFrameDecoder; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import static org.mockito.Mockito.*; + +public class TransportResponseHandlerSuite { + @Test + public void handleSuccessfulFetch() throws Exception { + StreamChunkId streamChunkId = new StreamChunkId(1, 0); + + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + ChunkReceivedCallback callback = mock(ChunkReceivedCallback.class); + handler.addFetchRequest(streamChunkId, callback); + assertEquals(1, handler.numOutstandingRequests()); + + handler.handle(new ChunkFetchSuccess(streamChunkId, new TestManagedBuffer(123))); + verify(callback, times(1)).onSuccess(eq(0), any()); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void handleFailedFetch() throws Exception { + StreamChunkId streamChunkId = new StreamChunkId(1, 0); + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + ChunkReceivedCallback callback = mock(ChunkReceivedCallback.class); + handler.addFetchRequest(streamChunkId, callback); + assertEquals(1, handler.numOutstandingRequests()); + + handler.handle(new ChunkFetchFailure(streamChunkId, "some error msg")); + verify(callback, times(1)).onFailure(eq(0), any()); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void clearAllOutstandingRequests() throws Exception { + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + ChunkReceivedCallback callback = mock(ChunkReceivedCallback.class); + handler.addFetchRequest(new StreamChunkId(1, 0), callback); + handler.addFetchRequest(new StreamChunkId(1, 1), callback); + handler.addFetchRequest(new StreamChunkId(1, 2), callback); + assertEquals(3, handler.numOutstandingRequests()); + + handler.handle(new ChunkFetchSuccess(new StreamChunkId(1, 0), new TestManagedBuffer(12))); + handler.exceptionCaught(new Exception("duh duh duhhhh")); + + // should fail both b2 and b3 + verify(callback, times(1)).onSuccess(eq(0), any()); + verify(callback, times(1)).onFailure(eq(1), any()); + verify(callback, times(1)).onFailure(eq(2), any()); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void handleSuccessfulRPC() throws Exception { + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + RpcResponseCallback callback = mock(RpcResponseCallback.class); + handler.addRpcRequest(12345, callback); + assertEquals(1, handler.numOutstandingRequests()); + + // This response should be ignored. + handler.handle(new RpcResponse(54321, new NioManagedBuffer(ByteBuffer.allocate(7)))); + assertEquals(1, handler.numOutstandingRequests()); + + ByteBuffer resp = ByteBuffer.allocate(10); + handler.handle(new RpcResponse(12345, new NioManagedBuffer(resp))); + verify(callback, times(1)).onSuccess(eq(ByteBuffer.allocate(10))); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void handleFailedRPC() throws Exception { + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + RpcResponseCallback callback = mock(RpcResponseCallback.class); + handler.addRpcRequest(12345, callback); + assertEquals(1, handler.numOutstandingRequests()); + + handler.handle(new RpcFailure(54321, "uh-oh!")); // should be ignored + assertEquals(1, handler.numOutstandingRequests()); + + handler.handle(new RpcFailure(12345, "oh no")); + verify(callback, times(1)).onFailure(any()); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void testActiveStreams() throws Exception { + Channel c = new LocalChannel(); + c.pipeline().addLast(TransportFrameDecoder.HANDLER_NAME, new TransportFrameDecoder()); + TransportResponseHandler handler = new TransportResponseHandler(c); + + StreamResponse response = new StreamResponse("stream", 1234L, null); + StreamCallback cb = mock(StreamCallback.class); + handler.addStreamCallback("stream", cb); + assertEquals(1, handler.numOutstandingRequests()); + handler.handle(response); + assertEquals(1, handler.numOutstandingRequests()); + handler.deactivateStream(); + assertEquals(0, handler.numOutstandingRequests()); + + StreamFailure failure = new StreamFailure("stream", "uh-oh"); + handler.addStreamCallback("stream", cb); + assertEquals(1, handler.numOutstandingRequests()); + handler.handle(failure); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void failOutstandingStreamCallbackOnClose() throws Exception { + Channel c = new LocalChannel(); + c.pipeline().addLast(TransportFrameDecoder.HANDLER_NAME, new TransportFrameDecoder()); + TransportResponseHandler handler = new TransportResponseHandler(c); + + StreamCallback cb = mock(StreamCallback.class); + handler.addStreamCallback("stream-1", cb); + handler.channelInactive(); + + verify(cb).onFailure(eq("stream-1"), isA(IOException.class)); + } + + @Test + public void failOutstandingStreamCallbackOnException() throws Exception { + Channel c = new LocalChannel(); + c.pipeline().addLast(TransportFrameDecoder.HANDLER_NAME, new TransportFrameDecoder()); + TransportResponseHandler handler = new TransportResponseHandler(c); + + StreamCallback cb = mock(StreamCallback.class); + handler.addStreamCallback("stream-1", cb); + handler.exceptionCaught(new IOException("Oops!")); + + verify(cb).onFailure(eq("stream-1"), isA(IOException.class)); + } +} diff --git a/network-common/src/test/java/com/bytedance/css/network/client/TransportClientFactorySuite.java b/network-common/src/test/java/com/bytedance/css/network/client/TransportClientFactorySuite.java new file mode 100644 index 0000000..bde8668 --- /dev/null +++ b/network-common/src/test/java/com/bytedance/css/network/client/TransportClientFactorySuite.java @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.client; + +import com.bytedance.css.network.TestUtils; +import com.bytedance.css.network.TransportContext; +import com.bytedance.css.network.server.NoOpRpcHandler; +import com.bytedance.css.network.server.RpcHandler; +import com.bytedance.css.network.server.TransportServer; +import com.bytedance.css.network.util.ConfigProvider; +import com.bytedance.css.network.util.JavaUtils; +import com.bytedance.css.network.util.MapConfigProvider; +import com.bytedance.css.network.util.TransportConf; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.*; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.junit.Assert.*; + +public class TransportClientFactorySuite { + private TransportConf conf; + private TransportContext context; + private TransportServer server1; + private TransportServer server2; + + @Before + public void setUp() { + conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); + RpcHandler rpcHandler = new NoOpRpcHandler(); + context = new TransportContext(conf, rpcHandler); + server1 = context.createServer(); + server2 = context.createServer(); + } + + @After + public void tearDown() { + JavaUtils.closeQuietly(server1); + JavaUtils.closeQuietly(server2); + } + + /** + * Request a bunch of clients to a single server to test + * we create up to maxConnections of clients. + * + * If concurrent is true, create multiple threads to create clients in parallel. + */ + private void testClientReuse(int maxConnections, boolean concurrent) + throws IOException, InterruptedException { + + Map configMap = new HashMap<>(); + configMap.put("css.shuffle.io.numConnectionsPerPeer", Integer.toString(maxConnections)); + TransportConf conf = new TransportConf("shuffle", new MapConfigProvider(configMap)); + + RpcHandler rpcHandler = new NoOpRpcHandler(); + TransportContext context = new TransportContext(conf, rpcHandler); + TransportClientFactory factory = context.createClientFactory(); + Set clients = Collections.synchronizedSet( + new HashSet()); + + AtomicInteger failed = new AtomicInteger(); + Thread[] attempts = new Thread[maxConnections * 10]; + + // Launch a bunch of threads to create new clients. + for (int i = 0; i < attempts.length; i++) { + attempts[i] = new Thread(() -> { + try { + TransportClient client = + factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + assertTrue(client.isActive()); + clients.add(client); + } catch (IOException e) { + failed.incrementAndGet(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }); + + if (concurrent) { + attempts[i].start(); + } else { + attempts[i].run(); + } + } + + // Wait until all the threads complete. + for (Thread attempt : attempts) { + attempt.join(); + } + + Assert.assertEquals(0, failed.get()); + Assert.assertTrue(clients.size() <= maxConnections); + + for (TransportClient client : clients) { + client.close(); + } + + factory.close(); + } + + @Test + public void reuseClientsUpToConfigVariable() throws Exception { + testClientReuse(1, false); + testClientReuse(2, false); + testClientReuse(3, false); + testClientReuse(4, false); + } + + @Test + public void reuseClientsUpToConfigVariableConcurrent() throws Exception { + testClientReuse(1, true); + testClientReuse(2, true); + testClientReuse(3, true); + testClientReuse(4, true); + } + + @Test + public void returnDifferentClientsForDifferentServers() throws IOException, InterruptedException { + TransportClientFactory factory = context.createClientFactory(); + TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server2.getPort()); + assertTrue(c1.isActive()); + assertTrue(c2.isActive()); + assertNotSame(c1, c2); + factory.close(); + } + + @Test + public void neverReturnInactiveClients() throws IOException, InterruptedException { + TransportClientFactory factory = context.createClientFactory(); + TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + c1.close(); + + long start = System.currentTimeMillis(); + while (c1.isActive() && (System.currentTimeMillis() - start) < 3000) { + Thread.sleep(10); + } + assertFalse(c1.isActive()); + + TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + assertNotSame(c1, c2); + assertTrue(c2.isActive()); + factory.close(); + } + + @Test + public void closeBlockClientsWithFactory() throws IOException, InterruptedException { + TransportClientFactory factory = context.createClientFactory(); + TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server2.getPort()); + assertTrue(c1.isActive()); + assertTrue(c2.isActive()); + factory.close(); + assertFalse(c1.isActive()); + assertFalse(c2.isActive()); + } + + @Test + public void closeIdleConnectionForRequestTimeOut() throws IOException, InterruptedException { + TransportConf conf = new TransportConf("shuffle", new ConfigProvider() { + + @Override + public String get(String name) { + if ("css.shuffle.io.connectionTimeout".equals(name)) { + // We should make sure there is enough time for us to observe the channel is active + return "1s"; + } + String value = System.getProperty(name); + if (value == null) { + throw new NoSuchElementException(name); + } + return value; + } + + @Override + public Iterable> getAll() { + throw new UnsupportedOperationException(); + } + }); + TransportContext context = new TransportContext(conf, new NoOpRpcHandler(), true); + try (TransportClientFactory factory = context.createClientFactory()) { + TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + assertTrue(c1.isActive()); + long expiredTime = System.currentTimeMillis() + 10000; // 10 seconds + while (c1.isActive() && System.currentTimeMillis() < expiredTime) { + Thread.sleep(10); + } + assertFalse(c1.isActive()); + } + } + + @Test(expected = IOException.class) + public void closeFactoryBeforeCreateClient() throws IOException, InterruptedException { + TransportClientFactory factory = context.createClientFactory(); + factory.close(); + factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + } +} diff --git a/network-common/src/test/java/com/bytedance/css/network/protocol/MessageWithHeaderSuite.java b/network-common/src/test/java/com/bytedance/css/network/protocol/MessageWithHeaderSuite.java new file mode 100644 index 0000000..06a2857 --- /dev/null +++ b/network-common/src/test/java/com/bytedance/css/network/protocol/MessageWithHeaderSuite.java @@ -0,0 +1,188 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.protocol; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.CompositeByteBuf; +import io.netty.buffer.Unpooled; +import com.bytedance.css.network.TestManagedBuffer; +import com.bytedance.css.network.buffer.ManagedBuffer; +import com.bytedance.css.network.buffer.NettyManagedBuffer; +import com.bytedance.css.network.util.AbstractFileRegion; +import com.bytedance.css.network.util.ByteArrayWritableChannel; +import org.junit.Test; +import org.mockito.Mockito; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class MessageWithHeaderSuite { + + @Test + public void testSingleWrite() throws Exception { + testFileRegionBody(8, 8); + } + + @Test + public void testShortWrite() throws Exception { + testFileRegionBody(8, 1); + } + + @Test + public void testByteBufBody() throws Exception { + testByteBufBody(Unpooled.copyLong(42)); + } + + @Test + public void testCompositeByteBufBodySingleBuffer() throws Exception { + ByteBuf header = Unpooled.copyLong(42); + CompositeByteBuf compositeByteBuf = Unpooled.compositeBuffer(); + compositeByteBuf.addComponent(true, header); + assertEquals(1, compositeByteBuf.nioBufferCount()); + testByteBufBody(compositeByteBuf); + } + + @Test + public void testCompositeByteBufBodyMultipleBuffers() throws Exception { + ByteBuf header = Unpooled.copyLong(42); + CompositeByteBuf compositeByteBuf = Unpooled.compositeBuffer(); + compositeByteBuf.addComponent(true, header.retainedSlice(0, 4)); + compositeByteBuf.addComponent(true, header.slice(4, 4)); + assertEquals(2, compositeByteBuf.nioBufferCount()); + testByteBufBody(compositeByteBuf); + } + + /** + * Test writing a {@link MessageWithHeader} using the given {@link ByteBuf} as header. + * + * @param header the header to use. + * @throws Exception thrown on error. + */ + private void testByteBufBody(ByteBuf header) throws Exception { + long expectedHeaderValue = header.getLong(header.readerIndex()); + ByteBuf bodyPassedToNettyManagedBuffer = Unpooled.copyLong(84); + assertEquals(1, header.refCnt()); + assertEquals(1, bodyPassedToNettyManagedBuffer.refCnt()); + ManagedBuffer managedBuf = new NettyManagedBuffer(bodyPassedToNettyManagedBuffer); + + Object body = managedBuf.convertToNetty(); + assertEquals(2, bodyPassedToNettyManagedBuffer.refCnt()); + assertEquals(1, header.refCnt()); + + MessageWithHeader msg = new MessageWithHeader(managedBuf, header, body, managedBuf.size()); + ByteBuf result = doWrite(msg, 1); + assertEquals(msg.count(), result.readableBytes()); + assertEquals(expectedHeaderValue, result.readLong()); + assertEquals(84, result.readLong()); + + assertTrue(msg.release()); + assertEquals(0, bodyPassedToNettyManagedBuffer.refCnt()); + assertEquals(0, header.refCnt()); + } + + @Test + public void testDeallocateReleasesManagedBuffer() throws Exception { + ByteBuf header = Unpooled.copyLong(42); + ManagedBuffer managedBuf = Mockito.spy(new TestManagedBuffer(84)); + ByteBuf body = (ByteBuf) managedBuf.convertToNetty(); + assertEquals(2, body.refCnt()); + MessageWithHeader msg = new MessageWithHeader(managedBuf, header, body, body.readableBytes()); + assertTrue(msg.release()); + Mockito.verify(managedBuf, Mockito.times(1)).release(); + assertEquals(0, body.refCnt()); + } + + private void testFileRegionBody(int totalWrites, int writesPerCall) throws Exception { + ByteBuf header = Unpooled.copyLong(42); + int headerLength = header.readableBytes(); + TestFileRegion region = new TestFileRegion(totalWrites, writesPerCall); + MessageWithHeader msg = new MessageWithHeader(null, header, region, region.count()); + + ByteBuf result = doWrite(msg, totalWrites / writesPerCall); + assertEquals(headerLength + region.count(), result.readableBytes()); + assertEquals(42, result.readLong()); + for (long i = 0; i < 8; i++) { + assertEquals(i, result.readLong()); + } + assertTrue(msg.release()); + } + + private ByteBuf doWrite(MessageWithHeader msg, int minExpectedWrites) throws Exception { + int writes = 0; + ByteArrayWritableChannel channel = new ByteArrayWritableChannel((int) msg.count()); + while (msg.transfered() < msg.count()) { + msg.transferTo(channel, msg.transfered()); + writes++; + } + assertTrue("Not enough writes!", minExpectedWrites <= writes); + return Unpooled.wrappedBuffer(channel.getData()); + } + + private static class TestFileRegion extends AbstractFileRegion { + + private final int writeCount; + private final int writesPerCall; + private int written; + + TestFileRegion(int totalWrites, int writesPerCall) { + this.writeCount = totalWrites; + this.writesPerCall = writesPerCall; + } + + @Override + public long count() { + return 8 * writeCount; + } + + @Override + public long position() { + return 0; + } + + @Override + public long transferred() { + return 8 * written; + } + + @Override + public long transferTo(WritableByteChannel target, long position) throws IOException { + for (int i = 0; i < writesPerCall; i++) { + ByteBuf buf = Unpooled.copyLong((position / 8) + i); + ByteBuffer nio = buf.nioBuffer(); + while (nio.remaining() > 0) { + target.write(nio); + } + buf.release(); + written++; + } + return 8 * writesPerCall; + } + + @Override + protected void deallocate() { + } + + } + +} diff --git a/network-common/src/test/java/com/bytedance/css/network/server/OneForOneStreamManagerSuite.java b/network-common/src/test/java/com/bytedance/css/network/server/OneForOneStreamManagerSuite.java new file mode 100644 index 0000000..6345676 --- /dev/null +++ b/network-common/src/test/java/com/bytedance/css/network/server/OneForOneStreamManagerSuite.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.server; + +import io.netty.channel.Channel; +import com.bytedance.css.network.TestManagedBuffer; +import com.bytedance.css.network.buffer.ManagedBuffer; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; +import org.mockito.Mockito; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +public class OneForOneStreamManagerSuite { + + List managedBuffersToRelease = new ArrayList<>(); + + @After + public void tearDown() { + managedBuffersToRelease.forEach(managedBuffer -> managedBuffer.release()); + managedBuffersToRelease.clear(); + } + + private ManagedBuffer getChunk(OneForOneStreamManager manager, long streamId, int chunkIndex) { + ManagedBuffer chunk = manager.getChunk(streamId, chunkIndex); + if (chunk != null) { + managedBuffersToRelease.add(chunk); + } + return chunk; + } + + @Test + public void testMissingChunk() { + OneForOneStreamManager manager = new OneForOneStreamManager(); + List buffers = new ArrayList<>(); + TestManagedBuffer buffer1 = Mockito.spy(new TestManagedBuffer(10)); + TestManagedBuffer buffer2 = Mockito.spy(new TestManagedBuffer(20)); + TestManagedBuffer buffer3 = Mockito.spy(new TestManagedBuffer(20)); + + buffers.add(buffer1); + // the nulls here are to simulate a file which goes missing before being read, + // just as a defensive measure + buffers.add(null); + buffers.add(buffer2); + buffers.add(null); + buffers.add(buffer3); + + Channel dummyChannel = Mockito.mock(Channel.class, Mockito.RETURNS_SMART_NULLS); + long streamId = manager.registerStream("appId", buffers.iterator(), dummyChannel); + Assert.assertEquals(1, manager.numStreamStates()); + Assert.assertNotNull(getChunk(manager, streamId, 0)); + Assert.assertNull(getChunk(manager, streamId, 1)); + Assert.assertNotNull(getChunk(manager, streamId, 2)); + manager.connectionTerminated(dummyChannel); + + // loaded buffers are not released yet as in production a ManagedBuffer returned by getChunk() + // would only be released by Netty after it is written to the network + Mockito.verify(buffer1, Mockito.never()).release(); + Mockito.verify(buffer2, Mockito.never()).release(); + Mockito.verify(buffer3, Mockito.times(1)).release(); + } + + @Test + public void managedBuffersAreFeedWhenConnectionIsClosed() throws Exception { + OneForOneStreamManager manager = new OneForOneStreamManager(); + List buffers = new ArrayList<>(); + TestManagedBuffer buffer1 = Mockito.spy(new TestManagedBuffer(10)); + TestManagedBuffer buffer2 = Mockito.spy(new TestManagedBuffer(20)); + buffers.add(buffer1); + buffers.add(buffer2); + + Channel dummyChannel = Mockito.mock(Channel.class, Mockito.RETURNS_SMART_NULLS); + manager.registerStream("appId", buffers.iterator(), dummyChannel); + assert manager.numStreamStates() == 1; + + manager.connectionTerminated(dummyChannel); + + Mockito.verify(buffer1, Mockito.times(1)).release(); + Mockito.verify(buffer2, Mockito.times(1)).release(); + assert manager.numStreamStates() == 0; + } + + @Test + public void streamStatesAreFreedWhenConnectionIsClosedEvenIfBufferIteratorThrowsException() { + OneForOneStreamManager manager = new OneForOneStreamManager(); + + @SuppressWarnings("unchecked") + Iterator buffers = Mockito.mock(Iterator.class); + Mockito.when(buffers.hasNext()).thenReturn(true); + Mockito.when(buffers.next()).thenThrow(RuntimeException.class); + + ManagedBuffer mockManagedBuffer = Mockito.mock(ManagedBuffer.class); + + @SuppressWarnings("unchecked") + Iterator buffers2 = Mockito.mock(Iterator.class); + Mockito.when(buffers2.hasNext()).thenReturn(true).thenReturn(true); + Mockito.when(buffers2.next()).thenReturn(mockManagedBuffer).thenThrow(RuntimeException.class); + + Channel dummyChannel = Mockito.mock(Channel.class, Mockito.RETURNS_SMART_NULLS); + manager.registerStream("appId", buffers, dummyChannel); + manager.registerStream("appId", buffers2, dummyChannel); + + Assert.assertEquals(2, manager.numStreamStates()); + + try { + manager.connectionTerminated(dummyChannel); + Assert.fail("connectionTerminated should throw exception when fails to release all buffers"); + + } catch (RuntimeException e) { + + Mockito.verify(buffers, Mockito.times(1)).hasNext(); + Mockito.verify(buffers, Mockito.times(1)).next(); + + Mockito.verify(buffers2, Mockito.times(2)).hasNext(); + Mockito.verify(buffers2, Mockito.times(2)).next(); + + Mockito.verify(mockManagedBuffer, Mockito.times(1)).release(); + + Assert.assertEquals(0, manager.numStreamStates()); + } + } +} diff --git a/network-common/src/test/java/com/bytedance/css/network/util/NettyMemoryMetricsSuite.java b/network-common/src/test/java/com/bytedance/css/network/util/NettyMemoryMetricsSuite.java new file mode 100644 index 0000000..4129e71 --- /dev/null +++ b/network-common/src/test/java/com/bytedance/css/network/util/NettyMemoryMetricsSuite.java @@ -0,0 +1,173 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.util; + +import com.codahale.metrics.Gauge; +import com.codahale.metrics.Metric; +import com.codahale.metrics.MetricRegistry; +import com.codahale.metrics.MetricSet; +import com.bytedance.css.network.TestUtils; +import com.bytedance.css.network.TransportContext; +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.client.TransportClientFactory; +import com.bytedance.css.network.server.NoOpRpcHandler; +import com.bytedance.css.network.server.RpcHandler; +import com.bytedance.css.network.server.TransportServer; +import io.netty.buffer.NettyMemoryMetrics; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class NettyMemoryMetricsSuite { + + private TransportConf conf; + private TransportContext context; + private TransportServer server; + private TransportClientFactory clientFactory; + + private void setUp(boolean enableVerboseMetrics) { + HashMap configMap = new HashMap<>(); + configMap.put("css.shuffle.io.enableVerboseMetrics", String.valueOf(enableVerboseMetrics)); + conf = new TransportConf("shuffle", new MapConfigProvider(configMap)); + RpcHandler rpcHandler = new NoOpRpcHandler(); + context = new TransportContext(conf, rpcHandler); + server = context.createServer(); + clientFactory = context.createClientFactory(); + } + + @After + public void tearDown() { + if (clientFactory != null) { + JavaUtils.closeQuietly(clientFactory); + clientFactory = null; + } + if (server != null) { + JavaUtils.closeQuietly(server); + server = null; + } + if (context != null) { + JavaUtils.closeQuietly(context); + context = null; + } + } + + @Test + @SuppressWarnings("unchecked") + public void testGeneralNettyMemoryMetrics() throws IOException, InterruptedException { + setUp(false); + + MetricSet serverMetrics = server.getAllMetrics(); + Assert.assertNotNull(serverMetrics); + Assert.assertNotNull(serverMetrics.getMetrics()); + Assert.assertNotEquals(serverMetrics.getMetrics().size(), 0); + + Map serverMetricMap = serverMetrics.getMetrics(); + serverMetricMap.forEach((name, metric) -> + Assert.assertTrue(name.startsWith("shuffle-server")) + ); + + MetricSet clientMetrics = clientFactory.getAllMetrics(); + Assert.assertNotNull(clientMetrics); + Assert.assertNotNull(clientMetrics.getMetrics()); + Assert.assertNotEquals(clientMetrics.getMetrics().size(), 0); + + Map clientMetricMap = clientMetrics.getMetrics(); + clientMetricMap.forEach((name, metrics) -> + Assert.assertTrue(name.startsWith("shuffle-client")) + ); + + // Make sure general metrics existed. + String heapMemoryMetric = "usedHeapMemory"; + String directMemoryMetric = "usedDirectMemory"; + Assert.assertNotNull(serverMetricMap.get( + MetricRegistry.name("shuffle-server", heapMemoryMetric))); + Assert.assertNotNull(serverMetricMap.get( + MetricRegistry.name("shuffle-server", directMemoryMetric))); + + Assert.assertNotNull(clientMetricMap.get( + MetricRegistry.name("shuffle-client", heapMemoryMetric))); + Assert.assertNotNull(clientMetricMap.get( + MetricRegistry.name("shuffle-client", directMemoryMetric))); + + try (TransportClient client = + clientFactory.createClient(TestUtils.getLocalHost(), server.getPort())) { + Assert.assertTrue(client.isActive()); + + Assert.assertTrue(((Gauge)serverMetricMap.get( + MetricRegistry.name("shuffle-server", heapMemoryMetric))).getValue() >= 0L); + Assert.assertTrue(((Gauge)serverMetricMap.get( + MetricRegistry.name("shuffle-server", directMemoryMetric))).getValue() >= 0L); + + Assert.assertTrue(((Gauge)clientMetricMap.get( + MetricRegistry.name("shuffle-client", heapMemoryMetric))).getValue() >= 0L); + Assert.assertTrue(((Gauge)clientMetricMap.get( + MetricRegistry.name("shuffle-client", directMemoryMetric))).getValue() >= 0L); + + } + } + + @Test + @SuppressWarnings("unchecked") + public void testAdditionalMetrics() throws IOException, InterruptedException { + setUp(true); + + // Make sure additional metrics are added. + Map serverMetricMap = server.getAllMetrics().getMetrics(); + serverMetricMap.forEach((name, metric) -> { + Assert.assertTrue(name.startsWith("shuffle-server")); + String metricName = name.substring(name.lastIndexOf(".") + 1); + Assert.assertTrue(metricName.equals("usedDirectMemory") + || metricName.equals("realDirectMemory") + || metricName.equals("usedHeapMemory") + || NettyMemoryMetrics.VERBOSE_METRICS.contains(metricName)); + }); + + Map clientMetricMap = clientFactory.getAllMetrics().getMetrics(); + clientMetricMap.forEach((name, metric) -> { + Assert.assertTrue(name.startsWith("shuffle-client")); + String metricName = name.substring(name.lastIndexOf(".") + 1); + Assert.assertTrue(metricName.equals("usedDirectMemory") + || metricName.equals("realDirectMemory") + || metricName.equals("usedHeapMemory") + || NettyMemoryMetrics.VERBOSE_METRICS.contains(metricName)); + }); + + TransportClient client = null; + try { + client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + Assert.assertTrue(client.isActive()); + + String activeBytesMetric = "numActiveBytes"; + Assert.assertTrue(((Gauge) serverMetricMap.get(MetricRegistry.name("shuffle-server", + "directArena0", activeBytesMetric))).getValue() >= 0L); + + Assert.assertTrue(((Gauge) clientMetricMap.get(MetricRegistry.name("shuffle-client", + "directArena0", activeBytesMetric))).getValue() >= 0L); + } finally { + if (client != null) { + client.close(); + } + } + } +} diff --git a/network-common/src/test/java/com/bytedance/css/network/util/TransportFrameDecoderSuite.java b/network-common/src/test/java/com/bytedance/css/network/util/TransportFrameDecoderSuite.java new file mode 100644 index 0000000..1a95fed --- /dev/null +++ b/network-common/src/test/java/com/bytedance/css/network/util/TransportFrameDecoderSuite.java @@ -0,0 +1,313 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.network.util; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import org.junit.AfterClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +public class TransportFrameDecoderSuite { + + private static final Logger logger = LoggerFactory.getLogger(TransportFrameDecoderSuite.class); + private static Random RND = new Random(); + + @AfterClass + public static void cleanup() { + RND = null; + } + + @Test + public void testFrameDecoding() throws Exception { + TransportFrameDecoder decoder = new TransportFrameDecoder(); + ChannelHandlerContext ctx = mockChannelHandlerContext(); + ByteBuf data = createAndFeedFrames(100, decoder, ctx); + verifyAndCloseDecoder(decoder, ctx, data); + } + + @Test + public void testConsolidationPerf() throws Exception { + long[] testingConsolidateThresholds = new long[] { + ByteUnit.MiB.toBytes(1), + ByteUnit.MiB.toBytes(5), + ByteUnit.MiB.toBytes(10), + ByteUnit.MiB.toBytes(20), + ByteUnit.MiB.toBytes(30), + ByteUnit.MiB.toBytes(50), + ByteUnit.MiB.toBytes(80), + ByteUnit.MiB.toBytes(100), + ByteUnit.MiB.toBytes(300), + ByteUnit.MiB.toBytes(500), + Long.MAX_VALUE + }; + for (long threshold : testingConsolidateThresholds) { + TransportFrameDecoder decoder = new TransportFrameDecoder(threshold); + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + List retained = new ArrayList<>(); + when(ctx.fireChannelRead(any())).thenAnswer(in -> { + ByteBuf buf = (ByteBuf) in.getArguments()[0]; + retained.add(buf); + return null; + }); + + // Testing multiple messages + int numMessages = 3; + long targetBytes = ByteUnit.MiB.toBytes(300); + int pieceBytes = (int) ByteUnit.KiB.toBytes(32); + for (int i = 0; i < numMessages; i++) { + try { + long writtenBytes = 0; + long totalTime = 0; + ByteBuf buf = Unpooled.buffer(8); + buf.writeLong(8 + targetBytes); + decoder.channelRead(ctx, buf); + while (writtenBytes < targetBytes) { + buf = Unpooled.buffer(pieceBytes * 2); + ByteBuf writtenBuf = Unpooled.buffer(pieceBytes).writerIndex(pieceBytes); + buf.writeBytes(writtenBuf); + writtenBuf.release(); + long start = System.currentTimeMillis(); + decoder.channelRead(ctx, buf); + long elapsedTime = System.currentTimeMillis() - start; + totalTime += elapsedTime; + writtenBytes += pieceBytes; + } + logger.info("Writing 300MiB frame buf with consolidation of threshold " + threshold + + " took " + totalTime + " millis"); + } finally { + for (ByteBuf buf : retained) { + release(buf); + } + } + } + long totalBytesGot = 0; + for (ByteBuf buf : retained) { + totalBytesGot += buf.capacity(); + } + assertEquals(numMessages, retained.size()); + assertEquals(targetBytes * numMessages, totalBytesGot); + } + } + + @Test + public void testInterception() throws Exception { + int interceptedReads = 3; + TransportFrameDecoder decoder = new TransportFrameDecoder(); + TransportFrameDecoder.Interceptor interceptor = spy(new MockInterceptor(interceptedReads)); + ChannelHandlerContext ctx = mockChannelHandlerContext(); + + byte[] data = new byte[8]; + ByteBuf len = Unpooled.copyLong(8 + data.length); + ByteBuf dataBuf = Unpooled.wrappedBuffer(data); + + try { + decoder.setInterceptor(interceptor); + for (int i = 0; i < interceptedReads; i++) { + decoder.channelRead(ctx, dataBuf); + assertEquals(0, dataBuf.refCnt()); + dataBuf = Unpooled.wrappedBuffer(data); + } + decoder.channelRead(ctx, len); + decoder.channelRead(ctx, dataBuf); + verify(interceptor, times(interceptedReads)).handle(any(ByteBuf.class)); + verify(ctx).fireChannelRead(any(ByteBuf.class)); + assertEquals(0, len.refCnt()); + assertEquals(0, dataBuf.refCnt()); + } finally { + release(len); + release(dataBuf); + } + } + + @Test + public void testRetainedFrames() throws Exception { + TransportFrameDecoder decoder = new TransportFrameDecoder(); + + AtomicInteger count = new AtomicInteger(); + List retained = new ArrayList<>(); + + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.fireChannelRead(any())).thenAnswer(in -> { + // Retain a few frames but not others. + ByteBuf buf = (ByteBuf) in.getArguments()[0]; + if (count.incrementAndGet() % 2 == 0) { + retained.add(buf); + } else { + buf.release(); + } + return null; + }); + + ByteBuf data = createAndFeedFrames(100, decoder, ctx); + try { + // Verify all retained buffers are readable. + for (ByteBuf b : retained) { + byte[] tmp = new byte[b.readableBytes()]; + b.readBytes(tmp); + b.release(); + } + verifyAndCloseDecoder(decoder, ctx, data); + } finally { + for (ByteBuf b : retained) { + release(b); + } + } + } + + @Test + public void testSplitLengthField() throws Exception { + byte[] frame = new byte[1024 * (RND.nextInt(31) + 1)]; + ByteBuf buf = Unpooled.buffer(frame.length + 8); + buf.writeLong(frame.length + 8); + buf.writeBytes(frame); + + TransportFrameDecoder decoder = new TransportFrameDecoder(); + ChannelHandlerContext ctx = mockChannelHandlerContext(); + try { + decoder.channelRead(ctx, buf.readSlice(RND.nextInt(7)).retain()); + verify(ctx, never()).fireChannelRead(any(ByteBuf.class)); + decoder.channelRead(ctx, buf); + verify(ctx).fireChannelRead(any(ByteBuf.class)); + assertEquals(0, buf.refCnt()); + } finally { + decoder.channelInactive(ctx); + release(buf); + } + } + + @Test(expected = IllegalArgumentException.class) + public void testNegativeFrameSize() throws Exception { + testInvalidFrame(-1); + } + + @Test(expected = IllegalArgumentException.class) + public void testEmptyFrame() throws Exception { + // 8 because frame size includes the frame length. + testInvalidFrame(8); + } + + /** + * Creates a number of randomly sized frames and feed them to the given decoder, verifying + * that the frames were read. + */ + private ByteBuf createAndFeedFrames( + int frameCount, + TransportFrameDecoder decoder, + ChannelHandlerContext ctx) throws Exception { + ByteBuf data = Unpooled.buffer(); + for (int i = 0; i < frameCount; i++) { + byte[] frame = new byte[1024 * (RND.nextInt(31) + 1)]; + data.writeLong(frame.length + 8); + data.writeBytes(frame); + } + + try { + while (data.isReadable()) { + int size = RND.nextInt(4 * 1024) + 256; + decoder.channelRead(ctx, data.readSlice(Math.min(data.readableBytes(), size)).retain()); + } + + verify(ctx, times(frameCount)).fireChannelRead(any(ByteBuf.class)); + } catch (Exception e) { + release(data); + throw e; + } + return data; + } + + private void verifyAndCloseDecoder( + TransportFrameDecoder decoder, + ChannelHandlerContext ctx, + ByteBuf data) throws Exception { + try { + decoder.channelInactive(ctx); + assertTrue("There shouldn't be dangling references to the data.", data.release()); + } finally { + release(data); + } + } + + private void testInvalidFrame(long size) throws Exception { + TransportFrameDecoder decoder = new TransportFrameDecoder(); + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + ByteBuf frame = Unpooled.copyLong(size); + try { + decoder.channelRead(ctx, frame); + } finally { + release(frame); + } + } + + private ChannelHandlerContext mockChannelHandlerContext() { + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + when(ctx.fireChannelRead(any())).thenAnswer(in -> { + ByteBuf buf = (ByteBuf) in.getArguments()[0]; + buf.release(); + return null; + }); + return ctx; + } + + private void release(ByteBuf buf) { + if (buf.refCnt() > 0) { + buf.release(buf.refCnt()); + } + } + + private static class MockInterceptor implements TransportFrameDecoder.Interceptor { + + private int remainingReads; + + MockInterceptor(int readCount) { + this.remainingReads = readCount; + } + + @Override + public boolean handle(ByteBuf data) throws Exception { + data.readerIndex(data.readerIndex() + data.readableBytes()); + assertFalse(data.isReadable()); + remainingReads -= 1; + return remainingReads != 0; + } + + @Override + public void exceptionCaught(Throwable cause) throws Exception { + + } + + @Override + public void channelInactive() throws Exception { + + } + + } + +} diff --git a/network-common/src/test/resources/log4j.properties b/network-common/src/test/resources/log4j.properties new file mode 100644 index 0000000..3c1122b --- /dev/null +++ b/network-common/src/test/resources/log4j.properties @@ -0,0 +1,29 @@ +# +# Copyright 2022 Bytedance Inc. +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=DEBUG, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Silence verbose logs from 3rd-party libraries. +log4j.logger.io.netty=INFO diff --git a/patch/spark-3.0-aqe-skewjoin.patch b/patch/spark-3.0-aqe-skewjoin.patch new file mode 100644 index 0000000..1f733e8 --- /dev/null +++ b/patch/spark-3.0-aqe-skewjoin.patch @@ -0,0 +1,48 @@ +Index: core/src/main/scala/org/apache/spark/SparkConf.scala +diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala +--- a/core/src/main/scala/org/apache/spark/SparkConf.scala ++++ b/core/src/main/scala/org/apache/spark/SparkConf.scala +@@ -458,6 +458,11 @@ + */ + def getAppId: String = get("spark.app.id") + ++ /** ++ * Return true if cloud shuffle service is enabled. ++ */ ++ def isCssEnable: Boolean = get("spark.shuffle.manager", "sort").endsWith("CssShuffleManager") ++ + /** Does the configuration contain a given parameter? */ + def contains(key: String): Boolean = { + settings.containsKey(key) || + + +Index: sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala +diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala +--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala ++++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/OptimizeSkewedJoin.scala +@@ -21,6 +21,7 @@ + + import org.apache.commons.io.FileUtils + ++import org.apache.spark.sql.SparkSession + import org.apache.spark.{MapOutputStatistics, MapOutputTrackerMaster, SparkEnv} + import org.apache.spark.sql.catalyst.plans._ + import org.apache.spark.sql.catalyst.rules.Rule +@@ -120,7 +121,15 @@ + } else { + mapStartIndices(i + 1) + } +- PartialReducerPartitionSpec(reducerId, startMapIndex, endMapIndex) ++ if (SparkSession.getActiveSession.map(_.sparkContext).get.conf.isCssEnable) { ++ // With CSS as Shuffle Service ++ // PartialReducerPartitionSpec is not carrying startMapIndex and endMapIndex, ++ // Because push-based shuffle normally mixed up all MapTask data for IO aggregation. ++ // PartialReducerPartitionSpec will carry split task index and total split task num. ++ PartialReducerPartitionSpec(reducerId, mapStartIndices.length, i) ++ } else { ++ PartialReducerPartitionSpec(reducerId, startMapIndex, endMapIndex) ++ } + }) + } else { + None + diff --git a/pom.xml b/pom.xml new file mode 100644 index 0000000..8f64e41 --- /dev/null +++ b/pom.xml @@ -0,0 +1,704 @@ + + + + + 4.0.0 + + com.bytedance.inf + css-root + 1.0.0 + pom + + Cloud Shuffle Service Project Parent POM + + + UTF-8 + UTF-8 + 1.8 + ${java.version} + ${java.version} + 3.5.4 + 1.7.30 + 1.2.17 + 14.0.1 + 512m + 4.2.2 + 3.1.0 + 3.0.1 + 2.6.0 + 2.7.1 + 0.3.6 + 2.10.0 + 4.1.47.Final + 1.0.0 + 3.9 + 3.0.0 + 1.7.1 + 1.5.0-4 + + + + api + network-common + common + client + service + + + + + gcs-maven-central-mirror + GCS Maven Central mirror + https://maven-central.storage-download.googleapis.com/maven2/ + + true + + + false + + + + central + Maven Repository + https://repo.maven.apache.org/maven2 + + true + + + false + + + + + + + gcs-maven-central-mirror + GCS Maven Central mirror + https://maven-central.storage-download.googleapis.com/maven2/ + + true + + + false + + + + central + https://repo.maven.apache.org/maven2 + + true + + + false + + + + + + + + com.netflix.concurrency-limits + concurrency-limits-core + ${netflix.concurrency.version} + + + com.fasterxml.jackson.core + jackson-databind + ${fasterxml.jackson.version} + + + com.fasterxml.jackson.core + jackson-annotations + ${fasterxml.jackson.version} + + + org.apache.spark + spark-tags_${scala.binary.version} + ${spark.version} + + + com.google.guava + guava + ${guava.version} + + + com.github.luben + zstd-jni + ${zstd-jni.version} + + + org.lz4 + lz4-java + ${lz4-java.version} + + + javax.servlet + javax.servlet-api + ${javaxservlet.version} + + + log4j + log4j + ${log4j.version} + + + org.slf4j + slf4j-api + ${slf4j.version} + + + org.slf4j + slf4j-log4j12 + ${slf4j.version} + + + org.slf4j + jul-to-slf4j + ${slf4j.version} + + + org.slf4j + jcl-over-slf4j + ${slf4j.version} + + + + junit + junit + 4.12 + test + + + org.scalatest + scalatest_${scala.binary.version} + 3.0.8 + test + + + + io.netty + netty-all + ${netty-all.version} + + + org.apache.commons + commons-lang3 + ${commons-lang3.version} + + + org.fusesource.leveldbjni + leveldbjni-all + 1.8 + compile + + + com.google.code.findbugs + jsr305 + ${jsr305.version} + compile + + + org.apache.commons + commons-crypto + ${commons-crypto.version} + compile + + + jna + net.java.dev.jna + + + + + org.mockito + mockito-core + 1.10.19 + test + + + io.dropwizard.metrics + metrics-core + ${codahale.metrics.version} + + + io.dropwizard.metrics + metrics-jvm + ${codahale.metrics.version} + + + io.dropwizard.metrics + metrics-jmx + ${codahale.metrics.version} + + + io.dropwizard.metrics + metrics-graphite + ${codahale.metrics.version} + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + compile + + + org.fusesource.leveldbjni + leveldbjni-all + + + asm + asm + + + org.codehaus.jackson + jackson-mapper-asl + + + org.ow2.asm + asm + + + org.jboss.netty + netty + + + io.netty + netty + + + + commons-beanutils + commons-beanutils-core + + + commons-logging + commons-logging + + + org.mockito + mockito-all + + + org.mortbay.jetty + servlet-api-2.5 + + + javax.servlet + servlet-api + + + junit + junit + + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + + + net.java.dev.jets3t + jets3t + + + + javax.ws.rs + jsr311-api + + + org.eclipse.jetty + jetty-webapp + + + + com.codahale.metrics + metrics-core + + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + + + io.grpc + grpc-core + + + org.apache.thrift + libthrift + + + + + org.apache.hadoop + hadoop-client + ${hadoop.version} + compile + + + org.fusesource.leveldbjni + leveldbjni-all + + + asm + asm + + + org.codehaus.jackson + jackson-mapper-asl + + + org.ow2.asm + asm + + + org.jboss.netty + netty + + + io.netty + netty + + + + commons-beanutils + commons-beanutils-core + + + commons-logging + commons-logging + + + org.mockito + mockito-all + + + org.mortbay.jetty + servlet-api-2.5 + + + javax.servlet + servlet-api + + + junit + junit + + + com.sun.jersey + * + + + com.sun.jersey.jersey-test-framework + * + + + com.sun.jersey.contribs + * + + + net.java.dev.jets3t + jets3t + + + + javax.ws.rs + jsr311-api + + + org.eclipse.jetty + jetty-webapp + + + + com.codahale.metrics + metrics-core + + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + + + io.grpc + grpc-core + + + org.apache.thrift + libthrift + + + + + org.apache.curator + curator-framework + ${curator.version} + + + org.apache.curator + curator-test + ${curator.version} + test + + + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.22.0 + + + + **/*Spec*.* + **/*Suite*.* + **/*Test*.* + + ${project.build.directory}/surefire-reports + false + 1 + false + 3 + + + + test + + test + + + + + + org.apache.maven.plugins + maven-shade-plugin + 2.4.3 + + + package + + shade + + + false + true + + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + **/log4j.properties + + + + + + reference.conf + + + + META-INF/services/org.apache.spark.sql.sources.DataSourceRegister + + + + + META-INF/services/org.apache.hadoop.fs.FileSystem + + + + + + + + + net.alchim31.maven + scala-maven-plugin + 3.3.2 + + + scala-compile-first + process-resources + + compile + + + + scala-test-compile-first + process-test-resources + + testCompile + + + + + + org.scalatest + scalatest-maven-plugin + 1.0 + + ${project.build.directory}/surefire-reports + . + WDF TestResult.txt + ${project.build.directory}/site/scalatest + false + + + + test + test + + test + + + + + + + + + org.scalastyle + scalastyle-maven-plugin + 1.0.0 + + false + true + true + false + ${basedir}/src/main/scala + ${basedir}/src/test/scala + scalastyle-config.xml + ${basedir}/target/scalastyle-output.xml + ${project.build.sourceEncoding} + ${project.reporting.outputEncoding} + + + + + check + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 3.1.2 + + true + true + + ${basedir}/src/main/java + ${basedir}/src/main/scala + + + ${basedir}/src/test/java + + dev/checkstyle.xml + ${basedir}/target/checkstyle-output.xml + ${project.build.sourceEncoding} + ${project.reporting.outputEncoding} + + + + + com.puppycrawl.tools + checkstyle + 8.43 + + + + + + check + + + + + + + + + + scala-11 + + css-assembly_2 + spark-shuffle-manager-2 + + + 2.11.12 + 2.11 + 2.3.2 + 2.9.9 + 4.1.17.Final + 3.5 + 1.3.9 + 4.0.1 + 1.7.16 + 1.4.0 + + + + + scala-12 + + true + + + css-assembly_3 + spark-shuffle-manager-3 + + + 2.12.10 + 2.12 + + + + + diff --git a/sbin/css-config.sh b/sbin/css-config.sh new file mode 100755 index 0000000..8891ede --- /dev/null +++ b/sbin/css-config.sh @@ -0,0 +1,45 @@ +#!/usr/bin/env bash +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# included in all the css scripts with source command +# should not be executable directly +# also should not be passed any arguments, since we need original $* + + +if [ -z "${CSS_HOME}" ]; then + export CSS_HOME="$(cd "`dirname "$0"`"/..; pwd)" +fi + +export CSS_CONF_DIR="${CSS_CONF_DIR:-"${CSS_HOME}/conf"}" + + +# mark default conf here +# export CSS_MASTER_HOST=localhost +export CSS_MASTER_PORT=9099 + +# export CSS_WORKER_HOST=localhost +export CSS_WORKER_PORT=0 + +# no need to set this (use random port instead) +export CSS_WORKER_PUSH_PORT=0 +export CSS_WORKER_FETCH_PORT=0 + +export MASTER_JAVA_OPTS="-Xmx1024m" +export WORKER_JAVA_OPTS="-Xmx1024m -XX:MaxDirectMemorySize=4096m" + +export CSS_WORKER_INSTANCES=1 diff --git a/sbin/css-daemon.sh b/sbin/css-daemon.sh new file mode 100755 index 0000000..e457517 --- /dev/null +++ b/sbin/css-daemon.sh @@ -0,0 +1,221 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +usage="Usage: css-daemon.sh (start|stop|status) (master|worker) " + +# arg must start with: + +# if no args specified, show usage +if [ $# -le 0 ]; then + echo $usage + exit 1 +fi + +if [ -z "${CSS_HOME}" ]; then + export CSS_HOME="$(cd "`dirname "$0"`"/..; pwd)" +fi + +. "${CSS_HOME}/sbin/css-config.sh" + + +# Find the java binary +if [ -n "${JAVA_HOME}" ]; then + RUNNER="${JAVA_HOME}/bin/java" +else + if [ "$(command -v java)" ]; then + RUNNER="java" + else + echo "JAVA_HOME is not set" >&2 + exit 1 + fi +fi + + +if [[ -d ${CSS_HOME} ]]; then + export CLASSPATH=${JAVA_HOME}/lib:${CSS_HOME}/lib/* +fi + +echo "use classpath ${CLASSPATH}" +# get arguments + +option=$1 +shift +instance=$1 +shift +instance_num=$1 +shift + + +css_rotate_log () +{ + log=$1; + num=5; + if [ -n "$2" ]; then + num=$2 + fi + + if [ -f "$log" ]; then # rotate logs + while [ $num -gt 1 ]; do + prev=`expr $num - 1` + [ -f "$log.$prev" ] && mv "$log.$prev" "$log.$num" + num=$prev + done + mv "$log" "$log.$num"; + fi +} + + +if [ "$CSS_IDENT_STRING" = "" ]; then + export CSS_IDENT_STRING="$USER" +fi + +# get log directory +if [ "$CSS_LOG_DIR" = "" ]; then + export CSS_LOG_DIR="${CSS_HOME}/logs" +fi +mkdir -p "$CSS_LOG_DIR" +touch "$CSS_LOG_DIR"/.css_test > /dev/null 2>&1 +TEST_LOG_DIR=$? +if [ "${TEST_LOG_DIR}" = "0" ]; then + rm -f "$CSS_LOG_DIR"/.css_test +else + chown "$CSS_IDENT_STRING" "$CSS_LOG_DIR" +fi + +if [ "$CSS_PID_DIR" = "" ]; then + CSS_PID_DIR=/tmp +fi + +# some variables +log="$CSS_LOG_DIR/css-$CSS_IDENT_STRING-$instance-$instance_num.out" +pid="$CSS_PID_DIR/css-$CSS_IDENT_STRING-$instance-$instance_num.pid" + +export JAVA_OPTS="$JAVA_OPTS -Dcss.log.dir=$CSS_LOG_DIR -Dcss.log.filename=css-$CSS_IDENT_STRING-$instance-$instance_num.out" + +# Set default scheduling priority +if [ "$CSS_NICENESS" = "" ]; then + export CSS_NICENESS=0 +fi + +execute_command() { + if [ -z ${CSS_NO_DAEMONIZE+set} ]; then + nohup -- "$@" >/dev/null 2>&1 & + newpid="$!" + + echo "$newpid" > "$pid" + + # Poll for up to 5 seconds for the java process to start + for i in {1..10} + do + if [[ $(ps -p "$newpid" -o comm=) =~ "java" ]]; then + break + fi + sleep 0.5 + done + + sleep 2 + # Check if the process has died; in that case we'll tail the log so the user can see + if [[ ! $(ps -p "$newpid" -o comm=) =~ "java" ]]; then + echo "failed to launch: $@" + tail -10 "$log" | sed 's/^/ /' + echo "full log in $log" + fi + else + "$@" + fi +} + +run_command() { + mode="$1" + shift + + mkdir -p "$CSS_PID_DIR" + + if [ -f "$pid" ]; then + TARGET_ID="$(cat "$pid")" + if [[ $(ps -p "$TARGET_ID" -o comm=) =~ "java" ]]; then + echo "$instance running as process $TARGET_ID. Stop it first." + exit 1 + fi + fi + + if [ "$CSS_MASTER" != "" ]; then + echo rsync from "$CSS_MASTER" + rsync -a -e ssh --delete --exclude=.svn --exclude='logs/*' --exclude='contrib/hod/logs/*' "$CSS_MASTER/" "${CSS_HOME}" + fi + + css_rotate_log "$log" + echo "starting $instance, logging to $log" + + case "$mode" in + (class) + echo "Command: $RUNNER ${JAVA_OPTS} -classpath ${CLASSPATH} $@" + execute_command nice -n "$CSS_NICENESS" "$RUNNER" ${JAVA_OPTS} -classpath ${CLASSPATH} "$@" + ;; + + (*) + echo "unknown mode: $mode" + exit 1 + ;; + esac + +} + +case $option in + + (start) + run_command class "$@" + ;; + + (stop) + if [ -f $pid ]; then + TARGET_ID="$(cat "$pid")" + if [[ $(ps -p "$TARGET_ID" -o comm=) =~ "java" ]]; then + echo "stopping $instance process" + kill "$TARGET_ID" && rm -f "$pid" + else + echo "no $instance to stop" + fi + else + echo "no $instance to stop" + fi + ;; + + (status) + if [ -f $pid ]; then + TARGET_ID="$(cat "$pid")" + if [[ $(ps -p "$TARGET_ID" -o comm=) =~ "java" ]]; then + echo $instance is running. + exit 0 + else + echo $pid file is present but $instance not running + exit 1 + fi + else + echo $instance not running. + exit 2 + fi + ;; + + (*) + echo $usage + exit 1 + ;; + +esac diff --git a/sbin/start-all.sh b/sbin/start-all.sh new file mode 100755 index 0000000..a2f8008 --- /dev/null +++ b/sbin/start-all.sh @@ -0,0 +1,37 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Start all css daemons. +# Starts the master on this node. +# Starts a worker on each node specified in conf/workers + +if [ -z "${CSS_HOME}" ]; then + export CSS_HOME="$(cd "`dirname "$0"`"/..; pwd)" +fi + +# Load the Css configuration +. "${CSS_HOME}/sbin/css-config.sh" + + +# Start Master +"${CSS_HOME}/sbin"/start-master.sh + + +# Start Workers +"${CSS_HOME}/sbin"/start-workers.sh diff --git a/sbin/start-master.sh b/sbin/start-master.sh new file mode 100755 index 0000000..8c4b5f6 --- /dev/null +++ b/sbin/start-master.sh @@ -0,0 +1,66 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +if [ -z "${CSS_HOME}" ]; then + export CSS_HOME="$(cd "`dirname "$0"`"/..; pwd)" +fi + +CLASS="com.bytedance.css.service.deploy.master.Master" + +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + echo "Usage: ./sbin/start-master.sh " + pattern="Usage:" + pattern+="\|Using CSS's default log4j profile:" + pattern+="\|Started daemon with process name" + pattern+="\|Registered signal handler for" + + "${CSS_HOME}"/sbin/css-daemon.sh start master 1 $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 + exit 1 +fi + +ORIGINAL_ARGS="$@" + +. "${CSS_HOME}/sbin/css-config.sh" + +if [ "$MASTER_JAVA_OPTS" != "" ]; then + JAVA_OPTS=$MASTER_JAVA_OPTS + export JAVA_OPTS=$JAVA_OPTS +else + export JAVA_OPTS="" +fi + +if [ "$CSS_MASTER_PORT" = "" ]; then + CSS_MASTER_PORT=9099 +fi + +if [ "$CSS_MASTER_HOST" = "" ]; then + case `uname` in + (SunOS) + CSS_MASTER_HOST="`/usr/sbin/check-hostname | awk '{print $NF}'`" + ;; + (*) + CSS_MASTER_HOST="`hostname -f`" + ;; + esac +fi + + +"${CSS_HOME}/sbin"/css-daemon.sh start master 1 $CLASS \ + --host $CSS_MASTER_HOST --port $CSS_MASTER_PORT \ + $ORIGINAL_ARGS diff --git a/sbin/start-worker.sh b/sbin/start-worker.sh new file mode 100755 index 0000000..9f5b06a --- /dev/null +++ b/sbin/start-worker.sh @@ -0,0 +1,108 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +if [ -z "${CSS_HOME}" ]; then + export CSS_HOME="$(cd "`dirname "$0"`"/..; pwd)" +fi + +CLASS="com.bytedance.css.service.deploy.worker.Worker" + +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + echo "Usage: ./sbin/start-worker.sh " + pattern="Usage:" + pattern+="\|Using CSS's default log4j profile:" + pattern+="\|Started daemon with process name" + pattern+="\|Registered signal handler for" + + "${CSS_HOME}"/sbin/css-daemon.sh start worker 1 $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 + exit 1 +fi + +. "${CSS_HOME}/sbin/css-config.sh" + +# First argument should be the master; we need to store it aside because we may +# need to insert arguments between it and the other arguments +MASTER=$1 +shift + +ORIGINAL_ARGS="$@" + +if [ "$WORKER_JAVA_OPTS" != "" ]; then + JAVA_OPTS=$WORKER_JAVA_OPTS + export JAVA_OPTS=$JAVA_OPTS +else + export JAVA_OPTS="" +fi + + +# Start up the appropriate number of workers on this machine. +# quick local function to start a worker +function start_instance { + WORKER_NUM=$1 + shift + + if [ "$CSS_WORKER_PORT" = "" ] || [ "$CSS_WORKER_PORT" = "0" ]; then + PORT_FLAG= + PORT_NUM= + else + PORT_FLAG="--port" + PORT_NUM=$(( $CSS_WORKER_PORT - $WORKER_NUM + 1 )) + fi + + if [ "$CSS_WORKER_PUSH_PORT" = "" ] || [ "$CSS_WORKER_PUSH_PORT" = "0" ]; then + PUSH_PORT_FLAG= + PUSH_PORT_NUM= + else + PUSH_PORT_FLAG="--pushPort" + PUSH_PORT_NUM=$(( $CSS_WORKER_PUSH_PORT - $WORKER_NUM + 1 )) + fi + + if [ "$CSS_WORKER_FETCH_PORT" = "" ] || [ "$CSS_WORKER_FETCH_PORT" = "0" ]; then + FETCH_PORT_FLAG= + FETCH_PORT_NUM= + else + FETCH_PORT_FLAG="--fetchPort" + FETCH_PORT_NUM=$(( $CSS_WORKER_FETCH_PORT - $WORKER_NUM + 1 )) + fi + + if [ "$CSS_WORKER_HOST" = "" ]; then + case `uname` in + (SunOS) + CSS_WORKER_HOST="`/usr/sbin/check-hostname | awk '{print $NF}'`" + ;; + (*) + CSS_WORKER_HOST="`hostname -f`" + ;; + esac + fi + + "${CSS_HOME}/sbin"/css-daemon.sh start worker $WORKER_NUM $CLASS \ + --host $CSS_WORKER_HOST $PORT_FLAG $PORT_NUM \ + $PUSH_PORT_FLAG $PUSH_PORT_NUM \ + $FETCH_PORT_FLAG $FETCH_PORT_NUM \ + $MASTER "$@" +} + +if [ "$CSS_WORKER_INSTANCES" = "" ]; then + start_instance 1 "$@" +else + for ((i=0; i<$CSS_WORKER_INSTANCES; i++)); do + start_instance $(( 1 + $i )) "$@" + done +fi diff --git a/sbin/start-workers.sh b/sbin/start-workers.sh new file mode 100755 index 0000000..818debc --- /dev/null +++ b/sbin/start-workers.sh @@ -0,0 +1,45 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Starts a worker instance on each machine specified in the conf/workers file. + +if [ -z "${CSS_HOME}" ]; then + export CSS_HOME="$(cd "`dirname "$0"`"/..; pwd)" +fi + +. "${CSS_HOME}/sbin/css-config.sh" + +# Find the port number for the master +if [ "$CSS_MASTER_PORT" = "" ]; then + CSS_MASTER_PORT=9099 +fi + +if [ "$CSS_MASTER_HOST" = "" ]; then + case `uname` in + (SunOS) + CSS_MASTER_HOST="`/usr/sbin/check-hostname | awk '{print $NF}'`" + ;; + (*) + CSS_MASTER_HOST="`hostname -f`" + ;; + esac +fi + +# Launch the workers +"${CSS_HOME}/sbin/workers.sh" cd "${CSS_HOME}" \; "${CSS_HOME}/sbin/start-worker.sh" "css://$CSS_MASTER_HOST:$CSS_MASTER_PORT" diff --git a/sbin/stop-all.sh b/sbin/stop-all.sh new file mode 100755 index 0000000..0687384 --- /dev/null +++ b/sbin/stop-all.sh @@ -0,0 +1,49 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Stop all css daemons. +# Run this on the master node. + +if [ -z "${CSS_HOME}" ]; then + export CSS_HOME="$(cd "`dirname "$0"`"/..; pwd)" +fi + +# Load the css configuration +. "${CSS_HOME}/sbin/css-config.sh" + +# Stop the workers, then the master +"${CSS_HOME}/sbin"/stop-workers.sh +"${CSS_HOME}/sbin"/stop-master.sh + +if [ "$1" == "--wait" ] +then + printf "Waiting for workers to shut down..." + while true + do + running=`${CSS_HOME}/sbin/workers.sh ps -ef | grep -v grep | grep deploy.worker.Worker` + if [ -z "$running" ] + then + printf "\nAll workers successfully shut down.\n" + break + else + printf "." + sleep 10 + fi + done +fi diff --git a/sbin/stop-master.sh b/sbin/stop-master.sh new file mode 100755 index 0000000..2602a18 --- /dev/null +++ b/sbin/stop-master.sh @@ -0,0 +1,30 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Stops the master on the machine this script is executed on. + +if [ -z "${CSS_HOME}" ]; then + export CSS_HOME="$(cd "`dirname "$0"`"/..; pwd)" +fi + +CLASS="com.bytedance.css.service.deploy.master.Master" + +. "${CSS_HOME}/sbin/css-config.sh" + +"${CSS_HOME}/sbin"/css-daemon.sh stop master 1 $CLASS diff --git a/sbin/stop-worker.sh b/sbin/stop-worker.sh new file mode 100755 index 0000000..e0e32a3 --- /dev/null +++ b/sbin/stop-worker.sh @@ -0,0 +1,45 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# A shell script to stop all workers on a single worker +# +# Environment variables +# +# CSS_WORKER_INSTANCES The number of worker instances that should be +# running on this worker machine. Default is 1. + +# Usage: stop-worker.sh +# Stops all workers on this worker machine + +if [ -z "${CSS_HOME}" ]; then + export CSS_HOME="$(cd "`dirname "$0"`"/..; pwd)" +fi + +CLASS="com.bytedance.css.service.deploy.worker.Worker" + +. "${CSS_HOME}/sbin/css-config.sh" + + +if [ "$CSS_WORKER_INSTANCES" = "" ]; then + "${CSS_HOME}/sbin"/css-daemon.sh stop worker 1 $CLASS +else + for ((i=0; i<$CSS_WORKER_INSTANCES; i++)); do + "${CSS_HOME}/sbin"/css-daemon.sh stop worker $(( $i + 1 )) $CLASS + done +fi diff --git a/sbin/stop-workers.sh b/sbin/stop-workers.sh new file mode 100755 index 0000000..c2fe57a --- /dev/null +++ b/sbin/stop-workers.sh @@ -0,0 +1,26 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +if [ -z "${CSS_HOME}" ]; then + export CSS_HOME="$(cd "`dirname "$0"`"/..; pwd)" +fi + +. "${CSS_HOME}/sbin/css-config.sh" + +"${CSS_HOME}/sbin/workers.sh" cd "${CSS_HOME}" \; "${CSS_HOME}/sbin"/stop-worker.sh diff --git a/sbin/workers.sh b/sbin/workers.sh new file mode 100755 index 0000000..2c69e28 --- /dev/null +++ b/sbin/workers.sh @@ -0,0 +1,98 @@ +#!/usr/bin/env bash + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Run a shell command on all slave hosts. +# +# Environment Variables +# +# CSS_SLAVES File naming remote hosts. +# Default is ${CSS_CONF_DIR}/slaves. +# CSS_CONF_DIR Alternate conf dir. Default is ${CSS_HOME}/conf. +# CSS_SLAVE_SLEEP Seconds to sleep between spawning remote commands. +# CSS_SSH_OPTS Options passed to ssh when running remote commands. +## + +usage="Usage: workers.sh [--config ] command..." + +# if no args specified, show usage +if [ $# -le 0 ]; then + echo $usage + exit 1 +fi + +if [ -z "${CSS_HOME}" ]; then + export CSS_HOME="$(cd "`dirname "$0"`"/..; pwd)" +fi + +. "${CSS_HOME}/sbin/css-config.sh" + +# If the slaves file is specified in the command line, +# then it takes precedence over the definition in +if [ -f "$CSS_SLAVES" ]; then + HOSTLIST=`cat "$CSS_SLAVES"` +fi + +# Check if --config is passed as an argument. It is an optional parameter. +# Exit if the argument is not a directory. +if [ "$1" == "--config" ] +then + shift + conf_dir="$1" + if [ ! -d "$conf_dir" ] + then + echo "ERROR : $conf_dir is not a directory" + echo $usage + exit 1 + else + export CSS_CONF_DIR="$conf_dir" + fi + shift +fi + +if [ "$HOSTLIST" = "" ]; then + if [ "$CSS_SLAVES" = "" ]; then + if [ -f "${CSS_CONF_DIR}/workers" ]; then + HOSTLIST=`cat "${CSS_CONF_DIR}/workers"` + else + HOSTLIST=localhost + fi + else + HOSTLIST=`cat "${CSS_SLAVES}"` + fi +fi + +# By default disable strict host key checking +if [ "$CSS_SSH_OPTS" = "" ]; then + CSS_SSH_OPTS="-o StrictHostKeyChecking=no" +fi + +for worker in `echo "$HOSTLIST"|sed "s/#.*$//;/^$/d"`; do + if [ -n "${CSS_SSH_FOREGROUND}" ]; then + ssh $CSS_SSH_OPTS "$worker" $"${@// /\\ }" \ + 2>&1 | sed "s/^/$worker: /" + else + ssh $CSS_SSH_OPTS "$worker" $"${@// /\\ }" \ + 2>&1 | sed "s/^/$worker: /" & + fi + if [ "$CSS_SLAVE_SLEEP" != "" ]; then + sleep $CSS_SLAVE_SLEEP + fi +done + +wait diff --git a/scalastyle-config.xml b/scalastyle-config.xml new file mode 100644 index 0000000..0815aed --- /dev/null +++ b/scalastyle-config.xml @@ -0,0 +1,397 @@ + + + + + Scalastyle standard configuration + + + + + + + + + + + + + + + + + + + + + + + + true + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ARROW, EQUALS, ELSE, TRY, CATCH, FINALLY, LARROW, RARROW + + + + + + ARROW, EQUALS, COMMA, COLON, IF, ELSE, DO, WHILE, FOR, MATCH, TRY, CATCH, FINALLY, LARROW, RARROW + + + + + + + + + ^println$ + + + + + @VisibleForTesting + + + + + Runtime\.getRuntime\.addShutdownHook + + + + + mutable\.SynchronizedBuffer + + + + + Class\.forName + + + + + Await\.result + + + + + Await\.ready + + + + + (\.toUpperCase|\.toLowerCase)(?!(\(|\(Locale.ROOT\))) + + + + + throw new \w+Error\( + + + + + + JavaConversions + Instead of importing implicits in scala.collection.JavaConversions._, import + scala.collection.JavaConverters._ and use .asScala / .asJava methods + + + + org\.apache\.commons\.lang\. + Use Commons Lang 3 classes (package org.apache.commons.lang3.*) instead + of Commons Lang 2 (package org.apache.commons.lang.*) + + + + FileSystem.get\([a-zA-Z_$][a-zA-Z_$0-9]*\) + + + + + extractOpt + Use jsonOption(x).map(.extract[T]) instead of .extractOpt[T], as the latter + is slower. + + + + + java,scala,3rdParty,spark + javax?\..* + scala\..* + (?!org\.apache\.spark\.).* + org\.apache\.spark\..* + + + + + + COMMA + + + + + + \)\{ + + + + + (?m)^(\s*)/[*][*].*$(\r|)\n^\1 [*] + Use Javadoc style indentation for multiline comments + + + + case[^\n>]*=>\s*\{ + Omit braces in case clauses. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 800> + + + + + 30 + + + + + 10 + + + + + 50 + + + + + + + + + + + -1,0,1,2,3 + + + + Objects.toStringHelper + Avoid using Object.toStringHelper. Use ToStringBuilder instead. + + diff --git a/service/pom.xml b/service/pom.xml new file mode 100644 index 0000000..b21647e --- /dev/null +++ b/service/pom.xml @@ -0,0 +1,128 @@ + + + + 4.0.0 + + + com.bytedance.inf + css-root + 1.0.0 + ../pom.xml + + + css-service_${scala.binary.version} + jar + Cloud Shuffle Service Service + + + + com.bytedance.inf + css-api + ${project.version} + + + com.bytedance.inf + css-common_${scala.binary.version} + ${project.version} + + + org.apache.curator + curator-framework + + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-client + + + + com.bytedance.inf + css-client_${scala.binary.version} + ${project.version} + test + + + + junit + junit + test + + + + org.scalatest + scalatest_${scala.binary.version} + test + + + + org.pegdown + pegdown + 1.4.2 + test + + + org.apache.curator + curator-test + test + + + + + + + net.alchim31.maven + scala-maven-plugin + + + org.apache.maven.plugins + maven-shade-plugin + + + org.apache.maven.plugins + maven-surefire-plugin + + 1 + false + + + + org.scalatest + scalatest-maven-plugin + + + org.apache.maven.plugins + maven-jar-plugin + 3.1.2 + + + package + + test-jar + + + + + + + diff --git a/service/src/main/java/com/bytedance/css/service/deploy/worker/DiskFileFlushTask.java b/service/src/main/java/com/bytedance/css/service/deploy/worker/DiskFileFlushTask.java new file mode 100644 index 0000000..3e0fc0e --- /dev/null +++ b/service/src/main/java/com/bytedance/css/service/deploy/worker/DiskFileFlushTask.java @@ -0,0 +1,67 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker; + +import io.netty.buffer.ByteBuf; +import io.netty.util.ReferenceCountUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.channels.FileChannel; +import com.codahale.metrics.Timer; + +public class DiskFileFlushTask extends FlushTask { + + private static final Logger logger = LoggerFactory.getLogger(DiskFileFlushTask.class); + + private final FileChannel fileChannel; + private final ByteBuf data; + + public DiskFileFlushTask(ByteBuf data, FileNotifier notifier, FileChannel fileChannel) { + super(data, notifier); + this.fileChannel = fileChannel; + this.data = data; + } + + @Override + public void flush() { + try { + if (!hasException()) { + Timer.Context timer = FileWriterMetrics.instance().diskFlushLatency.time(); + while (data.readableBytes() > 0) { + data.readBytes(fileChannel, data.readableBytes()); + } + // compositeBuf need to copy to HeapByteBuffer + // cost extra cpu time + // fileChannel.write(data.nioBuffer()); + timer.stop(); + } + } catch (IOException ex) { + FileWriterMetrics.instance().diskFlushFailed.mark(); + logger.error("DiskFileFlushTask flushed failed", ex); + getNotifier().setException(ex); + } finally { + if (data != null) { + ReferenceCountUtil.safeRelease(data); + } + } + } +} diff --git a/service/src/main/java/com/bytedance/css/service/deploy/worker/DiskFileWriter.java b/service/src/main/java/com/bytedance/css/service/deploy/worker/DiskFileWriter.java new file mode 100644 index 0000000..17c0e38 --- /dev/null +++ b/service/src/main/java/com/bytedance/css/service/deploy/worker/DiskFileWriter.java @@ -0,0 +1,311 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker; + +import com.bytedance.css.common.exception.AlreadyClosedException; +import com.bytedance.css.common.exception.EpochShouldRotateException; +import com.bytedance.css.common.protocol.ShuffleMode; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.CompositeByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.util.ReferenceCountUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.util.ArrayList; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +public final class DiskFileWriter implements FileWriter { + + private static final Logger logger = LoggerFactory.getLogger(DiskFileWriter.class); + + private static final long WAIT_INTERVAL_MS = 20; + private static final int COMPOSITE_BYTE_BUF_MAX_NUM = 128; + private final FileNotifier notifier = new FileNotifier(); + + private final File file; + private final FileChannel channel; + private volatile boolean closed; + private volatile boolean rotated; + + private final AtomicInteger numPendingWrites = new AtomicInteger(); + private final ArrayList chunkOffsets = new ArrayList<>(); + private long nextBoundary; + private long bytesFlushed = 0L; + private long bytesSubmitted = 0L; + + private final FileFlusher flusher; + private volatile boolean testSubmitFlushTaskException = false; + private CompositeByteBuf compositeByteBuf = null; + + private final long chunkSize; + private final long timeoutMs; + private final long flushBufferSize; + private final long epochRotateThreshold; + + public DiskFileWriter( + File file, + FileFlusher flusher, + long chunkSize, + long timeoutMs, + long flushBufferSize, + long epochRotateThreshold) throws IOException { + this.file = file; + this.channel = new FileOutputStream(file).getChannel(); + this.flusher = flusher; + this.chunkSize = chunkSize; + this.timeoutMs = timeoutMs; + this.flushBufferSize = flushBufferSize; + this.epochRotateThreshold = epochRotateThreshold; + this.nextBoundary = chunkSize; + this.chunkOffsets.add(0L); + FileWriterMetrics.instance().diskOpenedFileNum.inc(); + } + + private void submitFlushTask(FlushTask task) throws IOException { + if (testSubmitFlushTaskException || !flusher.submitTask(task, timeoutMs)) { + String msg = String.format("DiskFileWriter submit flush task timeout with file %s.", file.getAbsolutePath()); + IOException e = new IOException(msg); + notifier.setException(e); + throw e; + } + } + + private void flushBuffer(ByteBuf data, boolean finalFlush) throws IOException { + ByteBuf toBeFlushed = data == null ? compositeByteBuf : data; + notifier.getNumPendingFlushes().incrementAndGet(); + int numBytes = toBeFlushed.readableBytes(); + FlushTask task = new DiskFileFlushTask(toBeFlushed, notifier, channel); + submitFlushTask(task); + bytesFlushed += numBytes; + updateChunkOffsets(finalFlush); + } + + private void updateChunkOffsets(boolean forceSet) { + if (bytesFlushed >= nextBoundary || forceSet) { + chunkOffsets.add(bytesFlushed); + nextBoundary = bytesFlushed + chunkSize; + } + } + + private void waitOnNoPending(AtomicInteger counter) throws IOException { + long waitTime = timeoutMs; + while (counter.get() > 0 && waitTime > 0) { + try { + notifier.checkException(); + TimeUnit.MILLISECONDS.sleep(WAIT_INTERVAL_MS); + } catch (InterruptedException e) { + IOException ioe = new IOException(e); + notifier.setException(ioe); + throw ioe; + } + waitTime -= WAIT_INTERVAL_MS; + } + if (counter.get() > 0) { + String msg = String.format("DiskFileWriter wait pending actions timeout with %s .", file.getAbsolutePath()); + IOException ioe = new IOException(msg); + notifier.setException(ioe); + throw ioe; + } + notifier.checkException(); + } + + @Override + public ArrayList getChunkOffsets() { + return chunkOffsets; + } + + @Override + public void write(ByteBuf data, int mapperId, boolean ignoreRotate) throws IOException { + if (!ignoreRotate & rotated) { + // normally we don't hit this code branch, since we already call FileWriter.shouldRotate + // before write data. Just to in case + String msg = String.format("DiskFileWriter with %s rotated.", file.getAbsolutePath()); + logger.warn(msg); + throw new EpochShouldRotateException(msg); + } else if (closed) { + String msg = String.format("DiskFileWriter write with %s closed.", file.getAbsolutePath()); + logger.warn(msg); + throw new AlreadyClosedException(msg); + } + + synchronized (this) { + notifier.checkException(); + + if (compositeByteBuf == null) { + compositeByteBuf = Unpooled.compositeBuffer(COMPOSITE_BYTE_BUF_MAX_NUM); + } + final int numBytes = data.readableBytes(); + data.retain(); + + try { + if (flushBufferSize < numBytes) { + String msg = String.format("DiskFileWriter Flush giant batch with %s numBytes %s.", + file.getAbsolutePath(), numBytes); + logger.debug(msg); + flushBuffer(data, false); + } else { + // buffer not enough for data batch, trigger flush buffer to release memory + if (compositeByteBuf.readableBytes() + numBytes >= flushBufferSize) { + // buffer full, submit old compositeByteBuf to taskQueue, and re-create another one. + flushBuffer(null, false); + compositeByteBuf = Unpooled.compositeBuffer(COMPOSITE_BYTE_BUF_MAX_NUM); + } + + compositeByteBuf.addComponent(true, data); + } + } catch (IOException e) { + // here we only need to release data. compositeByteBuf will release by close & destroy. + ReferenceCountUtil.safeRelease(data); + throw e; + } + + bytesSubmitted += numBytes; + if (bytesSubmitted > epochRotateThreshold) { + rotated = true; + flushBuffer(null, !ignoreRotate); + compositeByteBuf = Unpooled.compositeBuffer(COMPOSITE_BYTE_BUF_MAX_NUM); + } + + numPendingWrites.decrementAndGet(); + } + } + + @Override + public long close() throws IOException { + if (closed) { + // if reallocation happens, the old partition will be closed in advanced + // to save StageEnd time. + checkException(); + String msg = String.format("DiskFileWriter with %s closed.", file.getAbsolutePath()); + logger.warn(msg); + return bytesFlushed; + } + + try { + waitOnNoPending(numPendingWrites); + closed = true; + + synchronized (this) { + if (compositeByteBuf != null && compositeByteBuf.readableBytes() > 0) { + flushBuffer(null, true); + compositeByteBuf = null; + } + if (chunkOffsets.get(chunkOffsets.size() - 1) != bytesFlushed) { + updateChunkOffsets(true); + } + } + + waitOnNoPending(notifier.getNumPendingFlushes()); + } finally { + if (compositeByteBuf != null) { + ReferenceCountUtil.safeRelease(compositeByteBuf); + compositeByteBuf = null; + } + channel.close(); + FileWriterMetrics.instance().diskOpenedFileNum.dec(); + } + + FileWriterMetrics.instance().diskRealFlushThroughput.mark(bytesFlushed); + return bytesFlushed; + } + + @Override + public void destroy() { + if (!closed) { + closed = true; + String msg = String.format("DiskFileWriter destroyed with path %s.", file.getAbsolutePath()); + logger.warn(msg); + try { + channel.close(); + FileWriterMetrics.instance().diskOpenedFileNum.dec(); + } catch (IOException e) { + String error = String.format("DiskFileWriter close channel failed with %s", file.getAbsolutePath()); + logger.warn(error, e); + } finally { + if (compositeByteBuf != null) { + ReferenceCountUtil.safeRelease(compositeByteBuf); + compositeByteBuf = null; + } + } + } + file.delete(); + } + + @Override + public void setException() { + notifier.setException(new IOException("ForTestOnly")); + } + + @Override + public void setSubmitQueueFullException(boolean exception) { + testSubmitFlushTaskException = exception; + } + + @Override + public CompositeByteBuf getCompositeByteBuf() { + return compositeByteBuf; + } + + @Override + public void checkException() throws IOException { + notifier.checkException(); + } + + @Override + public void incrementPendingWrites() { + numPendingWrites.incrementAndGet(); + } + + @Override + public void decrementPendingWrites() { + numPendingWrites.decrementAndGet(); + } + + @Override + public ShuffleMode getShuffleMode() { + return ShuffleMode.DISK; + } + + @Override + public String getFilePath() { + return file.getAbsolutePath(); + } + + @Override + public long getFileLength() { + return file.length(); + } + + @Override + public File getFile() { + return file; + } + + @Override + public boolean shouldRotate() { + return rotated; + } +} diff --git a/service/src/main/java/com/bytedance/css/service/deploy/worker/FileFlusher.java b/service/src/main/java/com/bytedance/css/service/deploy/worker/FileFlusher.java new file mode 100644 index 0000000..f873d17 --- /dev/null +++ b/service/src/main/java/com/bytedance/css/service/deploy/worker/FileFlusher.java @@ -0,0 +1,31 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker; + +public abstract class FileFlusher { + + public static final String DISK_FLUSHER_TYPE = "Flush"; + public static final String HDFS_FLUSHER_TYPE = "HdfsFlush"; + public static final String DISK_REAL_FLUSHER_TYPE = "RealFlush"; + + public abstract boolean submitTask(FlushTask task, long timeoutMs); + + public abstract int pendingFlushTaskNum(); +} diff --git a/service/src/main/java/com/bytedance/css/service/deploy/worker/FileFlusherImpl.java b/service/src/main/java/com/bytedance/css/service/deploy/worker/FileFlusherImpl.java new file mode 100644 index 0000000..f24d4d7 --- /dev/null +++ b/service/src/main/java/com/bytedance/css/service/deploy/worker/FileFlusherImpl.java @@ -0,0 +1,103 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker; + +import com.codahale.metrics.Meter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; + +public class FileFlusherImpl extends FileFlusher { + + private static final Logger logger = LoggerFactory.getLogger(FileFlusherImpl.class); + + private final String flushId; + // Flush means disk file flusher + // HdfsFlush means hdfs file flusher + // See. FileFlusher definition + private final String flusherType; + private int queueCapacity; + private int bufferSize; + + private LinkedBlockingQueue flushTaskQueue = null; + + private final Thread flushThread; + + public FileFlusherImpl( + String flushId, + String flusherType, + int queueCapacity, + int bufferSize) throws Exception { + this.flushId = flushId; + this.flusherType = flusherType; + this.queueCapacity = queueCapacity; + this.bufferSize = bufferSize; + flushTaskQueue = new LinkedBlockingQueue<>(this.queueCapacity); + + FileWriterMetrics.instance().registerFlushTaskQueueSize(() -> flushTaskQueue.size()); + Meter fileFlushThroughput = FileWriterMetrics.instance().getThroughput(this.flusherType); + Meter fileFlushQPS = FileWriterMetrics.instance().getQPS(this.flusherType); + + flushThread = new Thread(String.format("DiskFileFlusher-%s", flushId)) { + @Override + public void run() { + while (true) { + try { + FlushTask task = flushTaskQueue.take(); + + long dataSize = task.dataSize(); + // flush buffer into disk file + task.flush(); + fileFlushQPS.mark(); + fileFlushThroughput.mark(dataSize); + + task.getNotifier().getNumPendingFlushes().decrementAndGet(); + } catch (Exception ex) { + logger.error("DiskFileFlusher flushTaskQueue abnormal exception.", ex); + throw new RuntimeException(ex); + } + } + } + }; + + flushThread.setDaemon(true); + flushThread.setUncaughtExceptionHandler((t, e) -> { + logger.error(String.format("DiskFileFlusher flush thread %s abort abnormally.", t.getName()), e); + System.exit(-1); + }); + flushThread.start(); + } + + @Override + public boolean submitTask(FlushTask task, long timeoutMs) { + try { + return flushTaskQueue.offer(task, timeoutMs, TimeUnit.MILLISECONDS); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + @Override + public int pendingFlushTaskNum() { + return flushTaskQueue.size(); + } +} diff --git a/service/src/main/java/com/bytedance/css/service/deploy/worker/FileNotifier.java b/service/src/main/java/com/bytedance/css/service/deploy/worker/FileNotifier.java new file mode 100644 index 0000000..e654d0f --- /dev/null +++ b/service/src/main/java/com/bytedance/css/service/deploy/worker/FileNotifier.java @@ -0,0 +1,49 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +public class FileNotifier { + + private final AtomicInteger numPendingFlushes = new AtomicInteger(); + private final AtomicReference exception = new AtomicReference<>(); + + public AtomicInteger getNumPendingFlushes() { + return numPendingFlushes; + } + + void setException(IOException e) { + exception.set(e); + } + + boolean hasException() { + return exception.get() != null; + } + + void checkException() throws IOException { + IOException e = exception.get(); + if (e != null) { + throw e; + } + } +} diff --git a/service/src/main/java/com/bytedance/css/service/deploy/worker/FileWriter.java b/service/src/main/java/com/bytedance/css/service/deploy/worker/FileWriter.java new file mode 100644 index 0000000..57bc03c --- /dev/null +++ b/service/src/main/java/com/bytedance/css/service/deploy/worker/FileWriter.java @@ -0,0 +1,68 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker; + +import com.bytedance.css.common.protocol.ShuffleMode; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.CompositeByteBuf; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; + +public interface FileWriter { + + // For Hdfs file, it might not work, just return null would be fine + ArrayList getChunkOffsets(); + + // write ByteBuf data into target file + default void write(ByteBuf data, int mapperId) throws IOException { + write(data, mapperId, false); + } + + // write ByteBuf data into target file + void write(ByteBuf data, int mapperId, boolean ignoreRotate) throws IOException; + + // close file and return flushed length + long close() throws IOException; + + // close file and delete file + void destroy(); + + // for test only, set IOException into notifier + void setException(); + + // for test only, set IOException into flush queue. + void setSubmitQueueFullException(boolean exception); + + // for test only, get CompositeByteBuf which hold by file writer. + CompositeByteBuf getCompositeByteBuf(); + + void checkException() throws IOException; + boolean shouldRotate(); + + void incrementPendingWrites(); + void decrementPendingWrites(); + + ShuffleMode getShuffleMode(); + String getFilePath(); + long getFileLength(); + File getFile(); +} diff --git a/service/src/main/java/com/bytedance/css/service/deploy/worker/FileWriterMetrics.java b/service/src/main/java/com/bytedance/css/service/deploy/worker/FileWriterMetrics.java new file mode 100644 index 0000000..6dcf0d2 --- /dev/null +++ b/service/src/main/java/com/bytedance/css/service/deploy/worker/FileWriterMetrics.java @@ -0,0 +1,94 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker; + +import com.codahale.metrics.*; + +import java.util.HashMap; +import java.util.Map; + +public class FileWriterMetrics implements MetricSet { + + private static final FileWriterMetrics fileWriterMetrics = new FileWriterMetrics(); + + public static FileWriterMetrics instance() { + return fileWriterMetrics; + } + + private final Map allMetrics; + + // disk type metrics + public final Meter diskFlushQPS = new Meter(); + public final Timer diskFlushLatency = new Timer(); + public final Meter diskFlushThroughput = new Meter(); + public final Meter diskRealFlushThroughput = new Meter(); + public final Meter diskFlushFailed = new Meter(); + public final Counter diskOpenedFileNum = new Counter(); + + // hdfs type metrics + public final Meter hdfsFlushQPS = new Meter(); + public final Timer hdfsFlushLatency = new Timer(); + public final Meter hdfsFlushThroughput = new Meter(); + public final Meter hdfsFlushFailed = new Meter(); + public final Counter hdfsOpenedFileNum = new Counter(); + + private FileWriterMetrics() { + allMetrics = new HashMap<>(); + WorkerSource workerSource = WorkerSource.workerSource(); + allMetrics.put(workerSource.buildEventQPSName(FileFlusher.DISK_FLUSHER_TYPE), diskFlushQPS); + allMetrics.put(workerSource.buildEventLatencyName(FileFlusher.DISK_FLUSHER_TYPE), diskFlushLatency); + allMetrics.put(workerSource.buildDataThroughputName(FileFlusher.DISK_FLUSHER_TYPE), diskFlushThroughput); + allMetrics.put(workerSource.buildDataThroughputName(FileFlusher.DISK_REAL_FLUSHER_TYPE), diskRealFlushThroughput); + allMetrics.put(workerSource.buildEventFailedQPSName(FileFlusher.DISK_REAL_FLUSHER_TYPE), diskFlushFailed); + allMetrics.put(workerSource.buildOpenedFileName(""), diskOpenedFileNum); + + allMetrics.put(workerSource.buildEventQPSName(FileFlusher.HDFS_FLUSHER_TYPE), hdfsFlushQPS); + allMetrics.put(workerSource.buildEventLatencyName(FileFlusher.HDFS_FLUSHER_TYPE), hdfsFlushLatency); + allMetrics.put(workerSource.buildDataThroughputName(FileFlusher.HDFS_FLUSHER_TYPE), hdfsFlushThroughput); + allMetrics.put(workerSource.buildEventFailedQPSName(FileFlusher.HDFS_FLUSHER_TYPE), hdfsFlushFailed); + allMetrics.put(workerSource.buildOpenedFileName("hdfs"), hdfsOpenedFileNum); + } + + public Meter getThroughput(String flusherType) { + if (flusherType.equals(FileFlusher.HDFS_FLUSHER_TYPE)) { + return hdfsFlushThroughput; + } else { + return diskFlushThroughput; + } + } + + public Meter getQPS(String flusherType) { + if (flusherType.equals(FileFlusher.HDFS_FLUSHER_TYPE)) { + return hdfsFlushQPS; + } else { + return diskFlushQPS; + } + } + + public void registerFlushTaskQueueSize(Gauge gauge) { + WorkerSource workerSource = WorkerSource.workerSource(); + allMetrics.put(workerSource.buildFlushQueueName(), gauge); + } + + @Override + public Map getMetrics() { + return allMetrics; + } +} diff --git a/service/src/main/java/com/bytedance/css/service/deploy/worker/FileWriterStatus.java b/service/src/main/java/com/bytedance/css/service/deploy/worker/FileWriterStatus.java new file mode 100644 index 0000000..3548221 --- /dev/null +++ b/service/src/main/java/com/bytedance/css/service/deploy/worker/FileWriterStatus.java @@ -0,0 +1,39 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker; + +public enum FileWriterStatus { + + Writable(0), // Writable, file writer ready to write. + WritableButShouldRotate(1), // Writable, but needs to notify the client to Rotate, generally used for forced writing + ShouldRotate(2), // UnWritable, do not write data, and notify the client to Rotate + UnWritable(3); // UnWritable, file writer catch some exception. + + private final byte value; + + FileWriterStatus(int value) { + assert(value >= 0 && value < 256); + this.value = (byte) value; + } + + public byte getValue() { + return value; + } +} diff --git a/service/src/main/java/com/bytedance/css/service/deploy/worker/FlushTask.java b/service/src/main/java/com/bytedance/css/service/deploy/worker/FlushTask.java new file mode 100644 index 0000000..b32188b --- /dev/null +++ b/service/src/main/java/com/bytedance/css/service/deploy/worker/FlushTask.java @@ -0,0 +1,48 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker; + +import io.netty.buffer.ByteBuf; + +public abstract class FlushTask { + + private final ByteBuf data; + private final FileNotifier notifier; + + public FlushTask(ByteBuf data, FileNotifier notifier) { + this.data = data; + this.notifier = notifier; + } + + public int dataSize() { + return data.readableBytes(); + } + + public boolean hasException() { + return notifier.hasException(); + } + + // flush task's buffer into target file + public abstract void flush(); + + protected FileNotifier getNotifier() { + return notifier; + } +} diff --git a/service/src/main/java/com/bytedance/css/service/deploy/worker/HdfsFileFlushTask.java b/service/src/main/java/com/bytedance/css/service/deploy/worker/HdfsFileFlushTask.java new file mode 100644 index 0000000..b1b6baa --- /dev/null +++ b/service/src/main/java/com/bytedance/css/service/deploy/worker/HdfsFileFlushTask.java @@ -0,0 +1,61 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker; + +import com.codahale.metrics.Timer; +import io.netty.buffer.ByteBuf; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +public class HdfsFileFlushTask extends FlushTask { + + private static final Logger logger = LoggerFactory.getLogger(HdfsFileFlushTask.class); + + private final FSDataOutputStream outputStream; + private final ByteBuf data; + + public HdfsFileFlushTask(ByteBuf data, FileNotifier notifier, FSDataOutputStream outputStream) { + super(data, notifier); + this.outputStream = outputStream; + this.data = data; + } + + @Override + public void flush() { + if (!hasException()) { + try { + Timer.Context timer = FileWriterMetrics.instance().hdfsFlushLatency.time(); + data.getBytes(data.readerIndex(), outputStream, data.readableBytes()); + timer.stop(); + } catch (IOException ex) { + FileWriterMetrics.instance().hdfsFlushFailed.mark(); + logger.error("HdfsFileFlushTask flushed failed", ex); + getNotifier().setException(ex); + } finally { + if (data != null) { + data.release(); + } + } + } + } +} diff --git a/service/src/main/java/com/bytedance/css/service/deploy/worker/HdfsFileWriter.java b/service/src/main/java/com/bytedance/css/service/deploy/worker/HdfsFileWriter.java new file mode 100644 index 0000000..6ca20b2 --- /dev/null +++ b/service/src/main/java/com/bytedance/css/service/deploy/worker/HdfsFileWriter.java @@ -0,0 +1,300 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker; + +import com.bytedance.css.common.exception.AlreadyClosedException; +import com.bytedance.css.common.exception.EpochShouldRotateException; +import com.bytedance.css.common.protocol.ShuffleMode; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.CompositeByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.util.ReferenceCountUtil; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +public final class HdfsFileWriter implements FileWriter { + + private static final Logger logger = LoggerFactory.getLogger(HdfsFileWriter.class); + private static final long WAIT_INTERVAL_MS = 20; + private static final int COMPOSITE_BYTE_BUF_MAX_NUM = 32; + private final FileNotifier notifier = new FileNotifier(); + + private final FileSystem fs; + private final Path path; + private final FSDataOutputStream outputStream; + private volatile boolean closed; + private volatile boolean rotated; + + private final AtomicInteger numPendingWrites = new AtomicInteger(); + private long bytesFlushed = 0L; + private long bytesSubmitted = 0L; + + private final FileFlusher flusher; + private volatile boolean testSubmitFlushTaskException = false; + private CompositeByteBuf compositeByteBuf = null; + + private final long timeoutMs; + private final long flushBufferSize; + private final long epochRotateThreshold; + + public HdfsFileWriter( + FileSystem fs, + Path path, + FSDataOutputStream outputStream, + FileFlusher flusher, + long timeoutMs, + long flushBufferSize, + long epochRotateThreshold) { + this.fs = fs; + this.path = path; + this.outputStream = outputStream; + this.flusher = flusher; + this.timeoutMs = timeoutMs; + this.flushBufferSize = flushBufferSize; + this.epochRotateThreshold = epochRotateThreshold; + FileWriterMetrics.instance().hdfsOpenedFileNum.inc(); + } + + private void submitFlushTask(FlushTask task) throws IOException { + if (testSubmitFlushTaskException || !flusher.submitTask(task, timeoutMs)) { + String msg = String.format("HdfsFileWriter submit flush task timeout with file %s.", path); + IOException e = new IOException(msg); + notifier.setException(e); + throw e; + } + } + + private void flushBuffer(ByteBuf data) throws IOException { + ByteBuf toBeFlushed = data == null ? compositeByteBuf : data; + notifier.getNumPendingFlushes().incrementAndGet(); + int numBytes = toBeFlushed.readableBytes(); + FlushTask task = new HdfsFileFlushTask(toBeFlushed, notifier, outputStream); + submitFlushTask(task); + bytesFlushed += numBytes; + } + + private void waitOnNoPending(AtomicInteger counter) throws IOException { + long waitTime = timeoutMs; + while (counter.get() > 0 && waitTime > 0) { + try { + notifier.checkException(); + TimeUnit.MILLISECONDS.sleep(WAIT_INTERVAL_MS); + } catch (InterruptedException e) { + IOException ioe = new IOException(e); + notifier.setException(ioe); + throw ioe; + } + waitTime -= WAIT_INTERVAL_MS; + } + if (counter.get() > 0) { + String msg = String.format("HdfsFileWriter wait pending actions timeout with %s .", path); + IOException ioe = new IOException(msg); + notifier.setException(ioe); + throw ioe; + } + notifier.checkException(); + } + + @Override + public ArrayList getChunkOffsets() { + throw new UnsupportedOperationException("HdfsFileWriter does not support chunk split."); + } + + @Override + public void write(ByteBuf data, int mapperId, boolean ignoreRotate) throws IOException { + if (!ignoreRotate & rotated) { + // normally we don't hit this code branch, since we already call FileWriter.shouldRotate + // before write data. Just to in case + String msg = String.format("HdfsFileWriter with %s rotated.", path); + logger.warn(msg); + throw new EpochShouldRotateException(msg); + } else if (closed) { + String msg = String.format("HdfsFileWriter write with %s closed.", path); + logger.warn(msg); + throw new AlreadyClosedException(msg); + } + + synchronized (this) { + notifier.checkException(); + + if (compositeByteBuf == null) { + compositeByteBuf = Unpooled.compositeBuffer(COMPOSITE_BYTE_BUF_MAX_NUM); + } + final int numBytes = data.readableBytes(); + data.retain(); + + try { + if (flushBufferSize < numBytes) { + String msg = String.format("HdfsFileWriter Flush giant batch with %s numBytes %s.", path, numBytes); + logger.debug(msg); + flushBuffer(data); + } else { + // buffer not enough for data batch, trigger flush buffer to release memory + if (compositeByteBuf.readableBytes() + numBytes >= flushBufferSize) { + flushBuffer(null); + compositeByteBuf = Unpooled.compositeBuffer(COMPOSITE_BYTE_BUF_MAX_NUM); + } + + compositeByteBuf.addComponent(true, data); + } + } catch (IOException e) { + // here we only need to release data. compositeByteBuf will release by close & destroy. + ReferenceCountUtil.safeRelease(data); + throw e; + } + + bytesSubmitted += numBytes; + if (bytesSubmitted > epochRotateThreshold) { + rotated = true; + flushBuffer(null); + compositeByteBuf = Unpooled.compositeBuffer(COMPOSITE_BYTE_BUF_MAX_NUM); + String msg = String.format("HdfsFileWriter for %s enter rotate state.", path); + logger.warn(msg); + } + + numPendingWrites.decrementAndGet(); + } + } + + @Override + public long close() throws IOException { + if (closed) { + // if reallocation happens, the old partition will be closed in advanced + // to save StageEnd time. + checkException(); + String msg = String.format("HdfsFileWriter with %s closed.", path); + logger.warn(msg); + return bytesFlushed; + } + + try { + waitOnNoPending(numPendingWrites); + closed = true; + + synchronized (this) { + if (compositeByteBuf != null && compositeByteBuf.readableBytes() > 0) { + flushBuffer(null); + compositeByteBuf = null; + } + } + + waitOnNoPending(notifier.getNumPendingFlushes()); + } finally { + if (compositeByteBuf != null) { + ReferenceCountUtil.safeRelease(compositeByteBuf); + compositeByteBuf = null; + } + outputStream.close(); + FileWriterMetrics.instance().hdfsOpenedFileNum.dec(); + } + + return bytesFlushed; + } + + @Override + public void destroy() { + if (!closed) { + closed = true; + String msg = String.format("HdfsFileWriter destroyed with path %s.", path); + logger.warn(msg); + try { + outputStream.close(); + FileWriterMetrics.instance().hdfsOpenedFileNum.dec(); + } catch (IOException e) { + String error = String.format("HdfsFileWriter close channel failed with %s", path); + logger.warn(error, e); + } finally { + if (compositeByteBuf != null) { + ReferenceCountUtil.safeRelease(compositeByteBuf); + compositeByteBuf = null; + } + } + } + try { + fs.delete(path, true); + } catch (IOException e) { + logger.error("HdfsFileWriter failed to delete hdfs file {}.", path); + } + } + + @Override + public void setException() { + notifier.setException(new IOException("ForTestOnly")); + } + + @Override + public void setSubmitQueueFullException(boolean exception) { + testSubmitFlushTaskException = true; + } + + @Override + public CompositeByteBuf getCompositeByteBuf() { + return compositeByteBuf; + } + + @Override + public void checkException() throws IOException { + notifier.checkException(); + } + + @Override + public void incrementPendingWrites() { + numPendingWrites.incrementAndGet(); + } + + @Override + public void decrementPendingWrites() { + numPendingWrites.decrementAndGet(); + } + + @Override + public ShuffleMode getShuffleMode() { + return ShuffleMode.HDFS; + } + + @Override + public String getFilePath() { + return path.toUri().toString(); + } + + @Override + public long getFileLength() { + throw new UnsupportedOperationException("HdfsFileWriter does not support getFileLength api."); + } + + @Override + public File getFile() { + throw new UnsupportedOperationException("HdfsFileWriter does not support getFile api."); + } + + @Override + public boolean shouldRotate() { + return rotated; + } +} diff --git a/service/src/main/java/com/bytedance/css/service/deploy/worker/WritablePartitionInfo.java b/service/src/main/java/com/bytedance/css/service/deploy/worker/WritablePartitionInfo.java new file mode 100644 index 0000000..0c3c491 --- /dev/null +++ b/service/src/main/java/com/bytedance/css/service/deploy/worker/WritablePartitionInfo.java @@ -0,0 +1,38 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker; + +import com.bytedance.css.common.protocol.PartitionInfo; + +import java.io.IOException; + +public class WritablePartitionInfo extends PartitionInfo { + + private final transient FileWriter fileWriter; + + public WritablePartitionInfo(PartitionInfo partitionInfo, FileWriter fileWriter) { + super(partitionInfo); + this.fileWriter = fileWriter; + } + + public FileWriter getFileWriter() throws IOException { + return this.fileWriter; + } +} diff --git a/service/src/main/java/com/bytedance/css/service/deploy/worker/handler/FetchDataHandler.java b/service/src/main/java/com/bytedance/css/service/deploy/worker/handler/FetchDataHandler.java new file mode 100644 index 0000000..fcf030b --- /dev/null +++ b/service/src/main/java/com/bytedance/css/service/deploy/worker/handler/FetchDataHandler.java @@ -0,0 +1,26 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker.handler; + +import com.bytedance.css.network.server.CssFileInfo; + +public interface FetchDataHandler { + CssFileInfo handleOpenStreamRequest(String shuffleKey, String filePath); +} diff --git a/service/src/main/java/com/bytedance/css/service/deploy/worker/handler/FetchDataRpcHandler.java b/service/src/main/java/com/bytedance/css/service/deploy/worker/handler/FetchDataRpcHandler.java new file mode 100644 index 0000000..d7ce0b7 --- /dev/null +++ b/service/src/main/java/com/bytedance/css/service/deploy/worker/handler/FetchDataRpcHandler.java @@ -0,0 +1,162 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker.handler; + +import com.bytedance.css.common.exception.CssException; +import com.bytedance.css.network.client.RpcResponseCallback; +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.protocol.shuffle.BlockTransferMessage; +import com.bytedance.css.network.protocol.shuffle.OpenStream; +import com.bytedance.css.network.protocol.shuffle.StreamHandle; +import com.bytedance.css.network.server.*; +import com.bytedance.css.network.util.TransportConf; +import com.bytedance.css.service.deploy.worker.WorkerSource; +import com.codahale.metrics.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; + +public class FetchDataRpcHandler extends RpcHandler { + + private static final Logger logger = LoggerFactory.getLogger(FetchDataRpcHandler.class); + + private final TransportConf conf; + private final FetchDataHandler handler; + private final OneForOneStreamManager streamManager; + private final FetchMetrics metrics; + + public FetchDataRpcHandler(TransportConf conf, FetchDataHandler handler) { + this.conf = conf; + this.handler = handler; + streamManager = new OneForOneStreamManager(); + metrics = new FetchMetrics(); + streamManager.chunkFetchMetrics = metrics; + } + + @Override + public void receive(TransportClient client, ByteBuffer message, RpcResponseCallback callback) { + BlockTransferMessage msg = BlockTransferMessage.Decoder.fromByteBuffer(message); + OpenStream openStream = (OpenStream) msg; + String shuffleKey = openStream.shuffleKey; + String filePath = openStream.filePath; + int chunkIndex = openStream.initChunkIndex; + CssFileInfo fileInfo = handler.handleOpenStreamRequest(shuffleKey, filePath); + if (fileInfo != null) { + try { + CssManagedBufferIterator iterator = new CssManagedBufferIterator(fileInfo, conf); + iterator.setInitIndex(chunkIndex); + long streamId = streamManager.registerStream(client.getClientId(), iterator, client.getChannel()); + streamManager.setStreamStateCurIndex(streamId, chunkIndex); + + if (fileInfo.numChunks == 0) { + logger.warn(String.format("Zero numChunks for shuffle %s filePath %s", shuffleKey, filePath)); + } + StreamHandle streamHandle = new StreamHandle(streamId, fileInfo.numChunks); + callback.onSuccess(streamHandle.toByteBuffer()); + } catch (IOException ex) { + String errorMsg = String.format("OpenStream failed for shuffle %s filePath %s", shuffleKey, filePath); + logger.error(errorMsg, ex); + callback.onFailure(new CssException(errorMsg, ex)); + } + } else { + callback.onFailure(new FileNotFoundException()); + } + } + + @Override + public void channelActive(TransportClient client) { + metrics.fetchConnection.inc(); + logger.debug("channel active " + client.getSocketAddress()); + } + + @Override + public void channelInactive(TransportClient client) { + metrics.fetchConnection.dec(); + logger.debug("channel Inactive " + client.getSocketAddress()); + } + + @Override + public void exceptionCaught(Throwable cause, TransportClient client) { + logger.debug("exception caught " + cause + " " + client.getSocketAddress()); + } + + @Override + public StreamManager getStreamManager() { + return streamManager; + } + + /** + * A simple class to wrap all fetch service wrapper metrics + */ + private class FetchMetrics extends ChunkFetchMetrics implements MetricSet { + private final Map allMetrics; + private final Counter fetchConnection = new Counter(); + + // chunk fetch metrics + private final Meter fetchDataQPS = new Meter(); + private final Timer fetchDataLatency = new Timer(); + private final Meter fetchDataThroughput = new Meter(); + private final Meter fetchDataFailed = new Meter(); + + private FetchMetrics() { + allMetrics = new HashMap<>(); + allMetrics.put(WorkerSource.workerSource().buildConnectionName("fetch"), fetchConnection); + + allMetrics.put(WorkerSource.workerSource().buildEventQPSName("FetchData"), fetchDataQPS); + allMetrics.put(WorkerSource.workerSource().buildEventLatencyName("FetchData"), fetchDataLatency); + allMetrics.put(WorkerSource.workerSource().buildDataThroughputName("FetchData"), fetchDataThroughput); + allMetrics.put(WorkerSource.workerSource().buildEventFailedQPSName("FetchData"), fetchDataFailed); + } + + @Override + public Meter getChunkFetchQps() { + return fetchDataQPS; + } + + @Override + public Timer getChunkFetchLatency() { + return fetchDataLatency; + } + + @Override + public Meter getChunkFetchFailedQPS() { + return fetchDataFailed; + } + + @Override + public Meter getChunkFetchThroughput() { + return fetchDataThroughput; + } + + @Override + public Map getMetrics() { + return allMetrics; + } + } + + public MetricSet getAllMetrics() { + return metrics; + } +} diff --git a/service/src/main/java/com/bytedance/css/service/deploy/worker/handler/PushDataHandler.java b/service/src/main/java/com/bytedance/css/service/deploy/worker/handler/PushDataHandler.java new file mode 100644 index 0000000..8d8f3d1 --- /dev/null +++ b/service/src/main/java/com/bytedance/css/service/deploy/worker/handler/PushDataHandler.java @@ -0,0 +1,27 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker.handler; + +import com.bytedance.css.network.client.RpcResponseCallback; +import com.bytedance.css.network.protocol.BatchPushDataRequest; + +public interface PushDataHandler { + void handleBatchPushDataRequest(BatchPushDataRequest batchPushData, RpcResponseCallback callback); +} diff --git a/service/src/main/java/com/bytedance/css/service/deploy/worker/handler/PushDataRpcHandler.java b/service/src/main/java/com/bytedance/css/service/deploy/worker/handler/PushDataRpcHandler.java new file mode 100644 index 0000000..e06c23f --- /dev/null +++ b/service/src/main/java/com/bytedance/css/service/deploy/worker/handler/PushDataRpcHandler.java @@ -0,0 +1,155 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker.handler; + +import com.bytedance.css.network.client.RpcResponseCallback; +import com.bytedance.css.network.client.TransportClient; +import com.bytedance.css.network.protocol.BatchPushDataRequest; +import com.bytedance.css.network.server.OneForOneStreamManager; +import com.bytedance.css.network.server.RpcHandler; +import com.bytedance.css.network.server.StreamManager; +import com.bytedance.css.network.util.TransportConf; +import com.bytedance.css.service.deploy.worker.WorkerSource; +import com.codahale.metrics.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +public class PushDataRpcHandler extends RpcHandler { + + private static final Logger logger = LoggerFactory.getLogger(PushDataRpcHandler.class); + + private final TransportConf conf; + private final PushDataHandler handler; + private final OneForOneStreamManager streamManager; + private final PushMetrics metrics; + + public PushDataRpcHandler(TransportConf conf, PushDataHandler handler) { + this.conf = conf; + this.handler = handler; + streamManager = new OneForOneStreamManager(); + metrics = new PushMetrics(); + } + + @Override + public void receive(TransportClient client, ByteBuffer message, RpcResponseCallback callback) { + throw new UnsupportedOperationException("PushDataRpcHandler"); + } + + @Override + public StreamManager getStreamManager() { + return streamManager; + } + + @Override + public void receiveBatchPushDataRequest( + TransportClient client, + BatchPushDataRequest req, + RpcResponseCallback callback) { + metrics.pushDataQPS.mark(); + final Timer.Context latencyTimer = metrics.pushDataLatency.time(); + try { + handler.handleBatchPushDataRequest(req, new WrapPushMetricsCallBack(metrics, callback)); + } finally { + metrics.pushDataE2ELatency.update(System.currentTimeMillis() - req.clientStartTime, TimeUnit.MILLISECONDS); + latencyTimer.stop(); + } + } + + @Override + public void channelActive(TransportClient client) { + metrics.pushConnection.inc(); + } + + @Override + public void channelInactive(TransportClient client) { + metrics.pushConnection.dec(); + } + + public static class WrapPushMetricsCallBack implements RpcResponseCallback { + private RpcResponseCallback callback; + private PushMetrics metrics; + + public WrapPushMetricsCallBack(PushMetrics metrics, RpcResponseCallback callback) { + this.metrics = metrics; + this.callback = callback; + } + + public Meter getDataThroughputMetrics() { + return metrics.pushDataThroughput; + } + + @Override + public void onSuccess(ByteBuffer response) { + // Copy response if response ByteBuffer is not Empty. See. onSuccess Doc + if (response.remaining() > 0) { + ByteBuffer wrappedResponse = ByteBuffer.allocate(response.remaining()); + wrappedResponse.put(response); + wrappedResponse.flip(); + callback.onSuccess(wrappedResponse); + } else { + callback.onSuccess(response); + } + } + + @Override + public void onFailure(Throwable e) { + metrics.pushDataFailed.mark(); + callback.onFailure(e); + } + } + + /** + * A simple class to wrap all push service wrapper metrics + */ + private class PushMetrics implements MetricSet { + private final Map allMetrics; + private final Meter pushDataQPS = new Meter(); + private final Timer pushDataLatency = new Timer(); + private final Timer pushDataE2ELatency = new Timer(); + private final Meter pushDataThroughput = new Meter(); + private final Meter pushDataFailed = new Meter(); + + private final Counter pushConnection = new Counter(); + + private PushMetrics() { + allMetrics = new HashMap<>(); + allMetrics.put(WorkerSource.workerSource().buildEventQPSName("PushData"), pushDataQPS); + allMetrics.put(WorkerSource.workerSource().buildEventLatencyName("PushData"), pushDataLatency); + allMetrics.put(WorkerSource.workerSource().buildEventLatencyName("PushDataE2E"), pushDataE2ELatency); + allMetrics.put(WorkerSource.workerSource().buildDataThroughputName("PushData"), pushDataThroughput); + allMetrics.put(WorkerSource.workerSource().buildEventFailedQPSName("PushData"), pushDataFailed); + allMetrics.put(WorkerSource.workerSource().buildConnectionName("push"), pushConnection); + } + + @Override + public Map getMetrics() { + return allMetrics; + } + } + + public MetricSet getAllMetrics() { + return metrics; + } +} diff --git a/service/src/main/java/com/bytedance/css/service/deploy/worker/handler/RecycleShuffleHandler.java b/service/src/main/java/com/bytedance/css/service/deploy/worker/handler/RecycleShuffleHandler.java new file mode 100644 index 0000000..a714a9a --- /dev/null +++ b/service/src/main/java/com/bytedance/css/service/deploy/worker/handler/RecycleShuffleHandler.java @@ -0,0 +1,27 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker.handler; + +import java.util.HashSet; + +public interface RecycleShuffleHandler { + void recycleShuffle(HashSet expiredShuffleKeys); + void recycleApplication(HashSet expiredApplicationIds); +} diff --git a/service/src/main/resources/css-metrics.properties b/service/src/main/resources/css-metrics.properties new file mode 100644 index 0000000..077269a --- /dev/null +++ b/service/src/main/resources/css-metrics.properties @@ -0,0 +1,66 @@ +# +# Copyright 2022 Bytedance Inc. +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +*.sink.bytedance.class=com.bytedance.css.common.metrics.sink.BytedanceSink + +## Examples +# Enable JmxSink for all instances by class name +#*.sink.jmx.class=com.bytedance.css.common.metrics.sink.JmxSink + +# Enable ConsoleSink for all instances by class name +#*.sink.console.class=com.bytedance.css.common.metrics.sink.ConsoleSink + +# Enable StatsdSink for all instances by class name +#*.sink.statsd.class=com.bytedance.css.common.metrics.sink.StatsdSink +#*.sink.statsd.prefix=spark + +# Polling period for the ConsoleSink +#*.sink.console.period=10 +# Unit of the polling period for the ConsoleSink +#*.sink.console.unit=seconds + +# Polling period for the ConsoleSink specific for the master instance +#master.sink.console.period=15 +# Unit of the polling period for the ConsoleSink specific for the master +# instance +#master.sink.console.unit=seconds + +# Enable CsvSink for all instances by class name +#*.sink.csv.class=com.bytedance.css.common.metrics.sink.CsvSink + +# Polling period for the CsvSink +#*.sink.csv.period=1 +# Unit of the polling period for the CsvSink +#*.sink.csv.unit=minutes + +# Polling directory for CsvSink +#*.sink.csv.directory=/tmp/ + +# Polling period for the CsvSink specific for the worker instance +#worker.sink.csv.period=10 +# Unit of the polling period for the CsvSink specific for the worker instance +#worker.sink.csv.unit=minutes + +# Enable Slf4jSink for all instances by class name +#*.sink.slf4j.class=com.bytedance.css.common.metrics.sink.Slf4jSink + +# Polling period for the Slf4JSink +#*.sink.slf4j.period=1 +# Unit of the polling period for the Slf4jSink +#*.sink.slf4j.unit=minutes diff --git a/service/src/main/resources/log4j-defaults.properties b/service/src/main/resources/log4j-defaults.properties new file mode 100644 index 0000000..3bc9551 --- /dev/null +++ b/service/src/main/resources/log4j-defaults.properties @@ -0,0 +1,50 @@ +# +# Copyright 2022 Bytedance Inc. +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the console +log4j.rootCategory=INFO, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + +# Set the default spark-shell log level to WARN. When running the spark-shell, the +# log level for this class is used to overwrite the root logger's log level, so that +# the user can have different defaults for the shell and regular Spark apps. +log4j.logger.org.apache.spark.repl.Main=WARN + +# Settings to quiet third party logs that are too verbose +log4j.logger.org.spark_project.jetty=WARN +log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO +log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO +log4j.logger.org.apache.parquet=ERROR +log4j.logger.parquet=ERROR + +# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support +log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL +log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR + +log4j.rootLogger=INFO, file +log4j.appender.file=org.apache.log4j.RollingFileAppender +log4j.appender.file.file=${css.log.dir}/${css.log.filename} +log4j.appender.file.maxFileSize=524288000 +log4j.appender.file.maxBackupIndex=6 +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.conversionPattern=%d{ISO8601} %p [%t] %c: %m%n diff --git a/service/src/main/scala/com/bytedance/css/impl/CssShuffleContextImpl.scala b/service/src/main/scala/com/bytedance/css/impl/CssShuffleContextImpl.scala new file mode 100644 index 0000000..f7a7cb3 --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/impl/CssShuffleContextImpl.scala @@ -0,0 +1,95 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.impl + +import java.util + +import scala.collection.JavaConverters._ + +import com.bytedance.css.api.CssShuffleContext +import com.bytedance.css.common.CssConf +import com.bytedance.css.service.deploy.master.Master + +class CssShuffleContextImpl extends CssShuffleContext { + + private var master: Master = null + private val cssConf: CssConf = new CssConf() + + override def startMaster(host: String, port: Int, confMap: util.Map[String, String]): Unit = { + if (master == null) { + this.synchronized { + if (master == null) { + confMap.asScala.foreach(pair => cssConf.set(pair._1, pair._2)) + master = Master.getOrCreate(host, port, cssConf) + } + } + } + } + + override def stopMaster(): Unit = { + if (master != null) { + master.stop() + } + } + + override def getMasterHost: String = { + if (master == null) { + throw new Exception("Css Master is not created yet.") + } else { + master.rpcEnv.address.host + } + } + + override def getMasterPort: Int = { + if (master == null) { + throw new Exception("Css Master is not created yet.") + } else { + master.rpcEnv.address.port + } + } + + override def allocateWorkerIfNeeded(numWorkers: Int): Unit = { + if (master == null) { + throw new Exception("Css Master is not created yet.") + } else { + // async allocate numWorkers for current application + val actualNumWorkers = Math.min( + Math.max( + 2, + (numWorkers * CssConf.workerAllocateExtraRatio(cssConf)).toInt), + CssConf.maxAllocateWorker(cssConf)) + master.getShuffleWorkerManager().workerProvider.allocate(actualNumWorkers) + } + } + + override def waitUntilShuffleCommitted(appId: String, shuffleId: Int): Unit = { + if (master == null) { + throw new Exception("Css Master is not created yet.") + } else { + // sync await until shuffle stage ended. + master.getShuffleStageManager().finishShuffleStage(appId, shuffleId, true) + } + } + + override def eagerDestroyShuffle(appId: String, shuffleId: Int): Unit = { + // TODO this API is mainly for shuffle re-computation + // we need to destroy current shuffle meta and data, before we can actually retry shuffle. + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/common/BaseSource.scala b/service/src/main/scala/com/bytedance/css/service/deploy/common/BaseSource.scala new file mode 100644 index 0000000..c2d56e7 --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/common/BaseSource.scala @@ -0,0 +1,96 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.common + +import scala.collection.mutable + +import com.bytedance.css.common.metrics.source.Source +import com.codahale.metrics.{Meter, MetricRegistry, MetricSet, Timer} + +/** + * Server Side Base Metrics Source. + */ +abstract class BaseSource( + namespace: String, + serverId: String, + baseMetricRegistry: MetricRegistry = new MetricRegistry) extends Source { + + override def metricRegistry: MetricRegistry = baseMetricRegistry + + private val qpsMap = mutable.Map[String, Meter]() + private val latencyMap = mutable.Map[String, Timer]() + + def cssMetricsPrefix: String = s"namespace=$namespace|server=$serverId" + + def registerMetricSet(metricSet: MetricSet): Unit = { + metricRegistry.registerAll(metricSet) + } + + def withEventMetrics(eventName: String)(body: => Unit): Unit = { + qpsMap(eventName).mark() + val timer = latencyMap(eventName).time() + body + timer.stop() + } + + def buildEventQPSName(eventName: String): String = { + cssMetricsPrefix + s"#event.${eventName}.qps" + } + + def buildEventFailedQPSName(eventName: String): String = { + cssMetricsPrefix + s"#event.${eventName}.failed" + } + + def buildEventLatencyName(eventName: String): String = { + cssMetricsPrefix + s"#event.${eventName}.latency" + } + + def buildDataThroughputName(dataType: String): String = { + cssMetricsPrefix + s"#data.${dataType}.throughput" + } + + def buildConnectionName(dataType: String): String = { + cssMetricsPrefix + s"#worker.${dataType}.connection" + } + + def buildOpenedFileName(fileType: String = ""): String = { + if (fileType.isEmpty) { + cssMetricsPrefix + s"#worker.opened.file" + } else { + cssMetricsPrefix + s"#worker.opened.${fileType}.file" + } + } + + def buildFlushQueueName(): String = { + cssMetricsPrefix + s"#worker.flush.queue.size" + } + + protected def getEventQPS(eventName: String): Meter = { + val meter = metricRegistry.meter(buildEventQPSName(eventName)) + qpsMap.put(eventName, meter) + meter + } + + protected def getEventLatency(eventName: String): Timer = { + val timer = metricRegistry.timer(buildEventLatencyName(eventName)) + latencyMap.put(eventName, timer) + timer + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/common/HeartbeatReceiver.scala b/service/src/main/scala/com/bytedance/css/service/deploy/common/HeartbeatReceiver.scala new file mode 100644 index 0000000..2bf787e --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/common/HeartbeatReceiver.scala @@ -0,0 +1,100 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.common + +import java.util + +import scala.collection.JavaConverters._ + +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.protocol.CssRpcMessage._ +import com.bytedance.css.common.protocol.CssStatusCode +import com.bytedance.css.common.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} +import com.bytedance.css.service.deploy.master.{Master} +import com.bytedance.css.service.deploy.worker.WorkerInfo + +class HeartbeatReceiver( + val rpcEnv: RpcEnv, + master: Master) extends ThreadSafeRpcEndpoint with Logging { + + private lazy val shuffleAppManager = master.getShuffleAppManager() + private lazy val shuffleWorkerManager = master.getShuffleWorkerManager() + + override def onStart(): Unit = { + } + + override def receive: PartialFunction[Any, Unit] = { + case HeartbeatFromApp(appId) => + shuffleAppManager.handleApplicationHeartbeat(appId) + } + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case req: RegisterWorker => + handleRegisterWorker(context, req: RegisterWorker) + + case req: HeartbeatFromWorker => + handleHeartBeatFromWorker(context, req: HeartbeatFromWorker) + } + + def handleRegisterWorker( + context: RpcCallContext, + req: RegisterWorker): Unit = { + val workerInfo = new WorkerInfo(req.name, req.host, req.rpcPort, req.pushPort, req.fetchPort, req.workerRef) + val registerStatus = shuffleWorkerManager.handleRegisterWorker(workerInfo.name, workerInfo) + if (!registerStatus) { + context.reply(RegisterWorkerResponse(CssStatusCode.Failed)) + } else { + context.reply(RegisterWorkerResponse(CssStatusCode.Success)) + } + } + + def handleHeartBeatFromWorker( + context: RpcCallContext, + req: HeartbeatFromWorker): Unit = { + val handled = shuffleWorkerManager.handleWorkerHeartBeat(req.name, req.rttAvgStat, req.shuffleKeys) + val expiredShuffleKeys = new util.HashSet[String] + val expiredAppIds = new util.HashSet[String]() + if (handled) { + req.shuffleKeys.asScala.foreach { shuffleKey => + val splits = shuffleKey.split("-") + val appId = splits.dropRight(1).mkString("-") + val shuffleId = splits.last.toInt + if (!shuffleAppManager.shuffleStageManager.validateRegisterShuffle(appId, shuffleId)) { + expiredShuffleKeys.add(shuffleKey) + } + } + // appId must not exist in all shuffle relate event and clear after expired shuffle keys for app clear + val worker = shuffleWorkerManager.workerProvider.get(req.name) + if (!worker.appFinishSet.isEmpty) { + worker.appFinishSet.asScala.foreach { appId => + if (shuffleAppManager.shuffleStageManager.getAppShuffle(appId).isEmpty && + !expiredShuffleKeys.asScala.exists(_.startsWith(appId))) { + expiredAppIds.add(appId) + } + } + worker.appFinishSet.removeAll(expiredAppIds) + } + } + context.reply(HeartbeatResponse(expiredShuffleKeys, expiredAppIds)) + } + + override def onStop(): Unit = { + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/common/ScheduledManager.scala b/service/src/main/scala/com/bytedance/css/service/deploy/common/ScheduledManager.scala new file mode 100644 index 0000000..2665f68 --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/common/ScheduledManager.scala @@ -0,0 +1,86 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.common + +import java.util.concurrent.{ConcurrentHashMap, ScheduledFuture, TimeUnit} + +import scala.collection.JavaConverters._ + +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.util.{ThreadUtils, Utils} + +/** + * Creates a scheduled manager which will hold all scheduled task. + * + * @param name the name for scheduled manager. + * @param numThreads the num thread for scheduled manager. + */ +private[deploy] class ScheduledManager(name: String, numThreads: Int) extends Logging { + + // Executor for the forward message task + private val executors = ThreadUtils.newDaemonThreadPoolScheduledExecutor(name, numThreads) + private val taskMap = new ConcurrentHashMap[String, TaskInfo]() + + case class TaskInfo( + name: String, + scheduledTask: () => Unit, + initialDelay: Long, + period: Long, + var scheduledFuture: ScheduledFuture[_] = null) + + /** Schedules a task to run forward message. */ + def addScheduledTask( + name: String, + scheduledTask: () => Unit, + initialDelay: Long, + period: Long): Unit = { + taskMap.put(name, TaskInfo(name, scheduledTask, initialDelay, period)) + } + + /** Starts the scheduled manager. */ + def start(): Unit = { + taskMap.asScala.foreach { entry => + val taskName = entry._1 + val taskInfo = entry._2 + val task = new Runnable() { + override def run(): Unit = { + Utils.logUncaughtExceptions(taskInfo.scheduledTask()) + } + } + taskInfo.scheduledFuture = + executors.scheduleAtFixedRate(task, taskInfo.initialDelay, taskInfo.period, TimeUnit.MILLISECONDS) + logInfo(s"start scheduled task with [$taskName]") + } + } + + /** Stops the scheduled manager. */ + def stop(): Unit = { + taskMap.asScala.foreach { entry => + val taskName = entry._1 + val taskInfo = entry._2 + if (taskInfo.scheduledFuture != null) { + taskInfo.scheduledFuture.cancel(true) + logInfo(s"stop scheduled task with [$taskName]") + } + } + executors.shutdown() + executors.awaitTermination(10, TimeUnit.SECONDS) + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/master/AssignStrategy.scala b/service/src/main/scala/com/bytedance/css/service/deploy/master/AssignStrategy.scala new file mode 100644 index 0000000..ebc7f32 --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/master/AssignStrategy.scala @@ -0,0 +1,100 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.protocol.PartitionGroup +import com.bytedance.css.service.deploy.worker.WorkerInfo + +trait AssignStrategy extends Logging { + + /** + * if target numPartition less than worker combination numbers, + * Basically, it can indeed be a relatively small shuffle, + * but we still don’t want each partition to have a group, + * so the efficiency of push data will be relatively poor, + * so in this case, the step size can be defined as numPartition / workerSize. + * If numPartition is smaller than workerSize, then there is no way, + * it can only be a group according to 1 partition. + */ + protected def calculateGroupInfo( + numPartitions: Int, + maxPartitionsPerGroup: Int, + assignableWorkers: Seq[WorkerInfo], + replica: Int = 2): GroupInfo = { + + val possibleTuples = calculatePossibleTuples(assignableWorkers, replica) + val tupleSize = possibleTuples.size + + val groupLength = if (numPartitions < tupleSize) { + Math.max(1, numPartitions / assignableWorkers.size) + } else { + Math.min(numPartitions / tupleSize, maxPartitionsPerGroup) + } + + val groupNum = if (numPartitions % groupLength == 0) { + numPartitions / groupLength + } else { + (numPartitions + groupLength) / groupLength + } + GroupInfo(groupLength, groupNum, possibleTuples) + } + + protected def calculatePossibleTuples( + assignableWorkers: Seq[WorkerInfo], + replica: Int = 2): List[List[Int]] = { + if (replica > assignableWorkers.size) { + throw new RuntimeException("replica num must less than worker num") + } + val indices = assignableWorkers.indices.toList + indices.combinations(replica).toList + } + + /** + * Assign partition group to current shuffle. + */ + def assignPartitionGroup( + numPartitions: Int, + maxPartitionsPerGroup: Int, + assignableWorkers: Seq[WorkerInfo]): List[PartitionGroup] + + /** + * Reallocate a new partition group with epochId which big than old partition group. + */ + def reallocatePartitionGroup( + assignableWorkers: Seq[WorkerInfo], + oldPartitionGroup: PartitionGroup): PartitionGroup +} + +case class GroupInfo(groupLength: Int, groupNum: Int, possibleTuples: List[List[Int]]) + +object AssignStrategy { + + def buildAssignStrategy(conf: CssConf): AssignStrategy = { + val assignStrategy = CssConf.partitionAssignStrategy(conf) + // scalastyle:off caselocale + assignStrategy.toUpperCase match { + case "RANDOM" => new RandomAssignStrategy + case _ => throw new RuntimeException("not support assign strategy: " + assignStrategy) + } + // scalastyle:on caselocale + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/master/Master.scala b/service/src/main/scala/com/bytedance/css/service/deploy/master/Master.scala new file mode 100644 index 0000000..dfc4e74 --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/master/Master.scala @@ -0,0 +1,332 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import java.nio.ByteBuffer +import java.util.concurrent.ConcurrentHashMap + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.metrics.MetricsSystem +import com.bytedance.css.common.protocol._ +import com.bytedance.css.common.protocol.CssRpcMessage._ +import com.bytedance.css.common.rpc._ +import com.bytedance.css.common.rpc.netty.{NettyRpcEnv, RemoteNettyRpcCallContext} +import com.bytedance.css.common.util.Utils +import com.bytedance.css.service.deploy.common.HeartbeatReceiver +import com.bytedance.css.service.deploy.metadata.ExternalShuffleMeta +import org.apache.hadoop.util.ShutdownHookManager + +class Master( + override val rpcEnv: RpcEnv, + val conf: CssConf) + extends RpcEndpoint with Logging { + + private[deploy] val reducerFileGroupsReuseByteBufferCache = + new ConcurrentHashMap[String, ByteBuffer]() + + private lazy val assignStrategy = AssignStrategy.buildAssignStrategy(conf) + private lazy val externalShuffleMeta = ExternalShuffleMeta.create(conf) + private lazy val shuffleTaskManager = new ShuffleTaskManager + private lazy val shuffleWorkerManager = new ShuffleWorkerManager(conf, rpcEnv) + private lazy val shuffleStageManager = new ShuffleStageManagerImpl( + conf, assignStrategy, shuffleTaskManager, shuffleWorkerManager, externalShuffleMeta) + private lazy val shuffleAppManager = new ShuffleAppManager( + conf, shuffleStageManager, shuffleWorkerManager, externalShuffleMeta, self) + + private lazy val metricsSystem = MetricsSystem.createMetricsSystem(MetricsSystem.MASTER, conf) + private lazy val masterSource = MasterSource.create(CssConf.clusterName(conf), rpcEnv.address.host) + + override def onStart(): Unit = { + logInfo(s"Master onStart called.") + shuffleAppManager.start() + shuffleStageManager.start() + shuffleWorkerManager.start() + + metricsSystem.registerSource(masterSource) + metricsSystem.start() + } + + override def onStop(): Unit = { + logInfo(s"Master onStop called.") + metricsSystem.report() + shuffleAppManager.stop() + shuffleStageManager.stop() + shuffleWorkerManager.stop() + metricsSystem.stop() + } + + override def onDisconnected(address: RpcAddress): Unit = { + // The disconnected client could've been either a worker or an app; remove whichever it was + logInfo(s"Client $address got disassociated.") + } + + override def receive: PartialFunction[Any, Unit] = { + case req: StageEnd => + val startMs = System.currentTimeMillis() + masterSource.withEventMetrics("StageEnd") { + handleStageEnd(null, req) + } + logInfo(s"handleStageEnd for ${req.applicationId}-${req.shuffleId} " + + s"take ${System.currentTimeMillis() - startMs} ms.") + } + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case req: RegisterPartitionGroup => + masterSource.withEventMetrics("RegisterPartitionGroup") { + handleRegisterPartitionGroup(context, req) + } + + case req: ReallocatePartitionGroup => + masterSource.withEventMetrics("ReallocatePartitionGroup") { + handleReallocatePartitionGroup(context, req) + } + + case req: MapperEnd => + masterSource.withEventMetrics("MapperEnd") { + handleMapperEnd(context, req) + } + + case req: GetReducerFileGroups => + masterSource.withEventMetrics("GetReducerFileGroups") { + handleGetReducerFileGroups(context, req) + } + + case req: UnregisterShuffle => + handleUnregisterShuffle(context, req) + } + + def handleRegisterPartitionGroup( + context: RpcCallContext, + req: RegisterPartitionGroup): Unit = { + + val registeredPG = shuffleStageManager.registerShuffleStage( + req.applicationId, + req.shuffleId, + req.numMappers, + req.numPartitions, + req.maxPartitionsPerGroup + ) + context.reply(RegisterPartitionGroupResponse(CssStatusCode.Success, registeredPG)) + } + + def handleReallocatePartitionGroup( + context: RpcCallContext, + req: ReallocatePartitionGroup): Unit = { + + val shuffleKey = Utils.getShuffleKey(req.applicationId, req.shuffleId) + + val validRegistered = shuffleStageManager.validateRegisterShuffle(req.applicationId, req.shuffleId) + if (!validRegistered) { + logError(s"[handleReallocatePartitionGroup] shuffle $shuffleKey not registered!") + context.reply(ReallocatePartitionGroupResponse(CssStatusCode.ShuffleNotRegistered, null)) + return + } + + val validMapperEnded = shuffleTaskManager.validateShuffleTaskEnded(shuffleKey, req.mapId) + if (validMapperEnded) { + logError(s"[handleReallocatePartitionGroup] shuffle $shuffleKey mapper ended!") + context.reply(ReallocatePartitionGroupResponse(CssStatusCode.MapEnded, null)) + return + } + + val reallocatePG = shuffleStageManager.reallocateShufflePartition( + req.applicationId, + req.shuffleId, + req.mapId, + req.attemptId, + req.oldPartitionGroup + ) + + context.reply(ReallocatePartitionGroupResponse(CssStatusCode.Success, reallocatePG)) + } + + def handleMapperEnd( + context: RpcCallContext, + req: MapperEnd): Unit = { + + val shuffleKey = Utils.getShuffleKey(req.applicationId, req.shuffleId) + + shuffleTaskManager.shuffleTaskEnded( + shuffleKey, + req.mapId, + req.attemptId, + req.numMappers, + req.epochList, + req.batchBlacklist + ) + + val triggerStageEnd = shuffleTaskManager.validateEffectiveTaskAttempt(shuffleKey, req.mapId, req.attemptId) & + shuffleTaskManager.validateAllTaskEnded(shuffleKey) + + if (triggerStageEnd) { + self.send(StageEnd(req.applicationId, req.shuffleId)) + } + + // reply success + context.reply(MapperEndResponse(CssStatusCode.Success)) + } + + def handleStageEnd( + context: RpcCallContext, + req: StageEnd): Unit = { + + val shuffleKey = Utils.getShuffleKey(req.applicationId, req.shuffleId) + val validRegistered = shuffleStageManager.validateRegisterShuffle(req.applicationId, req.shuffleId) + if (!validRegistered) { + logWarning(s"handleStageEnd for non-register-shuffle $shuffleKey.") + shuffleStageManager.commitShuffleStage(req.applicationId, req.shuffleId, true) + if (context != null) { + context.reply(StageEndResponse(CssStatusCode.ShuffleNotRegistered)) + } + return + } + val dataLost = shuffleStageManager.commitShuffleStage(req.applicationId, req.shuffleId, false) + if (dataLost) { + if (context != null) { + context.reply(StageEndResponse(CssStatusCode.Failed)) + } + } else { + if (context != null) { + context.reply(StageEndResponse(CssStatusCode.Success)) + } + } + } + + def handleGetReducerFileGroups( + context: RpcCallContext, + req: GetReducerFileGroups): Unit = { + val shuffleKey = Utils.getShuffleKey(req.applicationId, req.shuffleId) + var status: CssStatusCode = null + val stageStatus = shuffleStageManager.finishShuffleStage(req.applicationId, req.shuffleId) + stageStatus match { + case NoStageEndStatus => status = CssStatusCode.Failed + case StageEndTimeOutStatus => status = CssStatusCode.Timeout + case StageEndRunningStatus => status = CssStatusCode.Waiting + case StageEndDataLostStatus => status = CssStatusCode.StageEndDataLost + case StageEndFinishStatus => status = { + if (!reducerFileGroupsReuseByteBufferCache.containsKey(shuffleKey)) { + reducerFileGroupsReuseByteBufferCache.synchronized { + if (!reducerFileGroupsReuseByteBufferCache.containsKey(shuffleKey)) { + val stageResult = shuffleStageManager.getShuffleStageResult(req.applicationId, req.shuffleId) + val response = GetReducerFileGroupsResponse( + CssStatusCode.Success, + stageResult.shuffleFileGroup, + stageResult.mapperAttempts, + stageResult.failedPartitionBatches + ) + val byteBuffer = rpcEnv.asInstanceOf[NettyRpcEnv].serialize(response) + reducerFileGroupsReuseByteBufferCache.put(shuffleKey, byteBuffer) + logInfo(s"reduce file groups cache buffer with ${shuffleKey} size ${byteBuffer.remaining()}") + } + } + } + CssStatusCode.Success + } + } + if (status == CssStatusCode.Success) { + // ByteBuffer reused + if (context.isInstanceOf[RemoteNettyRpcCallContext]) { + context.reply(reducerFileGroupsReuseByteBufferCache.get(shuffleKey).duplicate()) + } else { + val stageResult = shuffleStageManager.getShuffleStageResult(req.applicationId, req.shuffleId) + val response = GetReducerFileGroupsResponse( + status, + stageResult.shuffleFileGroup, + stageResult.mapperAttempts, + stageResult.failedPartitionBatches + ) + context.reply(response) + } + } else { + context.reply(GetReducerFileGroupsResponse(status, null, null, null)) + } + } + + def handleUnregisterShuffle( + context: RpcCallContext, + req: UnregisterShuffle): Unit = { + val shuffleKey = Utils.getShuffleKey(req.applicationId, req.shuffleId) + val validStageEnd = + shuffleStageManager.finishShuffleStage(req.applicationId, req.shuffleId) == StageEndFinishStatus + if (!validStageEnd) { + // trigger StageEnd in case resource and file leak. + val startMs = System.currentTimeMillis() + handleStageEnd(null, StageEnd(req.applicationId, req.shuffleId)) + logInfo(s"handleStageEnd(null) for ${req.applicationId}-${req.shuffleId} " + + s"take ${System.currentTimeMillis() - startMs} ms.") + } + + shuffleStageManager.unregisterShuffle(req.applicationId, req.shuffleId) + + logInfo(s"UnregisterShuffle for shuffle $shuffleKey success.") + if (context != null) { + context.reply(UnregisterShuffleResponse(CssStatusCode.Success)) + } + } + + def getShuffleAppManager(): ShuffleAppManager = { + this.shuffleAppManager + } + + def getShuffleStageManager(): ShuffleStageManager = { + this.shuffleStageManager + } + + def getShuffleWorkerManager(): ShuffleWorkerManager = { + this.shuffleWorkerManager + } +} + +object Master extends Logging { + + // for test, local cluster + @volatile + var master: Master = null + + def getOrCreate(conf: CssConf): Master = getOrCreate(Utils.localHostName(), 0, conf) + + def getOrCreate(host: String, port: Int, conf: CssConf): Master = synchronized { + if (master == null) { + val rpcEnv = RpcEnv.create(RpcNameConstants.MASTER_SYS, host, port, conf) + master = new Master(rpcEnv, conf) + rpcEnv.setupEndpoint(RpcNameConstants.MASTER_EP, master) + ShutdownHookManager.get().addShutdownHook(new Thread { + override def run(): Unit = { + logInfo(s"Master shutting down, hook execution.") + if (master != null) { + rpcEnv.shutdown() + } + } + }, 50) + val heartbeatReceiver = new HeartbeatReceiver(rpcEnv, master) + rpcEnv.setupEndpoint(RpcNameConstants.HEARTBEAT, heartbeatReceiver) + master + } else { + master + } + } + + def main(args: Array[String]): Unit = { + val conf = new CssConf() + val masterArgs = new MasterArguments(args, conf) + getOrCreate(masterArgs.host, masterArgs.port, conf).rpcEnv.awaitTermination() + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/master/MasterArguments.scala b/service/src/main/scala/com/bytedance/css/service/deploy/master/MasterArguments.scala new file mode 100644 index 0000000..7115fcd --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/master/MasterArguments.scala @@ -0,0 +1,92 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import scala.annotation.tailrec + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.util.{IntParam, Utils} + +class MasterArguments(args: Array[String], conf: CssConf) { + + var host = Utils.localHostName() + var port = 9099 + var propertiesFile: String = null + + if (System.getenv("CSS_MASTER_HOST") != null) { + host = System.getenv("CSS_MASTER_HOST") + } + if (System.getenv("CSS_MASTER_PORT") != null) { + port = System.getenv("CSS_MASTER_PORT").toInt + } + + parse(args.toList) + + propertiesFile = Utils.loadDefaultCssProperties(conf, propertiesFile) + + @tailrec + private def parse(args: List[String]): Unit = args match { + case ("--host" | "-h") :: value :: tail => + Utils.checkHost(value) + host = value + parse(tail) + + case ("--port" | "-p") :: IntParam(value) :: tail => + port = value + parse(tail) + + case ("--properties-file") :: value :: tail => + propertiesFile = value + parse(tail) + + case ("--local-mode") :: tail => + conf.set("css.local.mode", "true") + parse(tail) + + case ("--conf") :: confKey :: confValue :: tail => + conf.set(confKey, confValue) + parse(tail) + + case ("--help") :: tail => + printUsageAndExit(0) + + case Nil => // No-op + + case _ => + printUsageAndExit(1) + } + + /** + * Print usage and exit JVM with the given exit code. + */ + private def printUsageAndExit(exitCode: Int) { + // scalastyle:off println + System.err.println( + "Usage: Master [options]\n" + + "\n" + + "Options:\n" + + " -h HOST, --host HOST Hostname to listen on\n" + + " -p PORT, --port PORT Port to listen on (default: 9099)\n" + + " --properties-file FILE Path to a custom CSS properties file.\n" + + " Default is conf/css-defaults.conf.") + // scalastyle:on println + System.exit(exitCode) + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/master/MasterSource.scala b/service/src/main/scala/com/bytedance/css/service/deploy/master/MasterSource.scala new file mode 100644 index 0000000..69a2940 --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/master/MasterSource.scala @@ -0,0 +1,64 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.service.deploy.common.BaseSource + +class MasterSource( + namespace: String, + serverId: String) + extends BaseSource(namespace, serverId) { + + override val sourceName = "" + + // RegisterPartitionGroup Qps & Latency + val REGISTER_PARTITION_GROUP = getEventQPS("RegisterPartitionGroup") + val REGISTER_PARTITION_GROUP_LATENCY = getEventLatency("RegisterPartitionGroup") + + // ReallocatePartitionGroup Qps & Latency + val REALLOCATE_PARTITION_GROUP = getEventQPS("ReallocatePartitionGroup") + val REALLOCATE_PARTITION_GROUP_LATENCY = getEventLatency("ReallocatePartitionGroup") + + // MapperEnd Qps & Latency + val MAPPER_END = getEventQPS("MapperEnd") + val MAPPER_END_LATENCY = getEventLatency("MapperEnd") + + // StageEnd Qps & Latency + val STAGE_END = getEventQPS("StageEnd") + val STAGE_END_LATENCY = getEventLatency("StageEnd") + + // GetReducerFileGroups Qps & Latency + val GET_REDUCER_FILE_GROUPS = getEventQPS("GetReducerFileGroups") + val GET_REDUCER_FILE_GROUPS_LATENCY = getEventLatency("GetReducerFileGroups") +} + +object MasterSource extends Logging { + + @volatile var masterSource: MasterSource = _ + + def create(namespace: String, serverId: String): MasterSource = synchronized { + if (masterSource == null) { + masterSource = new MasterSource(namespace, serverId) + logInfo(s"${masterSource.getClass.getName} use cssMetricsPrefix ${masterSource.cssMetricsPrefix}") + } + masterSource + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/master/RandomAssignStrategy.scala b/service/src/main/scala/com/bytedance/css/service/deploy/master/RandomAssignStrategy.scala new file mode 100644 index 0000000..fa91f66 --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/master/RandomAssignStrategy.scala @@ -0,0 +1,79 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import java.util.Random + +import scala.collection.JavaConverters._ + +import com.bytedance.css.common.protocol.{PartitionGroup, WorkerAddress} +import com.bytedance.css.service.deploy.worker.WorkerInfo + +class RandomAssignStrategy extends AssignStrategy { + + lazy val rand = new Random + + override def assignPartitionGroup( + numPartitions: Int, + maxPartitionsPerGroup: Int, + assignableWorkers: Seq[WorkerInfo]): List[PartitionGroup] = { + val groupInfo = calculateGroupInfo(numPartitions, maxPartitionsPerGroup, assignableWorkers) + val groupNum = groupInfo.groupNum + val groupLength = groupInfo.groupLength + val possibleTuples = groupInfo.possibleTuples + + val groupPairs = (0 until groupNum).map(_ => { + val pairIndex = possibleTuples(rand.nextInt(possibleTuples.size)) + val assignWorkers = pairIndex.map(i => assignableWorkers(i)) + .filter(workerInfo => workerInfo.isActive()) + assignWorkers.map(worker => new WorkerAddress(worker.host, worker.pushPort)) + }) + + val partitionGroupList = groupPairs.zipWithIndex.map(f => { + // Register EpochId should all be 0 + new PartitionGroup(f._2, + 0, + f._2 * groupLength, + Math.min((f._2 + 1) * groupLength, numPartitions), + f._1.asJava + ) + }) + partitionGroupList.toList + } + + override def reallocatePartitionGroup( + assignableWorkers: Seq[WorkerInfo], + oldPartitionGroup: PartitionGroup): PartitionGroup = { + val possibleTuples = calculatePossibleTuples(assignableWorkers) + val tmpGroupPair = { + val pairIndex = possibleTuples(rand.nextInt(possibleTuples.size)) + val assignWorkers = pairIndex.map(i => assignableWorkers(i)) + .filter(workerInfo => workerInfo.isActive()) + assignWorkers.map(worker => new WorkerAddress(worker.host, worker.pushPort)) + } + val partitionGroup = new PartitionGroup( + oldPartitionGroup.partitionGroupId, + oldPartitionGroup.epochId + 1, + oldPartitionGroup.startPartition, + oldPartitionGroup.endPartition, + tmpGroupPair.asJava) + partitionGroup + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleAppManager.scala b/service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleAppManager.scala new file mode 100644 index 0000000..0ff255e --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleAppManager.scala @@ -0,0 +1,100 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import java.util +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.protocol.CssRpcMessage.UnregisterShuffle +import com.bytedance.css.common.rpc.RpcEndpointRef +import com.bytedance.css.common.util.ThreadUtils +import com.bytedance.css.service.deploy.common.ScheduledManager +import com.bytedance.css.service.deploy.metadata.ExternalShuffleMeta + +class ShuffleAppManager( + cssConf: CssConf, + val shuffleStageManager: ShuffleStageManager, + shuffleWorkerManager: ShuffleWorkerManager, + externalShuffleMeta: ExternalShuffleMeta, + masterRpcEndpointRef: RpcEndpointRef) extends Logging { + + private val appTimeoutMs = CssConf.appTimeoutMs(cssConf) + private val appLastHeartbeat = new ConcurrentHashMap[String, Long]() + + private val asyncThreadPool = ThreadUtils + .newDaemonCachedThreadPool("ThreadPool for time consuming operations", 16) + val scheduledManager = new ScheduledManager("app-forward-message-thread", 1) + scheduledManager.addScheduledTask("appTimeOutTask", checkApplicationTimeOut, 0, appTimeoutMs / 2) + + def handleApplicationHeartbeat(appId: String): Unit = { + if (!appLastHeartbeat.containsKey(appId)) { + externalShuffleMeta.appCreated(appId) + } + appLastHeartbeat.put(appId, System.currentTimeMillis()) + } + + def start(): Unit = { + scheduledManager.start() + } + + def stop(): Unit = { + externalShuffleMeta.cleanupIfNeeded() + asyncThreadPool.shutdown() + scheduledManager.stop() + } + + private def checkApplicationTimeOut(): Unit = { + val appLostSet = new util.HashSet[String]() + val currentTime = System.currentTimeMillis() + logDebug(s"Check for application timeout with $currentTime") + val keys = appLastHeartbeat.keySet().asScala.toList + keys.foreach { appId => + if (appLastHeartbeat.get(appId) < currentTime - appTimeoutMs) { + logWarning(s"Application $appId timeout, trigger ApplicationLost") + handleApplicationLost(appId) + appLastHeartbeat.remove(appId) + // mark appId to remove + appLostSet.add(appId) + } + } + shuffleWorkerManager.getActiveWorkers().foreach { worker => + worker.appFinishSet.addAll(appLostSet) + } + } + + private def handleApplicationLost(appId: String) : Unit = { + asyncThreadPool.submit(new Runnable { + override def run(): Unit = { + val expiredShuffles = shuffleStageManager.getAppShuffle(appId) + expiredShuffles.foreach { key => + val splits = key.split("-") + val appId = splits.dropRight(1).mkString("-") + val shuffleId = splits.last.toInt + masterRpcEndpointRef.ask(UnregisterShuffle(appId, shuffleId)) + } + } + }) + externalShuffleMeta.appRemoved(appId) + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleStageManager.scala b/service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleStageManager.scala new file mode 100644 index 0000000..e71e59e --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleStageManager.scala @@ -0,0 +1,75 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import java.util + +import com.bytedance.css.common.protocol.{CommittedPartitionInfo, FailedPartitionInfoBatch, PartitionGroup} + +trait ShuffleStageManager { + + def registerShuffleStage( + appId: String, + shuffleId: Int, + numMappers: Int, + numPartitions: Int, + maxPartitionsPerGroup: Int): util.List[PartitionGroup] + + def validateRegisterShuffle(appId: String, shuffleId: Int): Boolean + + def getAppShuffle(appId: String): Set[String] + + def reallocateShufflePartition( + appId: String, + shuffleId: Int, + mapId: Int, + attemptId: Int, + oldPartitionGroup: PartitionGroup): PartitionGroup + + def commitShuffleStage(appId: String, shuffleId: Int, forceFinish: Boolean): Boolean + + def finishShuffleStage(appId: String, shuffleId: Int, wait: Boolean = false): StageStatus + + def getShuffleStageResult(appId: String, shuffleId: Int): StageResult + + def unregisterShuffle(appId: String, shuffleId: Int): Unit + + def start(): Unit + + def stop(): Unit +} + + +sealed trait StageStatus + +case object NoStageEndStatus extends StageStatus + +case object StageEndRunningStatus extends StageStatus + +case object StageEndTimeOutStatus extends StageStatus + +case object StageEndDataLostStatus extends StageStatus + +case object StageEndFinishStatus extends StageStatus + +case class StageResult( + shuffleFileGroup: Array[Array[CommittedPartitionInfo]], + mapperAttempts: Array[Int], + failedPartitionBatches: util.HashSet[FailedPartitionInfoBatch]) diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleStageManagerImpl.scala b/service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleStageManagerImpl.scala new file mode 100644 index 0000000..3cea36c --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleStageManagerImpl.scala @@ -0,0 +1,290 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import java.util +import java.util.concurrent.{ConcurrentHashMap, TimeUnit} + +import scala.collection.JavaConverters._ +import scala.concurrent.duration.{FiniteDuration, MILLISECONDS} + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.protocol.{CommittedPartitionInfo, PartitionGroup} +import com.bytedance.css.common.protocol.CssRpcMessage.{CommitFiles, CommitFilesResponse} +import com.bytedance.css.common.rpc.RpcTimeout +import com.bytedance.css.common.util.{ThreadUtils, Utils} +import com.bytedance.css.common.util.Collections._ +import com.bytedance.css.service.deploy.common.ScheduledManager +import com.bytedance.css.service.deploy.metadata.ExternalShuffleMeta +import io.netty.util.internal.ConcurrentSet + +class ShuffleStageManagerImpl( + cssConf: CssConf, + assignStrategy: AssignStrategy, + shuffleTaskManager: ShuffleTaskManager, + shuffleWorkerManager: ShuffleWorkerManager, + externalShuffleMeta: ExternalShuffleMeta) extends ShuffleStageManager with Logging { + + // store all registered shuffle and its partition group. + // key appId-shuffleId + private[deploy] val registeredPartitionGroup = new ConcurrentHashMap[String, util.List[PartitionGroup]]() + + // store all reallocated shuffle and its current max epoch. + // key appId-shuffleId partitionGroupId + private[deploy] val reallocatedPGMaxEpochMap = + new ConcurrentHashMap[String, ConcurrentHashMap[Int, PartitionGroup]]() + + private[deploy] val stageEndShuffleSet = new ConcurrentSet[String]() + private[deploy] val stageEndShuffleTimeMap = new ConcurrentHashMap[String, Long]() + + private[deploy] val dataLostShuffleSet = new ConcurrentSet[String]() + + private[deploy] val unregisterShuffleTime = new ConcurrentHashMap[String, Long]() + private val removeShuffleDelayMs = CssConf.removeShuffleDelayMs(cssConf) + private[deploy] val scheduledManager = new ScheduledManager("shuffle-forward-message-thread", 1) + scheduledManager.addScheduledTask("shuffleTimeOutTask", checkExpiredShuffle, 0, removeShuffleDelayMs) + + override def registerShuffleStage( + appId: String, + shuffleId: Int, + numMappers: Int, + numPartitions: Int, + maxPartitionsPerGroup: Int): util.List[PartitionGroup] = { + val shuffleKey = Utils.getShuffleKey(appId, shuffleId) + + if (registeredPartitionGroup.get(shuffleKey) == null) { + shuffleKey.intern().synchronized { + if (registeredPartitionGroup.get(shuffleKey) == null) { + val startMs = System.currentTimeMillis() + + // try to assign partition group with current workers. + val stableAssignableWorkers = shuffleWorkerManager.getStableAssignableWorkers() + val partitionGroupList = assignStrategy.assignPartitionGroup( + numPartitions, + maxPartitionsPerGroup, + stableAssignableWorkers + ) + + externalShuffleMeta.shuffleCreated(shuffleKey) + shuffleTaskManager.initShuffleTask(shuffleKey, numMappers, numPartitions) + + val replicaWorks = partitionGroupList.flatMap(partitionGroup => partitionGroup.replicaWorkers.asScala) + shuffleWorkerManager.addShuffleStageWorker(shuffleKey, replicaWorks) + + // register shuffle done + logInfo(s"RegisterPartitionGroup for $shuffleKey ${numMappers} X ${numPartitions} " + + s"maxPartitionsPerGroup: ${maxPartitionsPerGroup} " + + s"used ${System.currentTimeMillis() - startMs}ms") + registeredPartitionGroup.put(shuffleKey, partitionGroupList.asJava) + } + } + } + registeredPartitionGroup.get(shuffleKey) + } + + override def validateRegisterShuffle(appId: String, shuffleId: Int): Boolean = { + val shuffleKey = Utils.getShuffleKey(appId, shuffleId) + registeredPartitionGroup.containsKey(shuffleKey) + } + + override def getAppShuffle(appId: String): Set[String] = { + registeredPartitionGroup.keys().asScala.filter(_.startsWith(appId)).toSet + } + + override def reallocateShufflePartition( + appId: String, + shuffleId: Int, + mapId: Int, + attemptId: Int, + oldPartitionGroup: PartitionGroup): PartitionGroup = { + val shuffleKey = Utils.getShuffleKey(appId, shuffleId) + val partitionGroupId = oldPartitionGroup.partitionGroupId + + def biggerEpochExists(): Boolean = { + val opPg = Option(reallocatedPGMaxEpochMap.get(shuffleKey)).map(_.get(partitionGroupId)) + if (opPg.isEmpty || opPg.get == null) { + return false + } + opPg.get.epochId > oldPartitionGroup.epochId + } + + def createBiggerEpoch(): PartitionGroup = { + val assignableWorkers = shuffleWorkerManager.getStableAssignableWorkers() + val partitionGroup = assignStrategy.reallocatePartitionGroup(assignableWorkers, oldPartitionGroup) + + if (partitionGroup == null) { + logError(s"assignGroupsWithRetry failed to reallocate after ${assignableWorkers.length} attempts.") + return null + } + val maxEpochMap = reallocatedPGMaxEpochMap + .computeWhenAbsent(shuffleKey, _ => { new ConcurrentHashMap[Int, PartitionGroup] }) + maxEpochMap.put(partitionGroup.partitionGroupId, partitionGroup) + return partitionGroup + } + + if (!biggerEpochExists()) { + reallocatedPGMaxEpochMap.synchronized { + if (!biggerEpochExists()) { + val newPartitionGroup = createBiggerEpoch() + if (newPartitionGroup == null) { + return null + } + val reallocateReplicaWorkers = + reallocatedPGMaxEpochMap.get(shuffleKey).get(partitionGroupId).replicaWorkers.asScala.toList + shuffleWorkerManager.addShuffleStageWorker(shuffleKey, reallocateReplicaWorkers) + } + } + } + reallocatedPGMaxEpochMap.get(shuffleKey).get(partitionGroupId) + } + + override def commitShuffleStage(appId: String, shuffleId: Int, forceFinish: Boolean): Boolean = { + + val shuffleKey = Utils.getShuffleKey(appId, shuffleId) + + stageEndShuffleTimeMap.putIfAbsent(shuffleKey, System.currentTimeMillis()) + if (forceFinish) { + stageEndShuffleSet.add(shuffleKey) + return true + } + + // whether the worker is alive or not, we need to CommitFiles. + val shuffleWorker = shuffleWorkerManager.getShuffleStageWorker(shuffleKey) + + val commitPieces = new ConcurrentHashMap[String, ConcurrentSet[CommittedPartitionInfo]] + val parallelism = Math.min(shuffleWorker.size, CssConf.commitFilesParallelism(cssConf)) + ThreadUtils.parmap(shuffleWorker.to, + "SendCommitFilesRequest", parallelism) { worker => + try { + val committed = worker.workerRpcRef.askSync[CommitFilesResponse]( + CommitFiles(shuffleKey), + new RpcTimeout(new FiniteDuration(CssConf.stageEndTimeoutMs(cssConf), MILLISECONDS), + "css.stage.end.timeout")).committed + // getFilePath == null means flush failed, piece is invalid. + committed.asScala.filter(_ != null).filter(_.getFilePath != null) + .foreach(p => { + val epochKey = p.getEpochKey + if (p.getFilePath.endsWith("data")) { + val commits = commitPieces.computeWhenAbsent(epochKey, _ => {new ConcurrentSet[CommittedPartitionInfo]}) + commits.add(p) + } else { + throw new Exception("filePath should either endsWith data.") + } + }) + } catch { + case ex: Exception => + // Log and do nothing for now. + logError(s"AskSync CommitFiles to Worker ${worker} for ${shuffleKey} failed.", ex) + } + } + + // after CommitFiles finished, clean all shuffle workers. + shuffleWorkerManager.removeShuffleStageWorker(shuffleKey) + + // commit all task partition info & check data lost + val dataLost = shuffleTaskManager.commitAllTaskPartitionInfo(shuffleKey, commitPieces) + if (dataLost) { + dataLostShuffleSet.add(shuffleKey) + } + stageEndShuffleSet.add(shuffleKey) + dataLost + } + + override def finishShuffleStage(appId: String, shuffleId: Int, wait: Boolean = false): StageStatus = { + val shuffleKey = Utils.getShuffleKey(appId, shuffleId) + val stageEndTimeoutMs = CssConf.stageEndTimeoutMs(cssConf) + val waitIntervalMs = CssConf.stageEndRetryIntervalMs(cssConf) + var loop = true + var status: StageStatus = null + while (loop) { + if (!stageEndShuffleTimeMap.containsKey(shuffleKey)) { + // no stage end request handled, should just return no stage end. + status = NoStageEndStatus + loop = false + } else if (!stageEndShuffleSet.contains(shuffleKey)) { + val stageEndStartMs = stageEndShuffleTimeMap.get(shuffleKey) + if (System.currentTimeMillis() - stageEndStartMs > stageEndTimeoutMs) { + // stageEnd timeout + status = StageEndTimeOutStatus + loop = false + } else { + // stageEnd processing + if (wait) { + TimeUnit.MILLISECONDS.sleep(waitIntervalMs) + } else { + status = StageEndRunningStatus + loop = false + } + } + } else if (dataLostShuffleSet.contains(shuffleKey)) { + status = StageEndDataLostStatus + loop = false + } else { + status = StageEndFinishStatus + loop = false + } + } + status + } + + override def getShuffleStageResult(appId: String, shuffleId: Int): StageResult = { + val shuffleKey = Utils.getShuffleKey(appId, shuffleId) + val (shuffleFileGroup, mapperAttempts, failedPartitionBatches) = + shuffleTaskManager.getReducerTaskFileGroups(shuffleKey) + StageResult(shuffleFileGroup, mapperAttempts, failedPartitionBatches) + } + + override def unregisterShuffle(appId: String, shuffleId: Int): Unit = { + // record the unregister shuffle request time, and delay unregister later via timer. + val shuffleKey = Utils.getShuffleKey(appId, shuffleId) + unregisterShuffleTime.put(shuffleKey, System.currentTimeMillis()) + } + + private def checkExpiredShuffle(): Unit = { + val currentTime = System.currentTimeMillis() + logDebug(s"Check for expired shuffle with $currentTime") + val keys = unregisterShuffleTime.keys().asScala.toList + val expiredKeys = keys.map { shuffleKey => + if (unregisterShuffleTime.get(shuffleKey) < currentTime - removeShuffleDelayMs) { + logInfo(s"Actually UnregisterShuffle handle with $currentTime for shuffle $shuffleKey") + registeredPartitionGroup.remove(shuffleKey) + reallocatedPGMaxEpochMap.remove(shuffleKey) + stageEndShuffleSet.remove(shuffleKey) + stageEndShuffleTimeMap.remove(shuffleKey) + dataLostShuffleSet.remove(shuffleKey) + unregisterShuffleTime.remove(shuffleKey) + shuffleWorkerManager.removeShuffleStageWorker(shuffleKey) + shuffleTaskManager.removeAllShuffleTask(shuffleKey) + shuffleKey + } else null + }.filter(_ != null) + externalShuffleMeta.shuffleRemoved(expiredKeys.toSet) + } + + override def start(): Unit = { + scheduledManager.start() + } + + override def stop(): Unit = { + scheduledManager.stop() + } + +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleTaskManager.scala b/service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleTaskManager.scala new file mode 100644 index 0000000..5dbfac4 --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleTaskManager.scala @@ -0,0 +1,183 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import java.util +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ + +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.protocol.{CommittedPartitionInfo, FailedPartitionInfoBatch, PartitionInfo} +import com.bytedance.css.common.util.Collections._ +import io.netty.util.internal.ConcurrentSet + +class ShuffleTaskManager extends Logging { + + // key appId-shuffleId + private[deploy] val shuffleMapperAttempts = new ConcurrentHashMap[String, Array[Int]]() + private[deploy] val reducerFileGroupsMap = + new ConcurrentHashMap[String, Array[Array[CommittedPartitionInfo]]]() + + // key appId-shuffleId + // store all reducerId-EpochId for a shuffleKey + private[deploy] val shuffleEpochSetMap = + new ConcurrentHashMap[String, ConcurrentHashMap[Int, util.List[PartitionInfo]]]() + private[deploy] val batchBlacklistMap = + new ConcurrentHashMap[String, ConcurrentHashMap[Int, util.List[FailedPartitionInfoBatch]]]() + + def initShuffleTask( + shuffleKey: String, + numMappers: Int, + numPartitions: Int): Unit = { + shuffleMapperAttempts.putIfAbsent(shuffleKey, Array.fill(numMappers)(-1)) + shuffleEpochSetMap.putIfAbsent(shuffleKey, new ConcurrentHashMap[Int, util.List[PartitionInfo]]()) + reducerFileGroupsMap.putIfAbsent(shuffleKey, new Array[Array[CommittedPartitionInfo]](numPartitions)) + } + + def shuffleTaskEnded( + shuffleKey: String, + mapId: Int, + attemptId: Int, + numMappers: Int, + epochList: util.List[PartitionInfo], + batchBlacklist: util.List[FailedPartitionInfoBatch]): Unit = { + shuffleMapperAttempts.synchronized { + var attempts = shuffleMapperAttempts.get(shuffleKey) + // for empty shuffle, e2e process could be non-register-shuffle at all + // and all mapper task start sending out MapperEnd after empty iterator + if (attempts == null) { + logWarning(s"Null shuffleMapperAttempts for shuffle $shuffleKey, create shuffleMapperAttempts.") + attempts = Array.fill(numMappers)(-1) + shuffleMapperAttempts.put(shuffleKey, attempts) + } + + // epochList not be null. + val epochSet = shuffleEpochSetMap.computeWhenAbsent(shuffleKey, _ => { + new ConcurrentHashMap[Int, util.List[PartitionInfo]]() + }) + epochSet.put(mapId, epochList) + + if (batchBlacklist != null) { + val blacklist = batchBlacklistMap.computeWhenAbsent(shuffleKey, _ => { + new ConcurrentHashMap[Int, util.List[FailedPartitionInfoBatch]] + }) + blacklist.put(mapId, batchBlacklist) + } + + // only remain the first success mapper & skip another attempt called. + if (attempts(mapId) < 0) { + attempts(mapId) = attemptId + } + } + } + + def validateShuffleTaskEnded( + shuffleKey: String, + mapId: Int): Boolean = { + // MapperAttempt != -1 means already has mapper calling MapperEnd + shuffleMapperAttempts.containsKey(shuffleKey) && shuffleMapperAttempts.get(shuffleKey)(mapId) != -1 + } + + def validateEffectiveTaskAttempt( + shuffleKey: String, + mapId: Int, + attemptId: Int): Boolean = { + shuffleMapperAttempts.containsKey(shuffleKey) && shuffleMapperAttempts.get(shuffleKey)(mapId) == attemptId + } + + def validateAllTaskEnded(shuffleKey: String): Boolean = { + shuffleMapperAttempts.containsKey(shuffleKey) && !shuffleMapperAttempts.get(shuffleKey).exists(_ < 0) + } + + def commitAllTaskPartitionInfo( + shuffleKey: String, + commitPieces: ConcurrentHashMap[String, ConcurrentSet[CommittedPartitionInfo]]): Boolean = { + + // check for data lost + val allEpochSets = new util.HashSet[PartitionInfo]() + allEpochSets.addAll( + shuffleEpochSetMap.getOrDefault(shuffleKey, new ConcurrentHashMap[Int, util.List[PartitionInfo]]()) + .values().asScala.flatMap(x => x.asScala.toSet[PartitionInfo]).toSet.asJava) + + var dataLost = false + val validCommitted = allEpochSets.asScala + .flatMap(partitionInfo => { + val epochKey = partitionInfo.getEpochKey + val commits = commitPieces.getOrDefault(epochKey, null) + if (commits == null) { + dataLost = true + logError(s"dataLost for $shuffleKey $epochKey, all replica null") + } + val availCommits = commits.asScala.filter(_ != null).filter(_.getFileLength >= 0).toSeq + if (availCommits.isEmpty) { + dataLost = true + logError(s"dataLost for $shuffleKey $epochKey, all replica file len < 0") + } + availCommits + }) + .filter(_ != null) + .filter(_.getFileLength > 0) + + if (!dataLost) { + val fileGroups = reducerFileGroupsMap.get(shuffleKey) + val sets = Array.fill(fileGroups.length)(new util.HashSet[CommittedPartitionInfo]()) + validCommitted.foreach { partition => + sets(partition.getReducerId).add(partition) + } + (0 until fileGroups.length).foreach(i => { + fileGroups(i) = sets(i).asScala.toSeq.toArray + }) + + var shuffleSize: Long = 0 + (0 until fileGroups.length).foreach(i => { + if (fileGroups(i) != null && fileGroups(i).length > 0) { + shuffleSize = shuffleSize + fileGroups(i).last.getFileLength + } + }) + logInfo(s"total shuffle size for shuffleKey ${shuffleKey} size ${shuffleSize}") + } + + dataLost + } + + def getReducerTaskFileGroups(shuffleKey: String): + (Array[Array[CommittedPartitionInfo]], Array[Int], util.HashSet[FailedPartitionInfoBatch]) = { + val shuffleFileGroup = reducerFileGroupsMap.get(shuffleKey) + val mapperAttempts = shuffleMapperAttempts.get(shuffleKey) + val failedPartitionBatches = if (batchBlacklistMap.get(shuffleKey) != null) { + val allBatches = new util.HashSet[FailedPartitionInfoBatch]() + allBatches.addAll( + batchBlacklistMap.getOrDefault(shuffleKey, new ConcurrentHashMap[Int, util.List[FailedPartitionInfoBatch]]()) + .values().asScala.flatMap(x => x.asScala.toSet[FailedPartitionInfoBatch]).toSet.asJava) + allBatches + } else { + null + } + (shuffleFileGroup, mapperAttempts, failedPartitionBatches) + } + + def removeAllShuffleTask(shuffleKey: String): Unit = { + shuffleMapperAttempts.remove(shuffleKey) + shuffleEpochSetMap.remove(shuffleKey) + reducerFileGroupsMap.remove(shuffleKey) + batchBlacklistMap.remove(shuffleKey) + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleWorkerManager.scala b/service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleWorkerManager.scala new file mode 100644 index 0000000..660c92b --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/master/ShuffleWorkerManager.scala @@ -0,0 +1,138 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import java.util +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.protocol.WorkerAddress +import com.bytedance.css.common.rpc.RpcEnv +import com.bytedance.css.common.util.Collections._ +import com.bytedance.css.service.deploy.common.ScheduledManager +import com.bytedance.css.service.deploy.metadata.WorkerProvider +import com.bytedance.css.service.deploy.worker.WorkerInfo +import io.netty.util.internal.ConcurrentSet + + +class ShuffleWorkerManager(cssConf: CssConf, rpcEnv: RpcEnv) extends Logging { + + // key appId-shuffleId + // store all relative shuffle worker (host:rpcPort) for a shuffleKey + private[deploy] val shuffleWorkerSetMap = new ConcurrentHashMap[String, ConcurrentSet[String]]() + + private val workerLostMap = new ConcurrentHashMap[String, WorkerInfo]() + + private val workerTimeoutMs = CssConf.workerTimeoutMs(cssConf) + val scheduledManager = new ScheduledManager("worker-forward-message-thread", 1) + scheduledManager.addScheduledTask("workerTimeOutTask", checkWorkerTimeout, 0, workerTimeoutMs) + + val workerProvider = WorkerProvider.create(cssConf, rpcEnv) + + def handleRegisterWorker(name: String, workerInfo: WorkerInfo): Boolean = { + logInfo(s"Registering worker(${workerInfo.name}) " + + s"${workerInfo.host}:${workerInfo.rpcPort}:${workerInfo.pushPort}:${workerInfo.fetchPort} .") + var registerStatus = false + if (workerProvider.contains(name) || workerLostMap.containsKey(name)) { + workerLostMap.synchronized { + if (!workerLostMap.containsKey(name)) { + workerLostMap.put(name, workerInfo) + } + } + workerProvider.remove(name) + logWarning(s"Worker ${name} re-register, might be WorkerLost. ignored register request.") + } else { + workerProvider.add(workerInfo) + logInfo(s"Registered worker $workerInfo") + registerStatus = true + } + registerStatus + } + + def handleWorkerHeartBeat( + name: String, + rttAvgStat: Long, + shuffleKeys: util.HashSet[String]): Boolean = { + logDebug(s"Received heartbeat from worker ${name}") + if (!workerProvider.contains(name)) { + logInfo(s"Received heartbeat from unknown worker ${name}") + return false + } + + val worker = workerProvider.get(name) + worker.synchronized { + worker.lastHeartbeat = System.currentTimeMillis() + } + return true + } + + private def checkWorkerTimeout() { + workerProvider.timeoutWorkers().foreach(lostWorker => { + logWarning(s"Worker ${lostWorker.name} timeout! Trigger WorkerLost event.") + if (!workerLostMap.containsKey(lostWorker.name)) { + workerLostMap.put(lostWorker.name, lostWorker) + } else { + workerLostMap.remove(lostWorker.name) + } + workerProvider.remove(lostWorker.name) + }) + } + + def addShuffleStageWorker( + shuffleKey: String, + assignWorkers: List[WorkerAddress]): Unit = { + val relativeWorkers = getStableAssignableWorkers().filter(w => { + assignWorkers.exists(p => (w.host.equals(p.host) && w.pushPort == p.port)) + }).map(_.name) + + shuffleWorkerSetMap.computeWhenAbsent(shuffleKey, _ => { new ConcurrentSet[String]() }) + .addAll(relativeWorkers.asJava) + } + + def removeShuffleStageWorker(shuffleKey: String): Unit = { + shuffleWorkerSetMap.remove(shuffleKey) + } + + def getShuffleStageWorker(shuffleKey: String): Seq[WorkerInfo] = { + val shuffleWorker = shuffleWorkerSetMap.get(shuffleKey).asScala + .map(workerProvider.get).filter(_ != null).toList + return shuffleWorker + } + + def getStableAssignableWorkers(): Seq[WorkerInfo] = { + workerProvider.stableAssignableWorkers + } + + def getActiveWorkers(): Seq[WorkerInfo] = { + workerProvider.activeWorkers.values().asScala.toSeq + } + + def start(): Unit = { + scheduledManager.start() + } + + def stop(): Unit = { + workerProvider.close() + scheduledManager.stop() + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/metadata/ExternalShuffleMeta.scala b/service/src/main/scala/com/bytedance/css/service/deploy/metadata/ExternalShuffleMeta.scala new file mode 100644 index 0000000..a406fcc --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/metadata/ExternalShuffleMeta.scala @@ -0,0 +1,64 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.metadata + +import com.bytedance.css.common.CssConf +import com.bytedance.css.service.deploy.metadata.WorkerRegistryFactory.{TYPE_STANDALONE, TYPE_ZOOKEEPER} +import com.bytedance.css.service.deploy.metadata.zookeeper.ZookeeperExternalShuffleMeta + +/** + * An external metadata hook which hold by master to inform outsider dependency to deal with following events. + * + * AppCreated + * AppRemoved + * ShuffleCreated + * ShuffleRemoved + */ +trait ExternalShuffleMeta { + + def appCreated(appId: String): Unit + + def appRemoved(appId: String): Unit + + def shuffleCreated(shuffleKey: String): Unit + + def shuffleRemoved(shuffleKeys: Set[String]): Unit + + def cleanupIfNeeded(): Unit +} + +class StandaloneExternalShuffleMeta extends ExternalShuffleMeta { + // Standalone mode does not need to inform metadata to External + override def appCreated(appId: String): Unit = {} + override def appRemoved(appId: String): Unit = {} + override def shuffleCreated(shuffleKey: String): Unit = {} + override def shuffleRemoved(shuffleKeys: Set[String]): Unit = {} + override def cleanupIfNeeded(): Unit = {} +} + +object ExternalShuffleMeta { + + def create(cssConf: CssConf): ExternalShuffleMeta = { + CssConf.workerRegistryType(cssConf) match { + case TYPE_STANDALONE => new StandaloneExternalShuffleMeta() + case TYPE_ZOOKEEPER => new ZookeeperExternalShuffleMeta(cssConf) + } + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/metadata/WorkerProvider.scala b/service/src/main/scala/com/bytedance/css/service/deploy/metadata/WorkerProvider.scala new file mode 100644 index 0000000..7ff41b8 --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/metadata/WorkerProvider.scala @@ -0,0 +1,101 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.metadata + +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.protocol.WorkerStatus +import com.bytedance.css.common.rpc.RpcEnv +import com.bytedance.css.service.deploy.metadata.WorkerRegistryFactory.{TYPE_STANDALONE, TYPE_ZOOKEEPER} +import com.bytedance.css.service.deploy.metadata.standalone.StandaloneWorkerProvider +import com.bytedance.css.service.deploy.metadata.zookeeper.ZookeeperWorkerProvider +import com.bytedance.css.service.deploy.worker.WorkerInfo + +/** + * hold by master and provides available workers. + */ +trait WorkerProvider extends Logging { + + // backup workers that could be as a supplement if activeWorker is not enough for partition assignment + // in Standalone mode, this is always empty + // in ZK mode, it's basically from the entire cluster + var candidateWorkers: ArrayBuffer[WorkerStatus] = null + + // normal worker management + val activeWorkers = new ConcurrentHashMap[String, WorkerInfo]() + + def stableAssignableWorkers(): Seq[WorkerInfo] = { + stableAssignableWorkers { + workerInfo: WorkerInfo => workerInfo.isActive() + } + } + + // used for worker assignment + // maybe less than activeWorkers basic on the worker load and connection status. + def stableAssignableWorkers(filter: WorkerInfo => Boolean): Seq[WorkerInfo] = { + // filter out expired / lost connection / high load workers + val assignable = activeWorkers.values().asScala + .filter(filter) + .toSeq + logInfo(s"assignableWorkers before:${activeWorkers.size()} after:${assignable.size}") + assignable.sortBy(_.hostPort) + } + + def contains(name: String): Boolean = { + activeWorkers.containsKey(name) + } + + def get(name: String): WorkerInfo = { + activeWorkers.get(name) + } + + def add(workerInfo: WorkerInfo): Unit = { + activeWorkers.put(workerInfo.name, workerInfo) + } + + def remove(name: String): WorkerInfo = { + activeWorkers.remove(name) + } + + def allocate(target: Int): Unit = {} + + // return worker which need exclude & not in service + def exclude(): Seq[WorkerInfo] = Seq.empty + + // zk mode does not check timeout worker through heartbeat + def timeoutWorkers(): Seq[WorkerInfo] = Seq.empty + + def close(): Unit = {} +} + +object WorkerProvider { + + def create(cssConf: CssConf, rpcEnv: RpcEnv): WorkerProvider = { + CssConf.workerRegistryType(cssConf) match { + case TYPE_STANDALONE => new StandaloneWorkerProvider(cssConf) + case TYPE_ZOOKEEPER => new ZookeeperWorkerProvider(cssConf, rpcEnv) + } + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/metadata/WorkerRegistry.scala b/service/src/main/scala/com/bytedance/css/service/deploy/metadata/WorkerRegistry.scala new file mode 100644 index 0000000..3f0f6fd --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/metadata/WorkerRegistry.scala @@ -0,0 +1,55 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.metadata + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.rpc.RpcEnv +import com.bytedance.css.service.deploy.metadata.standalone.StandaloneWorkerRegistry +import com.bytedance.css.service.deploy.metadata.zookeeper.ZooKeeperWorkerRegistry +import com.bytedance.css.service.deploy.worker.WorkerInfo +import com.bytedance.css.service.deploy.worker.handler.RecycleShuffleHandler + +/** + * hold by worker, registers with the master and provides its own node information. + */ +trait WorkerRegistry { + + def register(workerInfo: WorkerInfo): Unit + + def update(workerInfo: WorkerInfo, rttAvgStat: Long): Unit + + def close(): Unit +} + +object WorkerRegistryFactory { + + val TYPE_ZOOKEEPER = "zookeeper" + val TYPE_STANDALONE = "standalone" + + def create( + rpcEnv: RpcEnv, + cssConf: CssConf, + handler: RecycleShuffleHandler): WorkerRegistry = { + CssConf.workerRegistryType(cssConf) match { + case TYPE_STANDALONE => new StandaloneWorkerRegistry(rpcEnv, cssConf, handler) + case TYPE_ZOOKEEPER => new ZooKeeperWorkerRegistry(rpcEnv, cssConf, handler) + } + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/metadata/standalone/StandaloneWorkerProvider.scala b/service/src/main/scala/com/bytedance/css/service/deploy/metadata/standalone/StandaloneWorkerProvider.scala new file mode 100644 index 0000000..355e69f --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/metadata/standalone/StandaloneWorkerProvider.scala @@ -0,0 +1,37 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.metadata.standalone + +import scala.collection.JavaConverters._ + +import com.bytedance.css.common.CssConf +import com.bytedance.css.service.deploy.metadata.WorkerProvider +import com.bytedance.css.service.deploy.worker.WorkerInfo + + +class StandaloneWorkerProvider(cssConf: CssConf) extends WorkerProvider { + + private val workerTimeoutMs = CssConf.workerTimeoutMs(cssConf) + + override def timeoutWorkers(): Seq[WorkerInfo] = { + val currentTime = System.currentTimeMillis() + activeWorkers.values().asScala.filter(_.lastHeartbeat < currentTime - workerTimeoutMs).toSeq + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/metadata/standalone/StandaloneWorkerRegistry.scala b/service/src/main/scala/com/bytedance/css/service/deploy/metadata/standalone/StandaloneWorkerRegistry.scala new file mode 100644 index 0000000..b028bc5 --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/metadata/standalone/StandaloneWorkerRegistry.scala @@ -0,0 +1,82 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.metadata.standalone + +import scala.collection.JavaConverters._ + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.protocol.{CssStatusCode, RpcNameConstants} +import com.bytedance.css.common.protocol.CssRpcMessage.{HeartbeatFromWorker, HeartbeatResponse, RegisterWorker, RegisterWorkerResponse} +import com.bytedance.css.common.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} +import com.bytedance.css.common.util.Utils +import com.bytedance.css.service.deploy.metadata.WorkerRegistry +import com.bytedance.css.service.deploy.worker.WorkerInfo +import com.bytedance.css.service.deploy.worker.handler.RecycleShuffleHandler + +class StandaloneWorkerRegistry( + val rpcEnv: RpcEnv, + val cssConf: CssConf, + val handler: RecycleShuffleHandler) extends WorkerRegistry with Logging { + + val heartbeatRef: RpcEndpointRef = + rpcEnv.setupEndpointRef(RpcAddress.fromCssURL(CssConf.masterAddress(cssConf)), RpcNameConstants.HEARTBEAT) + + override def register(workerInfo: WorkerInfo): Unit = { + val req = RegisterWorker( + workerInfo.name, workerInfo.host, workerInfo.rpcPort, + workerInfo.pushPort, workerInfo.fetchPort, workerInfo.workerRpcRef) + + var res = heartbeatRef.askSync[RegisterWorkerResponse](req) + var registerTimeoutMs = CssConf.workerRegisterTimeoutMs(cssConf) + val sleepInterval = 2000 + while (!res.statusCode.equals(CssStatusCode.Success) && registerTimeoutMs > 0) { + logWarning(s"Register worker failed with StatusCode: ${res.statusCode}") + Thread.sleep(sleepInterval) + registerTimeoutMs = registerTimeoutMs - sleepInterval + logInfo("Trying to re-register with master.") + res = heartbeatRef.askSync[RegisterWorkerResponse](req) + } + + if (!res.statusCode.equals(CssStatusCode.Success)) { + logError(s"Failed to register worker within ${CssConf.workerRegisterTimeoutMs(cssConf)}ms") + System.exit(-1) + } + } + + override def update(workerInfo: WorkerInfo, rttAvgStat: Long): Unit = { + Utils.tryLogNonFatalError { + val res = heartbeatRef.askSync[HeartbeatResponse]( + HeartbeatFromWorker(workerInfo.name, rttAvgStat, workerInfo.shuffleKeySet())) + if (!res.expiredShuffleKeys.isEmpty) { + logInfo(s"Told from Standalone Master to cleanup shuffle files " + + s"${res.expiredShuffleKeys.asScala.mkString(",")}") + handler.recycleShuffle(res.expiredShuffleKeys) + } + if (!res.expiredAppIds.isEmpty) { + logInfo(s"Told from Standalone Master to cleanup expired appId dirs " + + s"${res.expiredAppIds.asScala.mkString(",")}") + handler.recycleApplication(res.expiredAppIds) + } + } + } + + override def close(): Unit = {} +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/metadata/zookeeper/ZooKeeperWorkerRegistry.scala b/service/src/main/scala/com/bytedance/css/service/deploy/metadata/zookeeper/ZooKeeperWorkerRegistry.scala new file mode 100644 index 0000000..932f365 --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/metadata/zookeeper/ZooKeeperWorkerRegistry.scala @@ -0,0 +1,220 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.metadata.zookeeper + +import java.util +import java.util.concurrent.TimeUnit + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.protocol.WorkerStatus +import com.bytedance.css.common.rpc.RpcEnv +import com.bytedance.css.common.util.{JsonUtils, ThreadUtils, Utils} +import com.bytedance.css.service.deploy.metadata.WorkerRegistry +import com.bytedance.css.service.deploy.worker.{WorkerInfo, WorkerSource} +import com.bytedance.css.service.deploy.worker.handler.RecycleShuffleHandler +import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent +import org.apache.curator.framework.recipes.locks.InterProcessMutex +import org.apache.zookeeper.{CreateMode, KeeperException} + +class ZooKeeperWorkerRegistry( + val rpcEnv: RpcEnv, + val cssConf: CssConf, + val handler: RecycleShuffleHandler) extends WorkerRegistry with Logging{ + + val zkClient = ZookeeperClient.build(cssConf) + var workerName: Option[String] = None + + private val expiredMetaMs = CssConf.extMetaExpireIntervalMs(cssConf) + private val zkMaxThreads = CssConf.zkMaxParallelism(cssConf) + private val expiredLogEnabled = CssConf.zkMetaExpiredLogEnable(cssConf) + private lazy val lock = new InterProcessMutex(zkClient.curatorClient(), zkClient.lockPath) + private val metaCleanupScheduler = + ThreadUtils.newDaemonSingleThreadScheduledExecutor("zkExpiredMetaCleanup-scheduler") + + override def toString: String = zkClient.toString + + override def register(workerInfo: WorkerInfo): Unit = { + if (workerName.isEmpty) { + workerName = Some(workerInfo.name) + } + + // initial zk WorkerStatus with empty load + val workerPath = zkClient.getWorkPath(workerInfo.name) + val jsonStatus = JsonUtils.serialize(WorkerInfo.toWorkerStatus(workerInfo, 0)) + + try { + if (zkClient.checkExists(workerPath)) { + zkClient.delete(workerPath) + } + zkClient.create(workerPath, jsonStatus, mode = CreateMode.PERSISTENT) + // enable watch on app & shuffle removable events + watchAppShuffleExpired() + } catch { + case ex: Exception => + logError(s"Zookeeper worker registry failed, client: $toString .", ex) + System.exit(1) + } + metaCleanupScheduler.scheduleAtFixedRate(new Runnable { + override def run(): Unit = Utils.tryLogNonFatalError { + zkExpiredMetaCleanup() + } + }, 0, expiredMetaMs / 6, TimeUnit.MILLISECONDS) + } + + private def watchAppShuffleExpired(): Unit = { + logInfo(s"Start watching ${zkClient.applicationsPath} and ${zkClient.shufflesPath} .") + + // watch application expired + zkClient.watchChild(zkClient.applicationsPath, + event => if (event.getType == PathChildrenCacheEvent.Type.CHILD_REMOVED) { + logInfo(s"Application expired, watch path ${event.getData.getPath}, watch event ${event.getType}") + val expiredApplication = event.getData.getPath.split("/").last + handler.recycleApplication(new util.HashSet[String]() { add(expiredApplication) }) + }) + + // watch shuffle expired + zkClient.watchChild(zkClient.shufflesPath, + event => if (event.getType == PathChildrenCacheEvent.Type.CHILD_REMOVED) { + logInfo(s"ShuffleKey expired, watch path ${event.getData.getPath}, watch event ${event.getType}") + val expiredShuffle = event.getData.getPath.split("/").last + handler.recycleShuffle(new util.HashSet[String]() { add(expiredShuffle) }) + }) + } + + private def zkExpiredMetaCleanup(): Unit = { + val current = System.currentTimeMillis() + if (lock.acquire(15, TimeUnit.SECONDS)) { + // try remove other shuffle and app zk node from other application + try { + if (!zkClient.checkExists(zkClient.expiredTimePath) || + current - zkClient.getData(zkClient.expiredTimePath).toLong > 120000L) { + logInfo("zkExpiredMetaCleanup executing .") + } else { + return + } + + val shuffles = zkClient.list(zkClient.shufflesPath) + val apps = zkClient.list(zkClient.applicationsPath) + val workers = zkClient.list(zkClient.workersPath) + + ThreadUtils.parmap( + workers, "worker cleanup", zkMaxThreads) { name => + try { + val workerPath = zkClient.getWorkPath(name) + val json = zkClient.getData(workerPath) + val status = JsonUtils.deserialize(json, classOf[WorkerStatus]) + if (current - status.lastHeartbeat > expiredMetaMs) { + val meter = WorkerSource.workerSource.WORKER_LOST_EVENT + logInfo(s"Detect worker lost ${status.name} via zk expired") + meter.mark(1L) + // After the meter worker lost is issued, it cannot be kept forever, + // so it needs to decrease by 1L after 90s to avoid constant count. + metaCleanupScheduler.schedule(new Runnable { + override def run(): Unit = { + meter.mark(-1L) + logInfo(s"Recover worker lost event ${status.name} -1L") + } + }, 90, TimeUnit.SECONDS) + zkClient.delete(workerPath) + } + } catch { + case ex: Exception => + if (expiredLogEnabled) { + logError(s"remove expired worker failed with $name", ex) + } + } + } + + ThreadUtils.parmap( + shuffles, "other shuffles", zkMaxThreads) { shuffleKey => + try { + val shufflePath = zkClient.getShufflePath(shuffleKey) + val timestamp = zkClient.getData(shufflePath).toLong + if (current - timestamp > expiredMetaMs) { + zkClient.delete(shufflePath) + } + } catch { + case ex: Exception => + if (expiredLogEnabled) { + logError(s"remove other shuffle failed with $shuffleKey", ex) + } + } + } + + ThreadUtils.parmap( + apps, "other apps", zkMaxThreads) { appId => + try { + val appPath = zkClient.getApplicationPath(appId) + val timestamp = zkClient.getData(appPath).toLong + if (current - timestamp > expiredMetaMs) { + zkClient.delete(appPath) + } + } catch { + case ex: Exception => + if (expiredLogEnabled) { + logError(s"remove other app failed with $appId", ex) + } + } + } + + // update meta expired processed time + if (zkClient.checkExists(zkClient.expiredTimePath)) { + zkClient.setData(zkClient.expiredTimePath, current.toString) + } else { + zkClient.create(zkClient.expiredTimePath, current.toString, CreateMode.PERSISTENT) + } + } catch { + case _: Exception => + logWarning(s"zkExpiredMetaCleanup Delete other app shuffle node failed.") + } finally { + lock.release() + } + } + } + + override def update(workerInfo: WorkerInfo, rttAvgStat: Long): Unit = synchronized { + try { + val status = WorkerInfo.toWorkerStatus(workerInfo, rttAvgStat) + val workerPath = zkClient.getWorkPath(workerInfo.name) + val jsonStatus = JsonUtils.serialize(status) + zkClient.setData(workerPath, jsonStatus) + } catch { + case nne: KeeperException.NoNodeException => + logError(s"Try update worker: ${workerInfo.name} but ZK Node is missing, abort worker process.", nne) + System.exit(1) + case ex: Exception => + logError(s"Try update worker: ${workerInfo.name} with rttAvgStat: $rttAvgStat failed.", ex) + } + } + + override def close(): Unit = { + try { + if (workerName.isDefined) { + val workerPath = zkClient.getWorkPath(workerName.get) + zkClient.delete(workerPath) + } + zkClient.close() + } catch { + case ex: Exception => + logError(s"Worker: ${workerName.get} close cleanup failed.", ex) + } + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/metadata/zookeeper/ZookeeperClient.scala b/service/src/main/scala/com/bytedance/css/service/deploy/metadata/zookeeper/ZookeeperClient.scala new file mode 100644 index 0000000..9c31214 --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/metadata/zookeeper/ZookeeperClient.scala @@ -0,0 +1,182 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.metadata.zookeeper + +import java.io.Closeable + +import scala.collection.JavaConverters._ + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.util.Utils +import org.apache.commons.lang3.StringUtils +import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} +import org.apache.curator.framework.recipes.cache.{PathChildrenCache, PathChildrenCacheEvent, PathChildrenCacheListener} +import org.apache.curator.framework.state.{ConnectionState, ConnectionStateListener} +import org.apache.curator.retry.ExponentialBackoffRetry +import org.apache.zookeeper.{CreateMode, WatchedEvent, Watcher} + +class ZookeeperClient(val cssConf: CssConf) extends Closeable with Logging { + import ZookeeperClient._ + + private val servers: String = CssConf.zkAddress(cssConf) + if (StringUtils.isBlank(servers)) { + throw new IllegalArgumentException(s"Invalid config: servers=$servers") + } + private lazy val clusterName: String = { + var activeClusterName = CssConf.clusterName(cssConf) + if (StringUtils.isNotEmpty(CssConf.haClusterName(cssConf))) { + activeClusterName = CssConf.haClusterName(cssConf) + try { + val haClusterPath = s"$clusterTagPath/${CssConf.haClusterName(cssConf)}" + val tag = getData(haClusterPath) + activeClusterName = s"${CssConf.haClusterName(cssConf)}${tag}" + } catch { + case t: Throwable => + val msg = s"error to get active tag for ha cluster ${activeClusterName}." + logError(msg, t) + throw new IllegalArgumentException(msg, t) + } + } + if (StringUtils.isBlank(activeClusterName)) { + throw new IllegalArgumentException(s"Invalid input: cluster=$activeClusterName") + } + logInfo(s"get current cluster name ${activeClusterName}") + activeClusterName + } + + private val retries: Int = CssConf.zkRetries(cssConf) + private val client: CuratorFramework = CuratorFrameworkFactory.builder.connectString(servers) + .sessionTimeoutMs(CssConf.zkSessionTimeoutMs(cssConf)) + .retryPolicy(new ExponentialBackoffRetry(MIN_RETRY_SLEEP_MS, retries, MAX_RETRY_SLEEP_MS)) + .build + client.start() + + override def toString: String = { + String.format("ZooKeeperServiceRegistry{servers=%s}", servers) + } + + override def close(): Unit = { + client.close() + } + + def curatorClient(): CuratorFramework = client + + val clusterTagPath: String = s"/$ZK_BASE_PATH/$ZK_CLUSTER_TAG_PATH" + + val rootPath: String = s"/$ZK_BASE_PATH/$clusterName" + + val workersPath: String = s"$rootPath/$ZK_WORKER_SUBPATH" + + val applicationsPath: String = s"$rootPath/$ZK_APPLICATION_SUBPATH" + + val shufflesPath: String = s"$rootPath/$ZK_SHUFFLE_SUBPATH" + + val lockPath: String = s"$rootPath/$ZK_LOCK_SUBPATH" + + val expiredTimePath: String = s"$rootPath/$ZK_EXPIRED_TIME_SUBPATH" + + def getWorkPath(workerName: String): String = s"$workersPath/$workerName" + + def getApplicationPath(applicationId: String): String = s"$applicationsPath/$applicationId" + + def getShufflePath(applicationId: String, shuffleId: Int): String = { + s"$shufflesPath/${Utils.getShuffleKey(applicationId, shuffleId)}" + } + + def getShufflePath(shuffleKey: String): String = s"$shufflesPath/$shuffleKey" + + def checkExists(path: String): Boolean = { + client.checkExists().forPath(path) != null + } + + def create(path: String, value: String, mode: CreateMode = CreateMode.EPHEMERAL): Unit = { + if (!checkExists(path)) { + client.create.creatingParentsIfNeeded + .withMode(mode).forPath(path, value.getBytes()) + } + } + + def addStateListener(targetState: ConnectionState, func: () => Unit): Unit = { + client.getConnectionStateListenable.addListener(new ConnectionStateListener { + override def stateChanged(curatorFramework: CuratorFramework, connectionState: ConnectionState): Unit = { + if (targetState == connectionState) { + logInfo(s"ZooKeeper state changed: $targetState") + func() + } + } + }) + } + + def setData(path: String, value: String): Unit = { + client.setData().forPath(path, value.getBytes()) + } + + def delete(path: String): Unit = { + client.delete().forPath(path) + } + + def getData(path: String): String = { + new String(client.getData.forPath(path)) + } + + def list(path: String): Seq[String] = { + client.getChildren.forPath(path).asScala + } + + def watch(path: String, watchFun: WatchedEvent => Unit): Unit = { + client.getData.usingWatcher(new Watcher { + override def process(event: WatchedEvent): Unit = { + watchFun(event) + } + }).forPath(path) + } + + def watchChild(path: String, watchFun: PathChildrenCacheEvent => Unit): Unit = { + val patchChildrenCache = new PathChildrenCache(client, path, true) + patchChildrenCache.start(PathChildrenCache.StartMode.BUILD_INITIAL_CACHE) + patchChildrenCache.getListenable.addListener(new PathChildrenCacheListener { + override def childEvent(client: CuratorFramework, event: PathChildrenCacheEvent): Unit = { + watchFun(event) + } + }) + } +} + +object ZookeeperClient { + val ZK_BASE_PATH: String = "css" + val ZK_CLUSTER_TAG_PATH: String = "tags" + val ZK_WORKER_SUBPATH: String = "workers" + val ZK_APPLICATION_SUBPATH: String = "applications" + val ZK_SHUFFLE_SUBPATH: String = "shuffles" + val ZK_LOCK_SUBPATH: String = "lock" + val ZK_EXPIRED_TIME_SUBPATH: String = "expiredTime" + val MIN_RETRY_SLEEP_MS: Int = 1000 + val MAX_RETRY_SLEEP_MS: Int = 10000 + + @volatile var zkClient: ZookeeperClient = null + + def build(cssConf: CssConf): ZookeeperClient = synchronized { + if (zkClient == null) { + zkClient = new ZookeeperClient(cssConf) + } + zkClient + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/metadata/zookeeper/ZookeeperExternalShuffleMeta.scala b/service/src/main/scala/com/bytedance/css/service/deploy/metadata/zookeeper/ZookeeperExternalShuffleMeta.scala new file mode 100644 index 0000000..71dcf57 --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/metadata/zookeeper/ZookeeperExternalShuffleMeta.scala @@ -0,0 +1,144 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.metadata.zookeeper + +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.util.{ThreadUtils, Utils} +import com.bytedance.css.service.deploy.metadata.ExternalShuffleMeta +import io.netty.util.internal.ConcurrentSet +import org.apache.zookeeper.CreateMode + +class ZookeeperExternalShuffleMeta(val cssConf: CssConf) extends ExternalShuffleMeta with Logging { + + val zkClient = ZookeeperClient.build(cssConf) + val appSet = new ConcurrentSet[String]() + val shuffleSet = new ConcurrentSet[String]() + val updateThread = ThreadUtils + .newDaemonSingleThreadScheduledExecutor("ZookeeperExternalShuffleMeta-update-thread") + + val intervalMs = CssConf.extMetaKeepaliveIntervalMs(cssConf) + + updateThread.scheduleAtFixedRate(new Runnable() { + override def run(): Unit = { + appSet.asScala.foreach(appId => { + try { + val appPath = zkClient.getApplicationPath(appId) + val timestamp = System.currentTimeMillis() + if (zkClient.checkExists(appPath)) { + zkClient.setData(appPath, timestamp.toString) + } + } catch { + case ex: Exception => + logError(s"Update zk app node $appId failed.", ex) + } + }) + + shuffleSet.asScala.foreach(shuffleKey => { + try { + val shufflePath = zkClient.getShufflePath(shuffleKey) + val timestamp = System.currentTimeMillis() + if (zkClient.checkExists(shufflePath)) { + zkClient.setData(shufflePath, timestamp.toString) + } + } catch { + case ex: Exception => + logError(s"Update zk shuffle node $shuffleKey failed.", ex) + } + }) + } + }, 0, intervalMs, TimeUnit.MILLISECONDS) + + override def appCreated(appId: String): Unit = { + appSet.add(appId) + val appPath = zkClient.getApplicationPath(appId) + val timestamp = System.currentTimeMillis() + zkClient.create(appPath, timestamp.toString, mode = CreateMode.PERSISTENT) + log.info(s"appCreated in $appPath with timestamp: $timestamp") + } + + override def appRemoved(appId: String): Unit = { + appSet.remove(appId) + val appPath = zkClient.getApplicationPath(appId) + if (zkClient.checkExists(appPath)) { + zkClient.delete(appPath) + } + log.info(s"appRemoved in $appPath") + } + + override def shuffleCreated(shuffleKey: String): Unit = { + shuffleSet.add(shuffleKey) + val shufflePath = zkClient.getShufflePath(shuffleKey) + val timestamp = System.currentTimeMillis() + zkClient.create(shufflePath, timestamp.toString, mode = CreateMode.PERSISTENT) + log.info(s"shuffleCreated in $shufflePath with timestamp: $timestamp") + } + + override def shuffleRemoved(shuffleKeys: Set[String]): Unit = { + shuffleKeys.foreach(shuffleKey => { + shuffleSet.remove(shuffleKey) + val shufflePath = zkClient.getShufflePath(shuffleKey) + if (zkClient.checkExists(shufflePath)) { + zkClient.delete(shufflePath) + } + log.info(s"shuffleRemoved in $shufflePath") + }) + } + + override def cleanupIfNeeded(): Unit = { + // Clean shuffleSet for current APP + shuffleSet.asScala.foreach(shuffleKey => { + try { + val shufflePath = zkClient.getShufflePath(shuffleKey) + if (zkClient.checkExists(shufflePath)) { + zkClient.delete(shufflePath) + } + logInfo(s"cleanupIfNeeded Delete zk shuffle node $shuffleKey") + } catch { + case ex: Exception => + logError(s"cleanupIfNeeded Delete zk shuffle node $shuffleKey failed.", ex) + } + }) + + // Clean appSet for current APP + appSet.asScala.foreach(appId => { + try { + val appPath = zkClient.getApplicationPath(appId) + if (zkClient.checkExists(appPath)) { + zkClient.delete(appPath) + } + logInfo(s"cleanupIfNeeded Delete zk app node $appId") + } catch { + case ex: Exception => + logError(s"cleanupIfNeeded Delete zk app node $appId failed.", ex) + } + }) + + // just try to clean up safely. + Utils.tryLogNonFatalError(() -> { + updateThread.shutdownNow() + zkClient.close() + }) + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/metadata/zookeeper/ZookeeperWorkerProvider.scala b/service/src/main/scala/com/bytedance/css/service/deploy/metadata/zookeeper/ZookeeperWorkerProvider.scala new file mode 100644 index 0000000..244b1ae --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/metadata/zookeeper/ZookeeperWorkerProvider.scala @@ -0,0 +1,112 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.metadata.zookeeper + +import java.util.Comparator + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.protocol.{RpcNameConstants, WorkerStatus} +import com.bytedance.css.common.rpc.{RpcAddress, RpcEnv} +import com.bytedance.css.common.util.{JsonUtils, ThreadUtils} +import com.bytedance.css.service.deploy.metadata.WorkerProvider +import com.bytedance.css.service.deploy.worker.WorkerInfo + +class ZookeeperWorkerProvider(cssConf: CssConf, rpcEnv: RpcEnv) extends WorkerProvider { + + private val zkClient = ZookeeperClient.build(cssConf) + private val zkMaxThreads = CssConf.zkMaxParallelism(zkClient.cssConf) + private val workerTimeoutMs = CssConf.workerTimeoutMs(zkClient.cssConf) + @volatile private var listFromZk: Boolean = false + private var zkWorkerNames: Seq[String] = null + + override def allocate(target: Int): Unit = synchronized { + val allocateStartMs = System.currentTimeMillis() + + // list from zk to get all candidate worker will only execute once + if (!listFromZk) { + val startMs = System.currentTimeMillis() + zkWorkerNames = zkClient.list(zkClient.workersPath) + listFromZk = true + logInfo(s"listFromZk used ${System.currentTimeMillis() - startMs}ms with ${zkWorkerNames.size} workers.") + } + + // get worker status for candidateWorkers + candidateWorkers = ThreadUtils.parmap( + zkWorkerNames, "candidateWorkers", zkMaxThreads) { name => + try { + val startMs = System.currentTimeMillis() + val json = zkClient.getData(zkClient.getWorkPath(name)) + val status = JsonUtils.deserialize(json, classOf[WorkerStatus]) + if (status.lastHeartbeat + workerTimeoutMs > startMs) { + status + } else { + null + } + } catch { + case ex: Exception => + logError(s"candidateWorkers WorkerStatus deserialize failed with $name", ex) + null + } + }.to[mutable.ArrayBuffer].filter(_ != null) + logInfo(s"candidateWorkers after heartbeat filtered, origin: " + + s"${zkWorkerNames.size}, current: ${candidateWorkers.size}") + + // sorted in ascending order of rttAvgStat, the top worker node is the node with relatively idle load + candidateWorkers.asJava.sort(new Comparator[WorkerStatus] { + override def compare(o1: WorkerStatus, o2: WorkerStatus): Int = { + o1.rttAvgStat.compareTo(o2.rttAvgStat) + } + }) + + candidateWorkers.foreach(w => logInfo(s"${w.host} ${w.rttAvgStat}ms")) + + val actualTarget = Math.min(target, candidateWorkers.size) + logInfo(s"target: $target, actualTarget: $actualTarget, candidateWorkers: ${candidateWorkers.size}") + + val newWorkers: Seq[WorkerStatus] = candidateWorkers.take(actualTarget) + candidateWorkers = candidateWorkers.drop(actualTarget) + + val startMs = System.currentTimeMillis() + ThreadUtils.parmap(newWorkers, "parallelism-init", zkMaxThreads) { status => + try { + val workerInfo = WorkerInfo.fromWorkerStatus( + status, + rpcEnv.setupEndpointRef( + RpcAddress(status.host, status.rpcPort), RpcNameConstants.WORKER_EP) + ) + workerInfo.rttAvgStat = status.rttAvgStat + activeWorkers.put(status.name, workerInfo) + } catch { + case ex: Exception => + logError(s"WorkerStatus to WorkerInfo failed with $status", ex) + } + } + logInfo(s"current: ${activeWorkers.size()}, target: $target, actualTarget: $actualTarget, " + + s"worker ctor used ${System.currentTimeMillis() - startMs}ms " + + s"entire allocate used ${System.currentTimeMillis() - allocateStartMs}ms") + } + + override def close(): Unit = { + zkClient.close() + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/worker/Storage.scala b/service/src/main/scala/com/bytedance/css/service/deploy/worker/Storage.scala new file mode 100644 index 0000000..b000bb9 --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/worker/Storage.scala @@ -0,0 +1,288 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker + +import java.io.{File, IOException} +import java.util.concurrent.atomic.AtomicInteger +import java.util.function.IntUnaryOperator + +import scala.sys.process._ + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.protocol.PartitionInfo +import com.bytedance.css.common.util.Utils +import com.bytedance.css.service.deploy.worker.Storage.workingDirName +import io.netty.util.internal.ConcurrentSet +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} + +/** + * Storage is mainly used to create a file writer for writing. + * and also used to clean up shuffle and app data + */ +trait Storage extends Logging { + + /** + * Create fileWriter for writing shuffle data + */ + @throws[IOException] + def createWriter( + appId: String, + shuffleId: Int, + partitionInfo: PartitionInfo, + replicaIndex: Int, + rotateThreshold: Long): FileWriter + + def bufferTotal(): Int = { + throw new UnsupportedOperationException + } + + def bufferLoad(): Int = { + throw new UnsupportedOperationException + } + + def cleanupExpiredShuffle(expiredShuffleKey: String): Unit + + def cleanupExpiredApp(expiredAppId: String): Unit +} + +object Storage { + val workingDirName = "css-worker/data" +} + +class DiskStorage(cssConf: CssConf) extends Storage { + // common conf + private val queueCapacity = CssConf.flushQueueCapacity(cssConf) + private val flushBufferSize = CssConf.flushBufferSize(cssConf).toInt + private val epochRotateThreshold = CssConf.epochRotateThreshold(cssConf) + + // disk flush conf + private val fetchChunkSize = CssConf.fetchChunkSize(cssConf) + private val timeoutMs = CssConf.flushTimeoutMs(cssConf) + private val diskFlushNum = CssConf.diskFlusherNum(cssConf) + + private val diskWorkingDirs: Array[File] = { + val baseDirs = CssConf.diskFlusherBaseDirs(cssConf).map(new File(_, workingDirName)) + baseDirs.foreach { dir => + try { + dir.mkdirs() + val file = new File(dir, s"_SUCCESS_${System.currentTimeMillis()}") + file.createNewFile() + file.delete() + } catch { + case ex: IOException => + logWarning(s"DiskFlusher base dir initialization check failed.", ex) + throw ex; + } + } + baseDirs + } + + private val diskFlushers = { + val flushNum = if (diskFlushNum != -1) diskFlushNum else diskWorkingDirs.length + logInfo(s"created $flushNum diskFlusher.") + (0 until flushNum).map(index => { + new FileFlusherImpl(s"DiskFileFlusher-$index", FileFlusher.DISK_FLUSHER_TYPE, queueCapacity, flushBufferSize) + }) + } + + private val diskCounter = new AtomicInteger() + private val diskCounterOperator = new IntUnaryOperator() { + override def applyAsInt(operand: Int): Int = (operand + 1) % diskWorkingDirs.length + } + + private val diskFlusherCounter = new AtomicInteger() + private val diskFlusherCounterOperator = new IntUnaryOperator() { + override def applyAsInt(operand: Int): Int = (operand + 1) % diskFlushers.size + } + + override def createWriter( + appId: String, + shuffleId: Int, + partitionInfo: PartitionInfo, + replicaIndex: Int, + rotateThreshold: Long): FileWriter = { + if (rotateThreshold < epochRotateThreshold) { + logDebug(s"Rotate threshold set by user is lesser than server side," + + s" use server config instead: $epochRotateThreshold") + } + val realThreshold = Math.max(rotateThreshold, epochRotateThreshold) + + val index = diskCounter.getAndUpdate(diskCounterOperator) + val flusherIndex = diskFlusherCounter.getAndUpdate(diskFlusherCounterOperator) + val shuffleDir = new File(diskWorkingDirs(index), s"$appId/$shuffleId") + val mode = s"r$replicaIndex-data" + val fileName = s"${partitionInfo.getReducerId}-${partitionInfo.getEpochId}-${mode}" + val file = new File(shuffleDir, fileName) + shuffleDir.mkdirs() + val created = file.createNewFile() + if (!created) { + throw new IOException(s"File ${file.getAbsoluteFile} is already Exists.") + } + val writer = new DiskFileWriter( + file, + diskFlushers(flusherIndex), + fetchChunkSize, + timeoutMs, + flushBufferSize, + realThreshold + ) + writer + } + + override def cleanupExpiredShuffle(expiredShuffleKey: String): Unit = { + val splits = expiredShuffleKey.split("-") + val appId = splits.dropRight(1).mkString("-") + val shuffleId = splits.last + diskWorkingDirs.foreach { workingDir => + val deleteDirCommand = s"rm -rf $workingDir/$appId/$shuffleId/" + logDebug(s"cleanupExpiredShuffleKey deleteDirCommand $deleteDirCommand") + try { + deleteDirCommand.! + } catch { + case ex: Exception => + logError(s"cleanupExpiredShuffleKey deleteDirCommand $deleteDirCommand failed.", ex) + } + } + } + + override def cleanupExpiredApp(expiredAppId: String): Unit = { + diskWorkingDirs.foreach { workingDir => + val deleteDirCommand = s"rm -rf $workingDir/$expiredAppId/" + logDebug(s"cleanupExpiredAppDir deleteDirCommand $deleteDirCommand") + try { + deleteDirCommand.! + } catch { + case ex: Exception => + logError(s"cleanupExpiredAppDir deleteDirCommand $deleteDirCommand failed.", ex) + } + } + } +} + + +class HdfsStorage(cssConf: CssConf) extends Storage { + // common conf + private val queueCapacity = CssConf.flushQueueCapacity(cssConf) + private val flushBufferSize = CssConf.flushBufferSize(cssConf).toInt + private val epochRotateThreshold = CssConf.epochRotateThreshold(cssConf) + + // hdfs flush conf + private val timeoutMs = CssConf.flushTimeoutMs(cssConf) + private val hdfsFlushNum = CssConf.hdfsFlusherNum(cssConf) + private val hdfsFlusherBaseDir = CssConf.hdfsFlusherBaseDir(cssConf) + + // get hadoop configuration from css conf + private lazy val hadoopConf: Configuration = { + val tmpHadoopConf = new Configuration() + for ((key, value) <- cssConf.getAll if key.startsWith("css.hadoop.")) { + tmpHadoopConf.set(key.substring("css.hadoop.".length), value) + } + // set replica for writer files + tmpHadoopConf.set("dfs.checksum.type", "NULL") + tmpHadoopConf.set("dfs.replication", CssConf.hdfsFlusherReplica(cssConf)) + tmpHadoopConf + } + + private lazy val fs: FileSystem = new Path(hdfsFlusherBaseDir).getFileSystem(hadoopConf) + + private val hdfsFlushers = { + (0 until hdfsFlushNum).map(index => { + new FileFlusherImpl(s"HdfsFileFlusher-$index", FileFlusher.HDFS_FLUSHER_TYPE, queueCapacity, flushBufferSize) + }) + } + + private val hdfsCounter = new AtomicInteger() + private val hdfsCounterOperator = new IntUnaryOperator() { + override def applyAsInt(operand: Int): Int = (operand + 1) % hdfsFlushNum + } + + // hold shuffle & app info which used hdfs storage + private val hdfsShuffleSet = new ConcurrentSet[String]() + private val hdfsAppSet = new ConcurrentSet[String]() + + override def createWriter( + appId: String, + shuffleId: Int, + partitionInfo: PartitionInfo, + replicaIndex: Int, + rotateThreshold: Long): FileWriter = { + if (rotateThreshold < epochRotateThreshold) { + logDebug(s"Rotate threshold set by user is lesser than server side," + + s" use server config instead: $epochRotateThreshold") + } + val realThreshold = Math.max(rotateThreshold, epochRotateThreshold) + + val index = hdfsCounter.getAndUpdate(hdfsCounterOperator) + val mode = s"r$replicaIndex-data" + val fileName = s"${partitionInfo.getReducerId}-${partitionInfo.getEpochId}-${mode}" + val path = new Path(s"$hdfsFlusherBaseDir/$appId/$shuffleId/$fileName") + val outputStream = fs.create(path, true) + val writer = new HdfsFileWriter( + fs, + path, + outputStream, + hdfsFlushers(index), + timeoutMs, + flushBufferSize, + realThreshold + ) + + val shuffleKey = Utils.getShuffleKey(appId, shuffleId) + hdfsShuffleSet.add(shuffleKey) + hdfsAppSet.add(appId) + writer + } + + override def cleanupExpiredShuffle(expiredShuffleKey: String): Unit = { + val splits = expiredShuffleKey.split("-") + val appId = splits.dropRight(1).mkString("-") + val shuffleId = splits.last + + // delete hdfs shuffle path + if (hdfsShuffleSet.contains(expiredShuffleKey)) { + val path = new Path(s"$hdfsFlusherBaseDir/$appId/$shuffleId") + try { + logDebug(s"cleanupExpiredHdfsShuffleKey path $path") + fs.delete(path, true) + } catch { + case ex: Exception => + logError(s"cleanupExpiredHdfsShuffleKey path $path failed.", ex) + } + hdfsShuffleSet.remove(expiredShuffleKey) + } + } + + override def cleanupExpiredApp(expiredAppId: String): Unit = { + // delete hdfs app path + if (hdfsAppSet.contains(expiredAppId)) { + val path = new Path(s"$hdfsFlusherBaseDir/$expiredAppId") + try { + logDebug(s"cleanupExpiredHdfsAppDir path $path") + fs.delete(path, true) + } catch { + case ex: Exception => + logError(s"cleanupExpiredHdfsAppDir path $path failed.", ex) + } + hdfsAppSet.remove(expiredAppId) + } + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/worker/StorageManager.scala b/service/src/main/scala/com/bytedance/css/service/deploy/worker/StorageManager.scala new file mode 100644 index 0000000..9a2be65 --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/worker/StorageManager.scala @@ -0,0 +1,147 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker + +import java.io.IOException +import java.util +import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue} + +import scala.collection.JavaConverters._ + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.protocol.PartitionInfo +import com.bytedance.css.common.util.Utils + +/** + * StorageManager to manage flusher and writer resource + * Also it should take care of shuffle and application expired data cleanup + */ +final class StorageManager(cssConf: CssConf) extends Logging { + + private val diskStorage = new DiskStorage(cssConf) + private val hdfsStorage = new HdfsStorage(cssConf) + + // key: shuffleKey + // value: Map(fileName -> writer) + private val fileWriters = new ConcurrentHashMap[String, ConcurrentHashMap[String, FileWriter]]() + + @throws[IOException] + def createFileWriter( + appId: String, + shuffleId: Int, + partitionInfo: PartitionInfo, + replicaIndex: Int, + rotateThreshold: Long, + isHdfsWriter: Boolean = false): FileWriter = { + var writer: FileWriter = null + if (isHdfsWriter) { + writer = hdfsStorage.createWriter(appId, shuffleId, partitionInfo, replicaIndex, rotateThreshold) + } else { + writer = diskStorage.createWriter(appId, shuffleId, partitionInfo, replicaIndex, rotateThreshold) + } + val shuffleKey = Utils.getShuffleKey(appId, shuffleId) + fileWriters.putIfAbsent(shuffleKey, new ConcurrentHashMap[String, FileWriter]()) + fileWriters.get(shuffleKey).put(writer.getFilePath, writer) + writer + } + + def getFileWriter(shuffleKey: String, filePath: String): FileWriter = { + val shuffleFileMap = fileWriters.get(shuffleKey) + if (shuffleFileMap != null) { + shuffleFileMap.get(filePath) + } else { + null + } + } + + // cleaner task will try to clean up expired shuffle data and writer + private val cleanupQueue = new LinkedBlockingQueue[ExpiredTask] + private val cleanupThread = new Thread("Cleanup thread for expired shuffle and app") { + override def run(): Unit = { + while (true) { + val expiredTask = cleanupQueue.take() + try { + expiredTask match { + case ShuffleKeyExpiredTask(expiredShuffleKeys) => + cleanupExpiredShuffleKey(expiredShuffleKeys) + case AppExpiredTask(expiredAppIds) => + cleanupExpiredAppDir(expiredAppIds) + case _ => + logWarning(s"Ignored cleanup expired ${expiredTask.taskType}") + } + } catch { + case ex: Exception => + logError(s"Failed to cleanup expired ${expiredTask.taskType}", ex) + } + } + } + } + + def start(): Unit = { + cleanupThread.setDaemon(true) + cleanupThread.start() + } + + def close(): Unit = { + fileWriters.asScala.foreach { entry => + if (entry != null && entry._2 != null) { + entry._2.values().asScala.foreach { fileWriter => + Utils.tryLogNonFatalError(fileWriter.close()) + Utils.tryLogNonFatalError(fileWriter.destroy()) + } + } + } + } + + def addCleanupExpiredTask(expiredTask: ExpiredTask): Unit = { + cleanupQueue.put(expiredTask) + } + + private def cleanupExpiredShuffleKey(expiredShuffleKeys: util.HashSet[String]): Unit = { + expiredShuffleKeys.asScala.foreach { shuffleKey => + val writers = fileWriters.remove(shuffleKey) + if (writers != null && !writers.isEmpty) { + writers.values().asScala.foreach(_.destroy()) + } + diskStorage.cleanupExpiredShuffle(shuffleKey) + hdfsStorage.cleanupExpiredShuffle(shuffleKey) + } + } + + private def cleanupExpiredAppDir(expiredAppIds: util.HashSet[String]): Unit = { + expiredAppIds.asScala.foreach(appId => { + diskStorage.cleanupExpiredApp(appId) + hdfsStorage.cleanupExpiredApp(appId) + }) + } +} + +case class ShuffleKeyExpiredTask(expiredShuffleKeys: util.HashSet[String]) extends ExpiredTask { + override var taskType: String = "ShuffleKeyTask" +} + +case class AppExpiredTask(expiredAppIds: util.HashSet[String]) extends ExpiredTask { + override var taskType: String = "AppTask" +} + +sealed trait ExpiredTask { + var taskType: String +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/worker/Worker.scala b/service/src/main/scala/com/bytedance/css/service/deploy/worker/Worker.scala new file mode 100644 index 0000000..fd5dec0 --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/worker/Worker.scala @@ -0,0 +1,561 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker + +import java.io.IOException +import java.nio.ByteBuffer +import java.util +import java.util.Random +import java.util.concurrent._ + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.exception.{AlreadyClosedException, EpochShouldRotateException} +import com.bytedance.css.common.exception.PartitionInfoNotFoundException +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.metrics.MetricsSystem +import com.bytedance.css.common.protocol._ +import com.bytedance.css.common.protocol.CssRpcMessage._ +import com.bytedance.css.common.rpc._ +import com.bytedance.css.common.util.{ThreadUtils, Utils} +import com.bytedance.css.network.TransportContext +import com.bytedance.css.network.buffer.NettyManagedBuffer +import com.bytedance.css.network.client.{RpcResponseCallback, TransportClientBootstrap} +import com.bytedance.css.network.protocol.BatchPushDataRequest +import com.bytedance.css.network.server.{CssFileInfo, TransportServerBootstrap} +import com.bytedance.css.service.deploy.common.ScheduledManager +import com.bytedance.css.service.deploy.metadata.WorkerRegistryFactory +import com.bytedance.css.service.deploy.metadata.WorkerRegistryFactory.{TYPE_STANDALONE, TYPE_ZOOKEEPER} +import com.bytedance.css.service.deploy.worker.handler._ +import com.bytedance.css.service.deploy.worker.handler.PushDataRpcHandler.WrapPushMetricsCallBack +import io.netty.buffer.ByteBuf +import io.netty.util.internal.ConcurrentSet +import org.apache.hadoop.util.ShutdownHookManager + +class Worker( + override val rpcEnv: RpcEnv, + val specificPushPort: Int, + val specificFetchPort: Int, + val conf: CssConf) + extends RpcEndpoint with PushDataHandler with FetchDataHandler with RecycleShuffleHandler with Logging { + + private val host = rpcEnv.address.host + val rpcPort = rpcEnv.address.port + + private val metricsSystem = MetricsSystem.createMetricsSystem(MetricsSystem.WORKER, conf) + private val workerSource = WorkerSource.create(CssConf.clusterName(conf), rpcEnv.address.host) + + private val (pushServer, pushRpcHandler, pushTransportContext) = { + val numThreads = CssConf.pushThreads(conf) + val transportConf = Utils.fromCssConf(conf, TransportModuleConstants.PUSH_MODULE, numThreads) + val pushRpcHandler = new PushDataRpcHandler(transportConf, this) + val transportContext: TransportContext = + new TransportContext(transportConf, pushRpcHandler, false) + val serverBootstraps: Seq[TransportServerBootstrap] = Nil + (transportContext.createServer(CssConf.pushServerPort(conf, specificPushPort), serverBootstraps.asJava), + pushRpcHandler, transportContext) + } + + private val (fetchServer, fetchRpcHandler, fetchTransportContext) = { + val numThreads = CssConf.fetchThreads(conf) + val transportConf = Utils.fromCssConf(conf, TransportModuleConstants.FETCH_MODULE, numThreads) + val fetchRpcHandler = new FetchDataRpcHandler(transportConf, this) + val transportContext: TransportContext = + new TransportContext(transportConf, fetchRpcHandler, false) + val serverBootstraps: Seq[TransportServerBootstrap] = Nil + (transportContext.createServer(CssConf.fetchServerPort(conf, specificFetchPort), serverBootstraps.asJava), + fetchRpcHandler, transportContext) + } + + val pushPort = pushServer.getPort + val fetchPort = fetchServer.getPort + + /** + * TODO: Generate a better worker name + * 1. Standalone mode: workerName = HostName + * 2. Yarn mode: workerName = RoleName-${index} + * 3. K8s mode: workerName = PodName-${index} + */ + private val workerName = s"$host:$rpcPort:$pushPort:$fetchPort" + private val workerInfo = new WorkerInfo(workerName, host, rpcPort, pushPort, fetchPort, self) + + private val storageManager = new StorageManager(conf) + + private val workerRegistry = WorkerRegistryFactory.create(rpcEnv, conf, this) + + private val scheduledManager = new ScheduledManager("worker-forward-message-scheduler", 2) + + // key: applicationId-shuffleId + // value: Set(CommitFileStringFormat[ReducerId-EpochId]) + private val committedInfo = new ConcurrentHashMap[String, ConcurrentSet[String]]() + + // Threads & Pools + private val commitThreadPool = ThreadUtils.newDaemonCachedThreadPool( + "for-commitFiles", CssConf.commitThreads(conf)) + + // Only for test + private val correctnessModeEnabled: Boolean = CssConf.correctnessModeEnable(conf) + private val rand = new Random + + override def onStart(): Unit = { + if (workerInfo.workerRpcRef == null) { + workerInfo.workerRpcRef = self + } + workerRegistry.register(workerInfo) + + val updateIntervalMs = CssConf.workerRegistryType(conf) match { + case TYPE_STANDALONE => CssConf.workerTimeoutMs(conf) / 4 + case TYPE_ZOOKEEPER => CssConf.workerUpdateIntervalMs(conf) + } + scheduledManager.addScheduledTask("updateWorkerStatusTask", () => { + Utils.tryLogNonFatalError { + val rttStat = 0L + logDebug(s"Update rttStat ${rttStat} to zk node.") + workerRegistry.update(workerInfo, rttStat) + } + }, 0, updateIntervalMs) + + val periodGcInterval = CssConf.cleanerPeriodicGCInterval(conf) + scheduledManager.addScheduledTask("periodGcTask", () => { + logDebug("periodGc start.") + System.gc() + }, periodGcInterval, periodGcInterval) + + scheduledManager.start() + storageManager.start() + + if (!CssConf.testMode(conf)) { + pushServer.registerCssMetrics(workerSource.cssMetricsPrefix) + fetchServer.registerCssMetrics(workerSource.cssMetricsPrefix) + } + workerSource.registerMetricSet(pushServer.getAllCssMetrics) + workerSource.registerMetricSet(fetchServer.getAllCssMetrics) + workerSource.registerMetricSet(pushRpcHandler.getAllMetrics) + workerSource.registerMetricSet(fetchRpcHandler.getAllMetrics) + workerSource.registerMetricSet(FileWriterMetrics.instance) + metricsSystem.registerSource(workerSource) + metricsSystem.start() + + // TODO add StorageManager buffer total count + logInfo(s"Starting Worker $workerName") + } + + override def onStop(): Unit = { + logInfo(s"onStop called.") + workerSource.WORKER_LOST_EVENT.mark() + metricsSystem.report() + logInfo(s"Detect worker lost ${workerInfo.name} via onStop") + workerRegistry.close() + storageManager.close() + scheduledManager.stop() + + pushServer.close() + pushTransportContext.close() + + fetchServer.close() + fetchTransportContext.close() + + metricsSystem.stop() + } + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case req: BreakPartition => + handleBreakPartitionForTest(context, req: BreakPartition) + case req: CommitFiles => + workerSource.withEventMetrics("CommitFiles") { + handleCommitFiles(context, req: CommitFiles) + } + case req: CloseFile => + workerSource.withEventMetrics("CloseFile") { + handleCloseFile(context, req: CloseFile) + } + } + + def handleBreakPartitionForTest( + context: RpcCallContext, + req: BreakPartition): Unit = { + val partition = workerInfo.getShufflePartition(req.shuffleKey, req.reducerId, req.epochId) + if (partition != null) { + partition.asInstanceOf[WritablePartitionInfo].getFileWriter.setException() + } + context.reply(BreakPartitionResponse(CssStatusCode.Success)) + } + + def handleCloseFile( + context: RpcCallContext, + req: CloseFile): Unit = { + val shuffleKey = req.shuffleKey + val partition = workerInfo.getShufflePartition(shuffleKey, req.partition.getReducerId, req.partition.getEpochId) + + // try best to close rotate partitions + // this forehead operation should decrease the time for stageEnd since we can pre-close some writer. + commitThreadPool.submit(new Runnable { + override def run(): Unit = { + try { + partition.asInstanceOf[WritablePartitionInfo].getFileWriter.close() + } catch { + case ex: Exception => + logError(s"CloseFile for shuffle $shuffleKey ${partition.getReducerId}-${partition.getEpochId} failed", ex) + } + } + }) + + context.reply(CloseFileResponse) + } + + def handleCommitFiles( + context: RpcCallContext, + req: CommitFiles): Unit = { + val shuffleKey = req.shuffleKey + committedInfo.putIfAbsent(shuffleKey, new ConcurrentSet[String]()) + val futures = new util.ArrayList[Future[_]]() + val committed = new util.ArrayList[CommittedPartitionInfo]() + val partitions = workerInfo.getAllShufflePartitions(shuffleKey) + + if (partitions != null) { + partitions.asScala.foreach { partition => + val future = commitThreadPool.submit(new Runnable { + override def run(): Unit = { + var fileWriter: FileWriter = null + committedInfo.get(shuffleKey).add(partition.getEpochKey) + try { + fileWriter = partition.asInstanceOf[WritablePartitionInfo].getFileWriter + val fileLength = fileWriter.close() + committed.synchronized { + committed.add( + // could be zero length file exists. + new CommittedPartitionInfo(partition.getReducerId, partition.getEpochId, + host, fetchPort, fileWriter.getShuffleMode, fileWriter.getFilePath, fileLength + ) + ) + } + } catch { + case ex: Exception => + logError(s"CommitFiles for shuffle $shuffleKey failed", ex) + committed.synchronized { + committed.add( + new CommittedPartitionInfo(partition.getReducerId, partition.getEpochId, + host, fetchPort, fileWriter.getShuffleMode, null, -1 + ) + ) + } + } + } + }) + futures.add(future) + } + } + + // wait for all fileWriter to commit + val startMs = System.currentTimeMillis() + futures.asScala.foreach(_.get(CssConf.stageEndTimeoutMs(conf), TimeUnit.MILLISECONDS)) + logInfo(s"CommitFiles for ${req.shuffleKey} for active num ${partitions.asScala.length} " + + s"await ${System.currentTimeMillis() - startMs} ms") + context.reply(CommitFilesResponse(committed)) + } + + private def writerLazyCreation( + shuffleKey: String, + reducerId: Int, + epochId: Int, + shuffleMode: String, + replicaIndex: Int, + epochRotateThreshold: String): PartitionInfo = synchronized { + val current = workerInfo.getShufflePartition(shuffleKey, reducerId, epochId) + if (current != null) { + return current + } + + if (shuffleMode.isEmpty || + epochRotateThreshold.isEmpty) { + logError(s"create writer and partition from pushDataRequest, but extra header is empty. " + + s"${shuffleMode} ${epochRotateThreshold}") + return null + } + + val splits = shuffleKey.split("-") + val appId = splits.dropRight(1).mkString("-") + val shuffleId = splits.last.toInt + val p = Utils.toPartitionInfo(reducerId, epochId) + + try { + val writer = storageManager.createFileWriter( + appId, shuffleId, p, replicaIndex, epochRotateThreshold.toLong, + ShuffleMode.valueOf(shuffleMode) == ShuffleMode.HDFS) + val partitionInfo = new WritablePartitionInfo(p, writer) + workerInfo.addShufflePartition(shuffleKey, partitionInfo) + partitionInfo + } catch { + case ex: Exception => + logError(s"createWriter for $shuffleKey failed.", ex) + null + } + } + + private def statusValidation( + shuffleKey: String, + reducerIds: Array[Int], + epochIds: Array[Int], + shuffleMode: String, + replicaIndex: Int, + epochRotateThreshold: String, + ignoreRotate: Boolean, + callback: RpcResponseCallback): FileWriterStatus = { + + require(reducerIds.length == epochIds.length, "length check") + + if (committedInfo.containsKey(shuffleKey)) { + // Check for speculative task mapper end, inform that speculate task that partition is already committed + callback.onSuccess(ByteBuffer.wrap(Array[Byte](CssStatusCode.StageEnded.getValue))) + return FileWriterStatus.UnWritable + } + + // correctness random failed + if (correctnessModeEnabled) { + // 1/2000 chance to fail + // epochId may fail when even numbered to prevent two consecutive failures + if (rand.nextInt(2000) == 0 && epochIds.head % 2 == 0) { + val msg = "correctnessModeEnabled is true, random failures" + logWarning(msg) + callback.onFailure(new Exception(msg)) + return FileWriterStatus.UnWritable + } + } + + reducerIds.indices.foreach { i => + var partitionInfo = workerInfo.getShufflePartition(shuffleKey, reducerIds(i), epochIds(i)) + if (partitionInfo == null) { + partitionInfo = writerLazyCreation(shuffleKey, reducerIds(i), epochIds(i), + shuffleMode, replicaIndex, epochRotateThreshold) + if (partitionInfo == null) { + val msg = s"${shuffleKey}-${reducerIds(i)}-${epochIds(i)} Lazy creation failed." + logError(msg) + callback.onFailure(new PartitionInfoNotFoundException(msg)) + return FileWriterStatus.UnWritable + } + } + + // check FileWriter status first and pass exception check + var writer: FileWriter = null + try { + writer = partitionInfo.asInstanceOf[WritablePartitionInfo].getFileWriter + writer.checkException() + if (writer.shouldRotate()) { + if (!ignoreRotate) { + // ignoreRotate = false, directly notify the client that Rotate does not perform writing + callback.onSuccess(ByteBuffer.wrap(Array[Byte](CssStatusCode.EpochShouldRotate.getValue))) + return FileWriterStatus.ShouldRotate + } else { + // ignoreRotate = true, notify the client Rotate to execute the write after writing + return FileWriterStatus.WritableButShouldRotate + } + } + } catch { + case ex: Exception => + callback.onFailure(ex) + logError(s"statusValidation failed with shuffleKey ${shuffleKey}", ex) + return FileWriterStatus.UnWritable + } + } + FileWriterStatus.Writable + } + + def writeBatchPushDataRequest(batchPushData: BatchPushDataRequest, callback: RpcResponseCallback): Boolean = { + val buf = batchPushData.body().asInstanceOf[NettyManagedBuffer].getBuf + + try { + batchPushData.reducerIds.indices.foreach { i => + val partitionInfo = workerInfo.getShufflePartition( + batchPushData.shuffleKey, + batchPushData.reducerIds(i), + batchPushData.epochId + ) + val writer: FileWriter = partitionInfo.asInstanceOf[WritablePartitionInfo].getFileWriter + val byteBuf = buf.slice( + buf.readerIndex() + batchPushData.offsets(i), + batchPushData.offsets(i + 1) - batchPushData.offsets(i) + ) + + val isWritten = writePushDataRequest( + batchPushData.shuffleKey, + batchPushData.reducerIds(i), + batchPushData.epochId, + batchPushData.mapperId, + byteBuf, writer, true, callback) + if (!isWritten) { + return false + } + } + true + } catch { + case t: Throwable => + callback.onFailure(t) + logError(s"writeBatchPushDataRequest failed with buf size ${buf.readableBytes()}", t) + false + } + } + + override def handleBatchPushDataRequest(batchPushData: BatchPushDataRequest, callback: RpcResponseCallback): Unit = { + // this callback will carry metrics. + val wrappedCallback = callback + + val valid = statusValidation( + batchPushData.shuffleKey, + batchPushData.reducerIds, + Array.fill(batchPushData.reducerIds.length)(batchPushData.epochId), + batchPushData.shuffleMode, + batchPushData.replicaIndex, + batchPushData.epochRotateThreshold, true, wrappedCallback + ) + + // batchPushData ignoreRotate = true + // can continue writing only when the return value is WritableButShouldRotate & Writable + // writableButShouldRotate needs to specify the return value to notify the client + if (valid != FileWriterStatus.Writable && valid != FileWriterStatus.WritableButShouldRotate) { + return + } + + val written = writeBatchPushDataRequest(batchPushData, wrappedCallback) + if (written) { + valid match { + case FileWriterStatus.Writable => + wrappedCallback.onSuccess(ByteBuffer.wrap(Array[Byte]())) + case FileWriterStatus.WritableButShouldRotate => + wrappedCallback.onSuccess(ByteBuffer.wrap(Array[Byte](CssStatusCode.EpochShouldRotate.getValue))) + case _ => throw new IOException("Wrong FileWriterStatus, should not reach here.") + } + } else { + // If the callback has been called before, this callback will be ignored. + wrappedCallback.onFailure(new Exception("Other Exception")) + } + } + + // return boolean value indicates whether need to do replication + private def writePushDataRequest( + shuffleKey: String, + reducerId: Int, + epochId: Int, + mapperId: Int, + byteBuf: ByteBuf, + writer: FileWriter, + ignoreRotate: Boolean, + callback: RpcResponseCallback): Boolean = { + // after fast ask, should continue to do write operation + writer.incrementPendingWrites() + try { + writer.write(byteBuf, mapperId, ignoreRotate) + callback match { + case wrapPushMetricsCallBack: WrapPushMetricsCallBack => + wrapPushMetricsCallBack.getDataThroughputMetrics.mark(byteBuf.readableBytes()) + case _ => + } + return true + } catch { + case _: AlreadyClosedException => + // Async write on both master and slave should not return failure logging only. + // Could be speculative task still pushData to an cleaned shuffle partition + // Silent passed for this case + logWarning(s"Try to write into a closed/committed shuffle partition, ${shuffleKey} ${reducerId} ${epochId}") + case _: EpochShouldRotateException => + // normally, when hit with exception, we use onFailure + // but EpochRotation will be a high frequency operation in big shuffle + // it will introduce some many error log if use onFailure API + logError(s"Hit EpochShouldRotateException, ${shuffleKey} ${reducerId} ${epochId}") + callback.onSuccess(ByteBuffer.wrap(Array[Byte](CssStatusCode.EpochShouldRotate.getValue))) + case ex: Throwable => + logError(s"Hit Unknown Exception, ${shuffleKey} ${reducerId} ${epochId}", ex) + callback.onFailure(ex) + } + writer.decrementPendingWrites() + false + } + + override def handleOpenStreamRequest(shuffleKey: String, filePath: String): CssFileInfo = { + // find FileWriter responsible for the data + val fileWriter = storageManager.getFileWriter(shuffleKey, filePath) + if (fileWriter == null) { + return null + } else { + new CssFileInfo(fileWriter.getFile, fileWriter.getChunkOffsets, fileWriter.getFileLength) + } + } + + override def recycleShuffle(expiredShuffleKeys: util.HashSet[String]): Unit = { + def cleanMetaData(expiredShuffleKeys: util.HashSet[String]): Unit = { + expiredShuffleKeys.asScala.foreach { shuffleKey => + workerInfo.removeShufflePartitions(shuffleKey) + committedInfo.remove(shuffleKey) + } + } + cleanMetaData(expiredShuffleKeys) + storageManager.addCleanupExpiredTask(ShuffleKeyExpiredTask(expiredShuffleKeys)) + } + + override def recycleApplication(expiredApplicationIds: util.HashSet[String]): Unit = { + // since try to cleanup expired app. try to clean all shuffle with target appId first. + expiredApplicationIds.asScala.foreach { appId => + val expiredShuffleKeys = workerInfo.getAllShuffleKeyByAppId(appId) + if (!expiredShuffleKeys.isEmpty) { + logInfo("try to clean up expired shuffle keys with no unregister shuffle. " + expiredShuffleKeys) + recycleShuffle(new util.HashSet[String](expiredShuffleKeys)) + } + } + storageManager.addCleanupExpiredTask(AppExpiredTask(expiredApplicationIds)) + } +} + +object Worker extends Logging { + + var workers: ArrayBuffer[Worker] = new ArrayBuffer[Worker]() + + def main(args: Array[String]): Unit = { + val conf = new CssConf + val workerArgs = new WorkerArguments(args, conf) + + if (!Utils.checkCssConfLegality(conf)) { + System.exit(13) + } + + val rpcEnv = RpcEnv.create(RpcNameConstants.WORKER_SYS, + workerArgs.bindHost, + workerArgs.host, + workerArgs.port, + conf, + 0, + false) + + val worker = new Worker(rpcEnv, workerArgs.pushPort, workerArgs.fetchPort, conf) + workers.synchronized { + workers.asJava.add(worker) + } + rpcEnv.setupEndpoint(RpcNameConstants.WORKER_EP, worker) + ShutdownHookManager.get().addShutdownHook(new Thread { + override def run(): Unit = { + logInfo(s"Worker shutting down, hook execution.") + if (worker != null) { + rpcEnv.shutdown() + } + } + }, 50) + rpcEnv.awaitTermination() + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/worker/WorkerArguments.scala b/service/src/main/scala/com/bytedance/css/service/deploy/worker/WorkerArguments.scala new file mode 100644 index 0000000..188143e --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/worker/WorkerArguments.scala @@ -0,0 +1,115 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker + +import scala.annotation.tailrec + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.util.{IntParam, Utils} + +class WorkerArguments(args: Array[String], conf: CssConf) { + + var bindHost = "0.0.0.0" + var host = Utils.localHostName() + // for rpc control message port + // pushPort and fetchPort should be configured at CssConf + var port = 0 + + var pushPort = -1 + var fetchPort = -1 + + // default css master url for workers to register + var master: String = s"css://$host:9099" + var propertiesFile: String = null + + parse(args.toList) + + propertiesFile = Utils.loadDefaultCssProperties(conf, propertiesFile) + + @tailrec + private def parse(args: List[String]): Unit = args match { + case ("--bindHost" | "-bh") :: value :: tail => + Utils.checkHost(value) + bindHost = value + parse(tail) + + case ("--host" | "-h") :: value :: tail => + Utils.checkHost(value) + host = value + parse(tail) + + case ("--port" | "-p") :: IntParam(value) :: tail => + port = value + parse(tail) + + case ("--pushPort" | "-pp") :: IntParam(value) :: tail => + pushPort = value + parse(tail) + + case ("--fetchPort" | "-fp") :: IntParam(value) :: tail => + fetchPort = value + parse(tail) + + case ("--properties-file") :: value :: tail => + propertiesFile = value + parse(tail) + + case ("--conf") :: confKey :: confValue :: tail => + conf.set(confKey, confValue) + parse(tail) + + case ("--help") :: tail => + printUsageAndExit(0) + + case value :: tail => + master = value + conf.set("css.master.address", master) + parse(tail) + + case Nil => + if (master == null) { // No positional argument was given + printUsageAndExit(1) + } + + case _ => + printUsageAndExit(1) + } + + /** + * Print usage and exit JVM with the given exit code. + */ + def printUsageAndExit(exitCode: Int) { + // scalastyle:off println + System.err.println( + "Usage: Worker [options] \n" + + "\n" + + "Master must be a URL of the form css://hostname:port\n" + + "\n" + + "Options:\n" + + " -h HOST, --host HOST Hostname to listen on\n" + + " -p PORT, --port PORT RpcPort to listen on (default: random)\n" + + " -pp PUSHPORT, --pushPort PUSHPORT PushPort to listen on (default: random)\n" + + " -fp FETCHPORT, --fetchPort FETCHPORT FetchPort to listen on (default: random)\n" + + " --properties-file FILE Path to a custom CSS properties file.\n" + + " Default is conf/css-defaults.conf.") + // scalastyle:on println + System.exit(exitCode) + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/worker/WorkerInfo.scala b/service/src/main/scala/com/bytedance/css/service/deploy/worker/WorkerInfo.scala new file mode 100644 index 0000000..4dcb772 --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/worker/WorkerInfo.scala @@ -0,0 +1,155 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker + +import java.util +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ + +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.protocol.{PartitionInfo, WorkerStatus} +import com.bytedance.css.common.rpc.RpcEndpointRef +import com.bytedance.css.common.rpc.netty.NettyRpcEndpointRef +import com.bytedance.css.common.util.Utils +import io.netty.util.internal.ConcurrentSet + +class WorkerInfo( + val name: String, // unique name for this worker + val host: String, // hostname for this worker + val rpcPort: Int, // rpc port for control message + val pushPort: Int, // rpc port for push data call + val fetchPort: Int, // rpc port for fetch data call + var workerRpcRef: RpcEndpointRef) // rpc ref for sending rpc to this worker, for control message flow + extends Serializable with Logging { + + def this( + host: String, // hostname for this worker + rpcPort: Int, // rpc port for control message + pushPort: Int, // rpc port for push data call + fetchPort: Int, // rpc port for fetch data call + workerRpcRef: RpcEndpointRef) = { + this(host, host, rpcPort, pushPort, fetchPort, workerRpcRef) + } + + var rttAvgStat: Long = 0L + + Utils.checkHost(host) + assert(rpcPort > 0) + assert(pushPort > 0) + assert(fetchPort > 0) + + // mark current app has lost or finish + val appFinishSet = new ConcurrentSet[String]() + + // heartbeat timestamp that kept and updated in master node to determined whether the target worker is dead + var lastHeartbeat: Long = System.currentTimeMillis() + + protected val partitionMapV2 = new ConcurrentHashMap[String, PartitionInfo]() + protected val shuffleKeySetV2 = new ConcurrentSet[String]() + + // TODO current conditions are always true + // rpcEndpointRef with client args will always be null. because this.client will be set by server side. + def isActive(): Boolean = { + workerRpcRef != null || workerRpcRef.asInstanceOf[NettyRpcEndpointRef].client.isActive + } + + def hostPort: String = { + host + ":" + rpcPort + } + + def shuffleKeySet(): util.HashSet[String] = { + new util.HashSet[String](shuffleKeySetV2) + } + + def getAllShufflePartitions(shuffleKey: String): util.List[PartitionInfo] = { + if (shuffleKeySetV2.contains(shuffleKey)) { + val allEpochKeys = partitionMapV2.keySet().asScala.filter(k => k.startsWith(s"${shuffleKey}-")) + allEpochKeys.map(k => partitionMapV2.get(k)).toList.asJava + } else { + List.empty[PartitionInfo].asJava + } + } + + def getAllShuffleKeyByAppId(appId: String): util.List[String] = { + shuffleKeySetV2.asScala.filter(_.startsWith(s"${appId}-")).toList.asJava + } + + def addShufflePartition(shuffleKey: String, partitionInfo: PartitionInfo): Unit = { + if (partitionInfo != null) { + shuffleKeySetV2.add(shuffleKey) + val epochKey = Utils.getEpochKeyWithShuffleKey(shuffleKey, partitionInfo.getReducerId, partitionInfo.getEpochId) + partitionMapV2.putIfAbsent(epochKey, partitionInfo) + } + } + + def getShufflePartition(shuffleKey: String, reducerId: Int, epochId: Int): PartitionInfo = { + val epochKey = Utils.getEpochKeyWithShuffleKey(shuffleKey, reducerId, epochId) + partitionMapV2.get(epochKey) + } + + def removeShufflePartitions(shuffleKey: String): Unit = { + if (shuffleKeySetV2.contains(shuffleKey)) { + val allEpochKeys = partitionMapV2.keySet().asScala.filter(k => k.startsWith(s"${shuffleKey}-")) + allEpochKeys.foreach(k => partitionMapV2.remove(k)) + shuffleKeySetV2.remove(shuffleKey) + } + } + + def containsShuffle(shuffleKey: String): Boolean = { + shuffleKeySetV2.contains(shuffleKey) + } + + override def toString(): String = { + s""" + |Name: $name + |Address: $hostPort + |PushPort: $pushPort + |FetchPort: $fetchPort + """.stripMargin + } + + override def equals(obj: Any): Boolean = if (!obj.isInstanceOf[WorkerInfo]) { + false + } else { + name.equals(obj.asInstanceOf[WorkerInfo].name) + } + + override def hashCode(): Int = { + name.hashCode() + } +} + +object WorkerInfo { + def toWorkerStatus(workerInfo: WorkerInfo, rttAvgStat: Long): WorkerStatus = { + new WorkerStatus( + workerInfo.name, + workerInfo.host, + workerInfo.rpcPort, + workerInfo.pushPort, + workerInfo.fetchPort, + rttAvgStat, + System.currentTimeMillis()) + } + + def fromWorkerStatus(status: WorkerStatus, workerRpcRef: RpcEndpointRef): WorkerInfo = { + new WorkerInfo(status.name, status.host, status.rpcPort, status.pushPort, status.fetchPort, workerRpcRef) + } +} diff --git a/service/src/main/scala/com/bytedance/css/service/deploy/worker/WorkerSource.scala b/service/src/main/scala/com/bytedance/css/service/deploy/worker/WorkerSource.scala new file mode 100644 index 0000000..ae2fd72 --- /dev/null +++ b/service/src/main/scala/com/bytedance/css/service/deploy/worker/WorkerSource.scala @@ -0,0 +1,84 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker + +import java.lang.management.ManagementFactory + +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.util.Utils +import com.bytedance.css.service.deploy.common.BaseSource +import com.codahale.metrics.{Gauge, MetricSet} +import com.codahale.metrics.jvm.{BufferPoolMetricSet, GarbageCollectorMetricSet, MemoryUsageGaugeSet} + +class WorkerSource( + namespace: String, + serverId: String) + extends BaseSource(namespace, serverId) { + + override def sourceName: String = "" + + // Gauge for exists worker instance + metricRegistry.register(cssMetricsPrefix + s"#worker.exists", + new Gauge[Int] { override def getValue: Int = 1 } + ) + + // Worker Heap Memory metrics. + // when use scala-2.11 in com.codahale.metrics.MetricRegistry + // registerAll(java.lang.String,com.codahale.metrics.MetricSet) has private access + try { + val clz = Utils.classForName("com.codahale.metrics.MetricRegistry") + val registerAllMethod = clz.getDeclaredMethod("registerAll", classOf[String], classOf[MetricSet]) + registerAllMethod.setAccessible(true) + registerAllMethod.invoke(metricRegistry, cssMetricsPrefix + s"#jvm.gc", new GarbageCollectorMetricSet) + registerAllMethod.invoke(metricRegistry, cssMetricsPrefix + s"#jvm.heap.memory", new MemoryUsageGaugeSet) + registerAllMethod.invoke(metricRegistry, cssMetricsPrefix + s"#jvm.buffer.pool", + new BufferPoolMetricSet(ManagementFactory.getPlatformMBeanServer)) + } catch { + case t: Throwable => + throw new RuntimeException(t) + } + + // WorkerLostEvent + val WORKER_LOST_EVENT = metricRegistry.meter(cssMetricsPrefix + s"|worker=$serverId#worker.lost.event") + + // CommitFiles Qps & Latency + val COMMIT_FILES_QPS = getEventQPS("CommitFiles") + val COMMIT_FILES_LATENCY = getEventLatency("CommitFiles") + + // CloseFile Qps & Latency + val CLOSE_FILE_QPS = getEventQPS("CloseFile") + val CLOSE_FILE_LATENCY = getEventLatency("CloseFile") +} + +object WorkerSource extends Logging { + + // only for test + private val dummyWorkerSource: WorkerSource = new WorkerSource("dummy", "test") + + @volatile var workerSource: WorkerSource = dummyWorkerSource + + def create(namespace: String, serverId: String): WorkerSource = synchronized { + if (workerSource == null || workerSource == dummyWorkerSource) { + workerSource = new WorkerSource(namespace, serverId) + logInfo(s"${workerSource.getClass.getName} use cssMetricsPrefix ${workerSource.cssMetricsPrefix}") + } + workerSource + } +} diff --git a/service/src/test/java/com/bytedance/css/service/deploy/worker/DiskFileFlushTaskSuite.java b/service/src/test/java/com/bytedance/css/service/deploy/worker/DiskFileFlushTaskSuite.java new file mode 100644 index 0000000..765a5f1 --- /dev/null +++ b/service/src/test/java/com/bytedance/css/service/deploy/worker/DiskFileFlushTaskSuite.java @@ -0,0 +1,68 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import org.junit.Test; + +import java.io.File; +import java.io.FileOutputStream; +import java.nio.channels.FileChannel; + +import static org.junit.Assert.*; + +public class DiskFileFlushTaskSuite { + + private String tmp = "DiskFileFlushTaskSuite java.nio.channels.FileChannel"; + + @Test + public void testFlush() throws Exception { + ByteBuf buffer = Unpooled.wrappedBuffer(tmp.getBytes()); + long length = buffer.readableBytes(); + + File file = File.createTempFile("DiskFileFlushTaskSuite", "testFlush"); + file.deleteOnExit(); + FileNotifier notifier = new FileNotifier(); + FileChannel channel = new FileOutputStream(file).getChannel(); + + DiskFileFlushTask task = new DiskFileFlushTask(buffer, notifier, channel); + task.flush(); + channel.close(); + assertFalse(notifier.hasException()); + assertEquals(file.length(), length); + } + + @Test + public void testFlushWithError() throws Exception { + ByteBuf buffer = Unpooled.wrappedBuffer(tmp.getBytes()); + + File file = File.createTempFile("DiskFileFlushTaskSuite", "testFlush"); + file.deleteOnExit(); + FileNotifier notifier = new FileNotifier(); + FileChannel channel = new FileOutputStream(file).getChannel(); + // close change in order to trigger flush error + channel.close(); + + DiskFileFlushTask task = new DiskFileFlushTask(buffer, notifier, channel); + task.flush(); + assertTrue(notifier.hasException()); + } +} diff --git a/service/src/test/java/com/bytedance/css/service/deploy/worker/DiskFileFlusherSuite.java b/service/src/test/java/com/bytedance/css/service/deploy/worker/DiskFileFlusherSuite.java new file mode 100644 index 0000000..3397c46 --- /dev/null +++ b/service/src/test/java/com/bytedance/css/service/deploy/worker/DiskFileFlusherSuite.java @@ -0,0 +1,93 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.nio.channels.FileChannel; + +import static org.junit.Assert.assertEquals; + +public class DiskFileFlusherSuite { + + private FileFlusherImpl flusher = null; + + @Before + public void createDiskFileFlusher() throws Exception { + flusher = new FileFlusherImpl("DiskFileFlusherTest", FileFlusher.DISK_FLUSHER_TYPE, 4, 128 * 1024); + } + + @Test + public void testFlush() throws Exception { + String tmp = "Hello-World!!!"; + ByteBuf buffer = Unpooled.wrappedBuffer(tmp.getBytes()); + int bufferLength = buffer.readableBytes(); + File file = File.createTempFile("DiskFileFlusherSuite", "testFlush"); + file.deleteOnExit(); + FileNotifier notifier = new FileNotifier(); + FileChannel channel = new FileOutputStream(file).getChannel(); + + DiskFileFlushTask task = new DiskFileFlushTask(buffer, notifier, channel); + flusher.submitTask(task, 1000); + + Thread.sleep(1000); + channel.close(); + + FileInputStream inputStream = new FileInputStream(file); + + byte[] bytes = new byte[bufferLength]; + String result = null; + while (inputStream.read(bytes) != -1){ + result = new String(bytes); + } + inputStream.close(); + assertEquals(result, tmp); + } + + @Test + public void testContinuousFlushTask() throws Exception { + String tmp = "Hello-World!!! testContinuousFlushTask"; + File file = File.createTempFile("DiskFileFlusherSuite", "testContinuousFlushTask"); + file.deleteOnExit(); + FileNotifier notifier = new FileNotifier(); + FileChannel channel = new FileOutputStream(file).getChannel(); + + long totalFlushBytes = 0; + + for (int i = 0; i < 1000; i ++) { + ByteBuf buffer = Unpooled.wrappedBuffer(tmp.getBytes()); + totalFlushBytes += buffer.readableBytes(); + notifier.getNumPendingFlushes().incrementAndGet(); + DiskFileFlushTask task = new DiskFileFlushTask(buffer, notifier, channel); + flusher.submitTask(task, 1000); + } + + Thread.sleep(3000); + assertEquals(notifier.getNumPendingFlushes().get(), 0); + channel.close(); + assertEquals(totalFlushBytes, file.length()); + } +} diff --git a/service/src/test/java/com/bytedance/css/service/deploy/worker/DiskFileWriterSuite.java b/service/src/test/java/com/bytedance/css/service/deploy/worker/DiskFileWriterSuite.java new file mode 100644 index 0000000..75fa983 --- /dev/null +++ b/service/src/test/java/com/bytedance/css/service/deploy/worker/DiskFileWriterSuite.java @@ -0,0 +1,309 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.CompositeByteBuf; +import io.netty.buffer.Unpooled; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Random; +import java.util.concurrent.atomic.AtomicLong; + +import static org.junit.Assert.assertEquals; + +public class DiskFileWriterSuite { + + private FileFlusherImpl flusher = null; + + @Before + public void createDiskFileFlusher() throws Exception { + flusher = new FileFlusherImpl("DiskFileWriterTest", FileFlusher.DISK_FLUSHER_TYPE, 4, 128 * 1024); + } + + @Test + public void testSplitChunkValid() throws Exception { + File file = File.createTempFile("DiskFileWriterSuite", "testSplitChunkValid"); + file.deleteOnExit(); + + DiskFileWriter writer = new DiskFileWriter(file, flusher, 8 * 1024 * 1024L, 30000, 128 * 1024, 512 * 1024 * 1024L); + + Random rand = new Random(); + AtomicLong fileLength = new AtomicLong(0); + Thread normalThread = new Thread() { + @Override + public void run() { + for (int i = 0; i < 1000; i ++) { + int length = 1 + rand.nextInt(16 * 1024); + fileLength.addAndGet(length); + byte[] tmp = new byte[length]; + tmp[length - 1] = 77; // magic num to check chunk split boundary + ByteBuf buf = Unpooled.wrappedBuffer(tmp); + try { + writer.write(buf, 0); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + } + }; + + Thread giantThread = new Thread() { + @Override + public void run() { + for (int i = 0; i < 100; i ++) { + int length = 128 * 1024 + rand.nextInt(100) + 1; + fileLength.addAndGet(length); + byte[] tmp = new byte[length]; + tmp[length - 1] = 77; // magic num to check chunk split boundary + ByteBuf buf = Unpooled.wrappedBuffer(tmp); + try { + writer.write(buf, 0); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + } + }; + + normalThread.start(); + giantThread.start(); + normalThread.join(); + giantThread.join(); + + writer.close(); + assertEquals(file.length(), fileLength.get()); + + ArrayList chunkOffsets = writer.getChunkOffsets(); + int numChunks = chunkOffsets.size() - 1; + FileInputStream inputStream = new FileInputStream(file); + for (int i = 0; i < numChunks; i ++) { + long start = chunkOffsets.get(i); + long end = chunkOffsets.get(i + 1); + int length = (int)(end - start); + byte[] tmp = new byte[length]; + assertEquals(inputStream.read(tmp), length); + assertEquals(tmp[length - 1], 77); + } + inputStream.close(); + } + + @Test + public void testNoMemoryLeakIfWriteWithNotifyException() throws Exception { + File file = File.createTempFile("DiskFileWriterSuite", "testNoMemoryLeakIfWriteWithNotifyException"); + file.deleteOnExit(); + + DiskFileWriter writer = new DiskFileWriter(file, flusher, 8 * 1024 * 1024L, 30000, 128 * 1024, 512 * 1024 * 1024L); + + Random rand = new Random(); + AtomicLong fileLength = new AtomicLong(0); + + // first flush small buf to only write into composite buf. + for (int i = 0; i < 100; i ++) { + int length = 1 + rand.nextInt(16 * 1024); + fileLength.addAndGet(length); + byte[] tmp = new byte[length]; + tmp[length - 1] = 77; // magic num to check chunk split boundary + ByteBuf buf = Unpooled.wrappedBuffer(tmp); + try { + writer.write(buf, 0); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + writer.setException(); + + // write giant batch to just to flush. + { + int length = 1 + 129 * 1024; + fileLength.addAndGet(length); + byte[] tmp = new byte[length]; + tmp[length - 1] = 77; // magic num to check chunk split boundary + ByteBuf buf = Unpooled.wrappedBuffer(tmp); + try { + writer.write(buf, 0); + } catch (Exception ex) { + assert (ex instanceof IOException); + assert (ex.getMessage().equalsIgnoreCase("ForTestOnly")); + ex.printStackTrace(); + } + assert (buf.refCnt() == 1); + } + + // write common small buf to composite. + { + int length = 1 + 1 * 1024; + fileLength.addAndGet(length); + byte[] tmp = new byte[length]; + tmp[length - 1] = 77; // magic num to check chunk split boundary + ByteBuf buf = Unpooled.wrappedBuffer(tmp); + try { + writer.write(buf, 0); + } catch (Exception ex) { + assert (ex instanceof IOException); + assert (ex.getMessage().equalsIgnoreCase("ForTestOnly")); + ex.printStackTrace(); + } + assert (buf.refCnt() == 1); + } + + CompositeByteBuf compositeByteBuf = writer.getCompositeByteBuf(); + assert (compositeByteBuf.refCnt() == 1); + + try { + writer.close(); + } catch (IOException e) { + assert (e instanceof IOException); + assert (e.getMessage().equalsIgnoreCase("ForTestOnly")); + e.printStackTrace(); + } + + assert (compositeByteBuf.refCnt() == 0); + assert (writer.getCompositeByteBuf() == null); + + writer.destroy(); + } + + @Test + public void testNoMemoryLeakWithSubmitFlushTaskFullWithGiantBatch() throws Exception { + File file = File.createTempFile("DiskFileWriterSuite", "testNoMemoryLeakIfWriteWithNotifyException"); + file.deleteOnExit(); + + DiskFileWriter writer = new DiskFileWriter(file, flusher, 8 * 1024 * 1024L, 30000, 128 * 1024, 512 * 1024 * 1024L); + + Random rand = new Random(); + AtomicLong fileLength = new AtomicLong(0); + + // first flush small buf to only write into composite buf. + for (int i = 0; i < 100; i ++) { + int length = 1 + rand.nextInt(16 * 1024); + fileLength.addAndGet(length); + byte[] tmp = new byte[length]; + tmp[length - 1] = 77; // magic num to check chunk split boundary + ByteBuf buf = Unpooled.wrappedBuffer(tmp); + try { + writer.write(buf, 0); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + writer.setSubmitQueueFullException(true); + + // write giant batch to just to flush. it will be failed & buf will be release. + { + int length = 1 + 129 * 1024; + fileLength.addAndGet(length); + byte[] tmp = new byte[length]; + tmp[length - 1] = 77; // magic num to check chunk split boundary + ByteBuf buf = Unpooled.wrappedBuffer(tmp); + try { + writer.write(buf, 0); + } catch (Exception ex) { + assert (ex instanceof IOException); + assert (ex.getMessage().startsWith("DiskFileWriter submit flush task timeout")); + ex.printStackTrace(); + } + assert (buf.refCnt() == 1); + } + + CompositeByteBuf compositeByteBuf = writer.getCompositeByteBuf(); + assert (compositeByteBuf.refCnt() == 1); + + try { + writer.close(); + } catch (IOException e) { + assert (e instanceof IOException); + assert (e.getMessage().startsWith("DiskFileWriter submit flush task timeout")); + e.printStackTrace(); + } + + assert (compositeByteBuf.refCnt() == 0); + assert (writer.getCompositeByteBuf() == null); + + writer.destroy(); + } + + @Test + public void testNoMemoryLeakWithSubmitFlushTaskFullWithSmallBatch() throws Exception { + File file = File.createTempFile("DiskFileWriterSuite", "testNoMemoryLeakIfWriteWithNotifyException"); + file.deleteOnExit(); + + DiskFileWriter writer = new DiskFileWriter(file, flusher, 8 * 1024 * 1024L, 30000, 128 * 1024, 512 * 1024 * 1024L); + + Random rand = new Random(); + AtomicLong fileLength = new AtomicLong(0); + + // first flush small buf to only write into composite buf. + for (int i = 0; i < 100; i ++) { + int length = 1 + rand.nextInt(16 * 1024); + fileLength.addAndGet(length); + byte[] tmp = new byte[length]; + tmp[length - 1] = 77; // magic num to check chunk split boundary + ByteBuf buf = Unpooled.wrappedBuffer(tmp); + try { + writer.write(buf, 0); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + writer.setSubmitQueueFullException(true); + + // write common small buf to composite. + { + int length = 1 + 128 * 1024 - writer.getCompositeByteBuf().readableBytes(); + fileLength.addAndGet(length); + byte[] tmp = new byte[length]; + tmp[length - 1] = 77; // magic num to check chunk split boundary + ByteBuf buf = Unpooled.wrappedBuffer(tmp); + try { + writer.write(buf, 0); + } catch (Exception ex) { + assert (ex instanceof IOException); + assert (ex.getMessage().startsWith("DiskFileWriter submit flush task timeout")); + ex.printStackTrace(); + } + assert (buf.refCnt() == 1); + } + + CompositeByteBuf compositeByteBuf = writer.getCompositeByteBuf(); + assert (compositeByteBuf.refCnt() == 1); + + try { + writer.close(); + } catch (IOException e) { + assert (e instanceof IOException); + assert (e.getMessage().startsWith("DiskFileWriter submit flush task timeout")); + e.printStackTrace(); + } + + assert (compositeByteBuf.refCnt() == 0); + assert (writer.getCompositeByteBuf() == null); + + writer.destroy(); + } +} diff --git a/service/src/test/java/com/bytedance/css/service/deploy/worker/FileNotifierSuite.java b/service/src/test/java/com/bytedance/css/service/deploy/worker/FileNotifierSuite.java new file mode 100644 index 0000000..d108622 --- /dev/null +++ b/service/src/test/java/com/bytedance/css/service/deploy/worker/FileNotifierSuite.java @@ -0,0 +1,45 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker; + +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class FileNotifierSuite { + + @Test + public void test() { + FileNotifier notifier = new FileNotifier(); + assert !notifier.hasException(); + + notifier.setException(new IOException("Hello")); + assertTrue(notifier.hasException()); + + try { + notifier.checkException(); + } catch (IOException ex) { + assertEquals(ex.getMessage(), "Hello"); + } + } +} diff --git a/service/src/test/java/com/bytedance/css/service/deploy/worker/HdfsFileFlushTaskSuite.java b/service/src/test/java/com/bytedance/css/service/deploy/worker/HdfsFileFlushTaskSuite.java new file mode 100644 index 0000000..0f95cfc --- /dev/null +++ b/service/src/test/java/com/bytedance/css/service/deploy/worker/HdfsFileFlushTaskSuite.java @@ -0,0 +1,77 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.junit.Test; + +import java.io.File; + +import static org.junit.Assert.*; + +public class HdfsFileFlushTaskSuite { + + private String tmp = "HdfsFileFlushTaskSuite maybe let me do it !"; + + @Test + public void testFlush() throws Exception { + ByteBuf buffer = Unpooled.wrappedBuffer(tmp.getBytes()); + long length = buffer.readableBytes(); + + File file = File.createTempFile("HdfsFileFlushTaskSuite", "testFlush"); + file.deleteOnExit(); + FileNotifier notifier = new FileNotifier(); + Path path = new Path(String.format("file://%s", file.getAbsolutePath())); + FileSystem fs = FileSystem.getLocal(new Configuration()); + FSDataOutputStream outputStream = fs.create(path); + + HdfsFileFlushTask task = new HdfsFileFlushTask(buffer, notifier, outputStream); + task.flush(); + outputStream.close(); + fs.close(); + assertFalse(notifier.hasException()); + assertEquals(file.length(), length); + } + + @Test + public void testFlushWithError() throws Exception { + ByteBuf buffer = Unpooled.wrappedBuffer(tmp.getBytes()); + + File file = File.createTempFile("HdfsFileFlushTaskSuite", "testFlush"); + file.deleteOnExit(); + FileNotifier notifier = new FileNotifier(); + Path path = new Path(String.format("file://%s", file.getAbsolutePath())); + FileSystem fs = FileSystem.getLocal(new Configuration()); + FSDataOutputStream outputStream = fs.create(path); + // close outputStream in order to trigger flush error + outputStream.close(); + fs.close(); + + HdfsFileFlushTask task = new HdfsFileFlushTask(buffer, notifier, outputStream); + task.flush(); + assertTrue(notifier.hasException()); + } + +} diff --git a/service/src/test/java/com/bytedance/css/service/deploy/worker/HdfsFileFlusherSuite.java b/service/src/test/java/com/bytedance/css/service/deploy/worker/HdfsFileFlusherSuite.java new file mode 100644 index 0000000..3724637 --- /dev/null +++ b/service/src/test/java/com/bytedance/css/service/deploy/worker/HdfsFileFlusherSuite.java @@ -0,0 +1,101 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.FileInputStream; + +import static org.junit.Assert.assertEquals; + +public class HdfsFileFlusherSuite { + + private FileFlusherImpl flusher = null; + + @Before + public void createHdfsFileFlusher() throws Exception { + flusher = new FileFlusherImpl("HdfsFileFlusherTest", FileFlusher.HDFS_FLUSHER_TYPE, 4, 128 * 1024); + } + + @Test + public void testFlush() throws Exception { + String tmp = "Hello-World!!!"; + ByteBuf buffer = Unpooled.wrappedBuffer(tmp.getBytes()); + int bufferLength = buffer.readableBytes(); + File file = File.createTempFile("HdfsFileFlusherSuite", "testFlush"); + file.deleteOnExit(); + FileNotifier notifier = new FileNotifier(); + Path path = new Path(String.format("file://%s", file.getAbsolutePath())); + FileSystem fs = FileSystem.getLocal(new Configuration()); + FSDataOutputStream outputStream = fs.create(path); + + HdfsFileFlushTask task = new HdfsFileFlushTask(buffer, notifier, outputStream); + flusher.submitTask(task, 1000); + + Thread.sleep(1000); + outputStream.close(); + + FileInputStream inputStream = new FileInputStream(file); + + byte[] bytes = new byte[bufferLength]; + String result = null; + while (inputStream.read(bytes) != -1){ + result = new String(bytes); + } + inputStream.close(); + fs.close(); + assertEquals(result, tmp); + } + + @Test + public void testContinuousFlushTask() throws Exception { + String tmp = "Hello-World!!! testContinuousFlushTask"; + File file = File.createTempFile("HdfsFileFlusherSuite", "testContinuousFlushTask"); + file.deleteOnExit(); + FileNotifier notifier = new FileNotifier(); + Path path = new Path(String.format("file://%s", file.getAbsolutePath())); + FileSystem fs = FileSystem.getLocal(new Configuration()); + FSDataOutputStream outputStream = fs.create(path); + + long totalFlushBytes = 0; + + for (int i = 0; i < 1000; i ++) { + ByteBuf buffer = Unpooled.wrappedBuffer(tmp.getBytes()); + totalFlushBytes += buffer.readableBytes(); + notifier.getNumPendingFlushes().incrementAndGet(); + HdfsFileFlushTask task = new HdfsFileFlushTask(buffer, notifier, outputStream); + flusher.submitTask(task, 1000); + } + + Thread.sleep(3000); + assertEquals(notifier.getNumPendingFlushes().get(), 0); + outputStream.close(); + fs.close(); + assertEquals(totalFlushBytes, file.length()); + } +} diff --git a/service/src/test/java/com/bytedance/css/service/deploy/worker/HdfsFileWriterSuite.java b/service/src/test/java/com/bytedance/css/service/deploy/worker/HdfsFileWriterSuite.java new file mode 100644 index 0000000..ba207b3 --- /dev/null +++ b/service/src/test/java/com/bytedance/css/service/deploy/worker/HdfsFileWriterSuite.java @@ -0,0 +1,104 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.worker; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.util.Random; +import java.util.concurrent.atomic.AtomicLong; + +import static org.junit.Assert.assertEquals; + +public class HdfsFileWriterSuite { + + private FileFlusherImpl flusher = null; + + @Before + public void createDiskFileFlusher() throws Exception { + flusher = new FileFlusherImpl("HdfsFileWriterTest", FileFlusher.HDFS_FLUSHER_TYPE, 4, 128 * 1024); + } + + @Test + public void testSplitChunkValid() throws Exception { + File file = File.createTempFile("HdfsFileWriterSuite", "testSplitChunkValid"); + file.deleteOnExit(); + + Path path = new Path(String.format("file://%s", file.getAbsolutePath())); + FileSystem fs = FileSystem.getLocal(new Configuration()); + FSDataOutputStream outputStream = fs.create(path); + + HdfsFileWriter writer = new HdfsFileWriter(fs, path, outputStream, flusher, 30000, 128 * 1024, 512 * 1024 * 1024L); + + Random rand = new Random(); + AtomicLong fileLength = new AtomicLong(0); + Thread normalThread = new Thread() { + @Override + public void run() { + for (int i = 0; i < 1000; i ++) { + int length = 1 + rand.nextInt(16 * 1024); + fileLength.addAndGet(length); + byte[] tmp = new byte[length]; + tmp[length - 1] = 77; // magic num to check chunk split boundary + ByteBuf buf = Unpooled.wrappedBuffer(tmp); + try { + writer.write(buf, 0); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + } + }; + + Thread giantThread = new Thread() { + @Override + public void run() { + for (int i = 0; i < 100; i ++) { + int length = 128 * 1024 + rand.nextInt(100) + 1; + fileLength.addAndGet(length); + byte[] tmp = new byte[length]; + ByteBuf buf = Unpooled.wrappedBuffer(tmp); + try { + writer.write(buf, 0); + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + } + }; + + normalThread.start(); + giantThread.start(); + normalThread.join(); + giantThread.join(); + + writer.close(); + outputStream.close(); + fs.close(); + assertEquals(file.length(), fileLength.get()); + } +} diff --git a/service/src/test/scala/com/bytedance/css/service/deploy/master/AssignStrategySuite.scala b/service/src/test/scala/com/bytedance/css/service/deploy/master/AssignStrategySuite.scala new file mode 100644 index 0000000..5b9bffc --- /dev/null +++ b/service/src/test/scala/com/bytedance/css/service/deploy/master/AssignStrategySuite.scala @@ -0,0 +1,139 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.protocol.PartitionGroup +import com.bytedance.css.common.rpc.RpcEndpointRef +import com.bytedance.css.service.deploy.worker.WorkerInfo +import org.scalatest.FunSuite + +class AssignStrategySuite extends FunSuite { + + val cssConf = new CssConf() + val assignStrategy: AssignStrategy = AssignStrategy.buildAssignStrategy(cssConf) + + val assignableWorkers = Seq( + new MockWorkerInfo("NeverMind"), + new MockWorkerInfo("NeverMind"), + new MockWorkerInfo("NeverMind"), + new MockWorkerInfo("NeverMind"), + new MockWorkerInfo("NeverMind"), + new MockWorkerInfo("NeverMind") + ) + + class MockWorkerInfo( + host: String, + rpcPort: Int = 1, + pushPort: Int = 1, + fetchPort: Int = 1, + workerRpcRef: RpcEndpointRef = null) extends WorkerInfo(host, rpcPort, pushPort, fetchPort, workerRpcRef) { + override def isActive(): Boolean = true + } + + test("assign partition group with diff group len") { + // because there replica = 2 by default. so target possible tuple size = 15. + + // here use maxPartitionsPerGroup as group len. + val numPartitions1 = 1002 + val maxPartitionsPerGroup1 = 20 + var partitionGroups = + assignStrategy.assignPartitionGroup(numPartitions1, maxPartitionsPerGroup1, assignableWorkers) + + assert(partitionGroups.size == 51) + partitionGroups.foreach(p => assert(p.epochId == 0)) + + assert(partitionGroups(0).startPartition == 0) + assert(partitionGroups(0).endPartition == 20) + assert(partitionGroups(0).replicaWorkers.size() == 2) + assert(partitionGroups(50).startPartition == 1000) + assert(partitionGroups(50).endPartition == 1002) + assert(partitionGroups(50).replicaWorkers.size() == 2) + + // here use numPartitions / possible tuple size as group len. + val numPartitions2 = 1002 + val maxPartitionsPerGroup2 = 1000 + partitionGroups = + assignStrategy.assignPartitionGroup(numPartitions2, maxPartitionsPerGroup2, assignableWorkers) + + assert(partitionGroups.size == 16) + partitionGroups.foreach(p => assert(p.epochId == 0)) + + assert(partitionGroups(0).startPartition == 0) + assert(partitionGroups(0).endPartition == 66) + assert(partitionGroups(0).replicaWorkers.size() == 2) + assert(partitionGroups(15).startPartition == 990) + assert(partitionGroups(15).endPartition == 1002) + assert(partitionGroups(15).replicaWorkers.size() == 2) + + // here use numPartitions / worker size as group len. + val numPartitions3 = 12 + val maxPartitionsPerGroup3 = 1000 + partitionGroups = + assignStrategy.assignPartitionGroup(numPartitions3, maxPartitionsPerGroup3, assignableWorkers) + + assert(partitionGroups.size == 6) + partitionGroups.foreach(p => assert(p.epochId == 0)) + + assert(partitionGroups(0).startPartition == 0) + assert(partitionGroups(0).endPartition == 2) + assert(partitionGroups(0).replicaWorkers.size() == 2) + assert(partitionGroups(5).startPartition == 10) + assert(partitionGroups(5).endPartition == 12) + assert(partitionGroups(5).replicaWorkers.size() == 2) + + // here use one partition one group as group len. + val numPartitions4 = 5 + val maxPartitionsPerGroup4 = 100 + partitionGroups = + assignStrategy.assignPartitionGroup(numPartitions4, maxPartitionsPerGroup4, assignableWorkers) + + assert(partitionGroups.size == 5) + partitionGroups.foreach(p => assert(p.epochId == 0)) + + assert(partitionGroups(0).startPartition == 0) + assert(partitionGroups(0).endPartition == 1) + assert(partitionGroups(0).replicaWorkers.size() == 2) + assert(partitionGroups(4).startPartition == 4) + assert(partitionGroups(4).endPartition == 5) + assert(partitionGroups(4).replicaWorkers.size() == 2) + } + + test("reallocate partition group") { + val oldPartitionGroup = new PartitionGroup(0, 0, 0, 10, null) + + val newPartitionGroup1 = assignStrategy.reallocatePartitionGroup(assignableWorkers, oldPartitionGroup) + + assert(newPartitionGroup1.partitionGroupId == oldPartitionGroup.partitionGroupId) + assert(newPartitionGroup1.epochId == oldPartitionGroup.epochId + 1) + assert(newPartitionGroup1.startPartition == oldPartitionGroup.startPartition) + assert(newPartitionGroup1.endPartition == oldPartitionGroup.endPartition) + assert(newPartitionGroup1.replicaWorkers.size() == 2) + + val newPartitionGroup2 = assignStrategy.reallocatePartitionGroup(assignableWorkers, newPartitionGroup1) + + assert(newPartitionGroup2.partitionGroupId == newPartitionGroup1.partitionGroupId) + assert(newPartitionGroup2.epochId == newPartitionGroup1.epochId + 1) + assert(newPartitionGroup2.startPartition == newPartitionGroup1.startPartition) + assert(newPartitionGroup2.endPartition == newPartitionGroup1.endPartition) + assert(newPartitionGroup2.replicaWorkers.size() == 2) + } + +} diff --git a/service/src/test/scala/com/bytedance/css/service/deploy/master/BatchPushDataSuite.scala b/service/src/test/scala/com/bytedance/css/service/deploy/master/BatchPushDataSuite.scala new file mode 100644 index 0000000..f8ea650 --- /dev/null +++ b/service/src/test/scala/com/bytedance/css/service/deploy/master/BatchPushDataSuite.scala @@ -0,0 +1,193 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import java.io.File +import java.util +import java.util.Random +import java.util.concurrent.CompletableFuture +import java.util.concurrent.atomic.AtomicLong + +import scala.collection.JavaConverters._ + +import com.bytedance.css.common.protocol.{PartitionInfo, ShuffleMode, WorkerAddress} +import com.bytedance.css.common.protocol.CssRpcMessage._ +import com.bytedance.css.common.util.Utils + +class BatchPushDataSuite extends LocalClusterSuite { + + val rand = new Random() + + test("batch pushData success") { + Seq(ShuffleMode.DISK, ShuffleMode.HDFS).foreach { shuffleMode => + val appId = s"appId-${System.currentTimeMillis()}" + val shuffleId = 5 + + val mapperId = 0 + val partitionInfo = new PartitionInfo(0, 0) + + val workers = Array( + new WorkerAddress(worker(0).rpcEnv.address.host, worker(0).pushPort), + new WorkerAddress(worker(1).rpcEnv.address.host, worker(1).pushPort)) + + var fileLength = 0L + val allfutureList = new util.ArrayList[CompletableFuture[_]]() + val futuresList = new util.ArrayList[Array[CompletableFuture[Int]]]() + (0 until 1000).foreach(index => { + // send pushData to replica piece. + val length: Int = 1 + rand.nextInt(16 * 1024) + fileLength += length + val tmp: Array[Byte] = new Array[Byte](length) + val (allFuture, futures) = batchPushData(appId, shuffleId, + Array(partitionInfo.getReducerId), partitionInfo.getEpochId, mapperId, + tmp, Array(0, tmp.length), workers, shuffleMode.toString) + allfutureList.add(allFuture) + futuresList.add(futures) + }) + allfutureList.asScala.foreach(_.get()) + assert(!futuresList.asScala.flatten.exists(_.get() != 0)) + + val committed1 = sendWorkerRpc[CommitFiles, CommitFilesResponse](worker(0).rpcPort, + CommitFiles(Utils.getShuffleKey(appId, shuffleId))) + val replica1 = committed1.committed.asScala.head + + val committed2 = sendWorkerRpc[CommitFiles, CommitFilesResponse](worker(1).rpcPort, + CommitFiles(Utils.getShuffleKey(appId, shuffleId))) + val replica2 = committed2.committed.asScala.head + + var r1Path = replica1.getFilePath + if (shuffleMode == ShuffleMode.HDFS) { + r1Path = r1Path.split(":").drop(1).mkString(":") + } + assert(replica1.getReducerId == 0) + assert(replica1.getEpochId == 0) + assert(new File(r1Path).length() == replica1.getFileLength) + + var r2Path = replica2.getFilePath + if (shuffleMode == ShuffleMode.HDFS) { + r2Path = r2Path.split(":").drop(1).mkString(":") + } + assert(replica2.getReducerId == 0) + assert(replica2.getEpochId == 0) + assert(new File(r2Path).length() == replica2.getFileLength) + + assert(replica1.getFileLength == replica2.getFileLength) + } + } + + test("batch pushData empty/failed/giant mix") { + Seq(ShuffleMode.DISK, ShuffleMode.HDFS).foreach { shuffleMode => + val appId = s"appId-${System.currentTimeMillis()}" + val shuffleId = 10 + val mapperId = 0 + + // don't write any data into empty partition. + val empty = new PartitionInfo(0, 0) + + val workers = Array( + new WorkerAddress(worker(0).rpcEnv.address.host, worker(0).pushPort), + new WorkerAddress(worker(1).rpcEnv.address.host, worker(1).pushPort)) + + // set some write exception when write data into replica. + val failed = new PartitionInfo(0, 1) + val failedFileLength = new AtomicLong(0L) + + val allfutureList = new util.ArrayList[CompletableFuture[_]]() + val futuresList = new util.ArrayList[Array[CompletableFuture[Int]]]() + (0 until 10).foreach(index => { + // send pushData to all replica piece + val length: Int = 1 + rand.nextInt(16 * 1024) + failedFileLength.addAndGet(length) + val tmp: Array[Byte] = new Array[Byte](length) + val (allFuture, futures) = batchPushData(appId, shuffleId, + Array(failed.getReducerId), failed.getEpochId, mapperId, + tmp, Array(0, tmp.length), workers, shuffleMode.toString) + allfutureList.add(allFuture) + futuresList.add(futures) + }) + allfutureList.asScala.foreach(_.get()) + assert(!futuresList.asScala.flatten.exists(_.get() != 0)) + allfutureList.clear() + futuresList.clear() + + // try set Exception for failed partitionInfo with replica index 1. + sendWorkerRpc[BreakPartition, BreakPartitionResponse](worker(1).rpcPort, + BreakPartition(Utils.getShuffleKey(appId, shuffleId), failed.getReducerId, failed.getEpochId)) + (0 until 10).foreach(index => { + val (allFuture, futures) = batchPushData(appId, shuffleId, + Array(failed.getReducerId), failed.getEpochId, mapperId, + new Array[Byte](1024), Array(0, 1024), workers, shuffleMode.toString) + allfutureList.add(allFuture) + futuresList.add(futures) + }) + allfutureList.asScala.foreach(_.get()) + futuresList.asScala.foreach { futures => + assert(futures(0).get() == 0) + assert(futures(1).get() != 0) + } + allfutureList.clear() + futuresList.clear() + + // write data into replica with no exception. + val giant = new PartitionInfo(0, 2) + val giantFileLength = new AtomicLong(0L) + + (0 until 10).foreach(index => { + // send pushData to all replica piece + val length: Int = 8 * 1024 * 1024 + giantFileLength.addAndGet(length) + val tmp: Array[Byte] = new Array[Byte](length) + val (allFuture, futures) = batchPushData(appId, shuffleId, + Array(giant.getReducerId), giant.getEpochId, mapperId, + tmp, Array(0, tmp.length), workers, shuffleMode.toString) + allfutureList.add(allFuture) + futuresList.add(futures) + }) + allfutureList.asScala.foreach(_.get()) + assert(!futuresList.asScala.flatten.exists(_.get() != 0)) + + val committed1 = sendWorkerRpc[CommitFiles, CommitFilesResponse](worker(0).rpcPort, + CommitFiles(Utils.getShuffleKey(appId, shuffleId))) + val committed2 = sendWorkerRpc[CommitFiles, CommitFilesResponse](worker(1).rpcPort, + CommitFiles(Utils.getShuffleKey(appId, shuffleId))) + + Seq(committed1, committed2).foreach(f => { + f.committed.asScala.foreach(committed => { + if (committed.getReducerId == empty.getReducerId && committed.getEpochId == empty.getEpochId) { + assert(committed.getFileLength == 0) + } else if (committed.getReducerId == failed.getReducerId && committed.getEpochId == failed.getEpochId) { + // replica index 0 done, replica index 1 failed. + if (committed.getPort == worker(1).fetchPort) { + assert(committed.getFileLength == -1) + } else { + // because replica index 0 done still fine, + // the batchPushData with 10 * 1024 after break partition should be able to see in replica index 0 + assert(committed.getFileLength == failedFileLength.get() + 10 * 1024) + } + } else if (committed.getReducerId == giant.getReducerId && committed.getEpochId == giant.getEpochId) { + assert(committed.getFileLength == giantFileLength.get()) + } else { + throw new Exception("should not reach here.") + } + }) + }) + } + } +} diff --git a/service/src/test/scala/com/bytedance/css/service/deploy/master/CleanupSuite.scala b/service/src/test/scala/com/bytedance/css/service/deploy/master/CleanupSuite.scala new file mode 100644 index 0000000..af8a260 --- /dev/null +++ b/service/src/test/scala/com/bytedance/css/service/deploy/master/CleanupSuite.scala @@ -0,0 +1,98 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import java.io.File + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.protocol.CssRpcMessage.{HeartbeatFromApp, RegisterPartitionGroup, RegisterPartitionGroupResponse, UnregisterShuffle, UnregisterShuffleResponse} +import com.bytedance.css.service.deploy.worker.Storage + + +class CleanupSuite extends LocalClusterSuite { + + val cssConf = new CssConf().set("css.maxPartitionsPerGroup", "1") + + override def clusterConf: Map[String, String] = { + Map("css.remove.shuffle.delay" -> "3s", + "css.app.timeout" -> "3s", + "css.worker.timeout" -> "4s") + } + + test("test application timeout") { + + val appId = s"appId-${System.currentTimeMillis()}" + val shuffleId = 10 + heartbeatRef.send(HeartbeatFromApp(appId)) + + val res = masterRef.askSync[RegisterPartitionGroupResponse]( + RegisterPartitionGroup(appId, shuffleId, 10, 20, CssConf.maxPartitionsPerGroup(cssConf)) + ) + assert(res.partitionGroups.size() == 20) + + // sleep enough time to trigger app lost & unregister shuffle. + Thread.sleep(15000) + assert(!Master.master.getShuffleStageManager().validateRegisterShuffle(appId, shuffleId)) + } + + test("test unregister shuffle after delay ms") { + + val appId = s"appId-${System.currentTimeMillis()}" + val shuffleId = 10 + val res = masterRef.askSync[RegisterPartitionGroupResponse]( + RegisterPartitionGroup(appId, shuffleId, 10, 20, CssConf.maxPartitionsPerGroup(cssConf)) + ) + assert(res.partitionGroups.size() == 20) + + masterRef.askSync[UnregisterShuffleResponse](UnregisterShuffle(appId, shuffleId)) + assert (Master.master.getShuffleStageManager().validateRegisterShuffle(appId, shuffleId)) + + // sleep enough time to trigger app lost & unregister shuffle. + Thread.sleep(6000) + assert (!Master.master.getShuffleStageManager().validateRegisterShuffle(appId, shuffleId)) + } + + test("test cleanup application dir") { + val dirs = CssConf.diskFlusherBaseDirs(cssConf).map(new File(_, Storage.workingDirName)) + val workingDir = dirs(0) + workingDir.mkdirs() + + val size = 5 + val appIds = new Array[String](size) + appIds.indices.foreach(index => { + val appId = s"appId-${System.currentTimeMillis()}" + Thread.sleep(100) + appIds(index) = appId + + new File(workingDir, s"$appId").mkdirs() + }) + + appIds.indices.foreach(index => { + heartbeatRef.send(HeartbeatFromApp(appIds(index))) + }) + + // must > 2 * css.app.timeout + css.remove.shuffle.delay + css.worker.timeout / 4 + Thread.sleep(10000) + + appIds.indices.foreach(index => { + assert(! new File(workingDir, appIds(index)).exists()) + }) + } +} diff --git a/service/src/test/scala/com/bytedance/css/service/deploy/master/DeDuplicateSuite.scala b/service/src/test/scala/com/bytedance/css/service/deploy/master/DeDuplicateSuite.scala new file mode 100644 index 0000000..aac8b5c --- /dev/null +++ b/service/src/test/scala/com/bytedance/css/service/deploy/master/DeDuplicateSuite.scala @@ -0,0 +1,190 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import scala.collection.JavaConverters._ + +import com.bytedance.css.client.ShuffleClient +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.protocol.CssRpcMessage._ +import com.bytedance.css.common.protocol.CssStatusCode +import com.bytedance.css.common.util.Utils +import org.apache.commons.lang3.RandomStringUtils + +class DeDuplicateSuite extends LocalClusterSuite { + + Seq(true, false).foreach { mode => + test(s"failedBatchBlacklistEnable $mode") { + val cssConf: CssConf = new CssConf() + cssConf.set("css.master.address", masterRef.address.toCssURL) + cssConf.set("css.client.failed.batch.blacklist.enabled", mode.toString) + cssConf.set("css.local.chunk.fetch.enabled", "false") + cssConf.set("css.test.mode", "true") + + val appId = s"appId-${System.currentTimeMillis()}" + val shuffleId = 5 + val numMappers = 1 + val numPartitions = 1 + val maxPartitionsPerGroup = 1 + + val mapperId = 0 + val mapperAttemptId = 0 + val reducerId = 0 + + val shuffleClient = ShuffleClient.get(cssConf) + ShuffleClient.cleanShuffle(shuffleId) + + val res1 = masterRef.askSync[RegisterPartitionGroupResponse]( + RegisterPartitionGroup(appId, shuffleId, numMappers, numPartitions, maxPartitionsPerGroup) + ) + assert(res1.statusCode == CssStatusCode.Success) + assert(res1.partitionGroups.size() == 1) + + shuffleClient.applyShufflePartitionGroup(shuffleId, res1.partitionGroups) + val partitionGroup = res1.partitionGroups.asScala.head + val workers = partitionGroup.getReplicaWorkers.asScala.toArray + + val resultSet = new java.util.HashMap[String, Int]() + (0 until 100).foreach(t => { + // send pushData to replica piece. + val content = RandomStringUtils.randomAlphanumeric(10, 20) + if (resultSet.containsKey(content)) { + resultSet.put(content, resultSet.get(content) + 1) + } else { + resultSet.put(content, 1) + } + val bytes = (content + ",").getBytes + shuffleClient.batchPushData(appId, shuffleId, mapperId, mapperAttemptId, + Array(reducerId), bytes, Array(0), Array(bytes.length), numMappers, numPartitions, false) + }) + // wait for all batch push data write to partition file. + Thread.sleep(2000) + + val replica1PartitionPort = actualWorkers.filter(_._2 == workers(0).port).head._1 + + sendWorkerRpc[BreakPartition, BreakPartitionResponse](replica1PartitionPort, + BreakPartition(Utils.getShuffleKey(appId, shuffleId), reducerId, partitionGroup.epochId)) + Thread.sleep(2000) + + (0 until 100).foreach(t => { + val content = RandomStringUtils.randomAlphanumeric(10, 20) + if (resultSet.containsKey(content)) { + resultSet.put(content, resultSet.get(content) + 1) + } else { + resultSet.put(content, 1) + } + val bytes = (content + ",").getBytes + shuffleClient.batchPushData(appId, shuffleId, mapperId, mapperAttemptId, + Array(reducerId), bytes, Array(0), Array(bytes.length), numMappers, numPartitions, false) + }) + + // trigger mapped end & stage end. + shuffleClient.mapperEnd(appId, shuffleId, mapperId, mapperAttemptId, numMappers) + Thread.sleep(5000) + + val reducerFileGroupsRes = masterRef.askSync[GetReducerFileGroupsResponse]( + GetReducerFileGroups(appId, shuffleId)) + + if (mode) { + assert(reducerFileGroupsRes.batchBlacklist != null) + assert(reducerFileGroupsRes.batchBlacklist.size() >= 1) + assert(reducerFileGroupsRes.batchBlacklist.asScala.forall(_.getFailedPartitionBatchStr.startsWith("0-0-0-0-"))) + } else { + assert(reducerFileGroupsRes.batchBlacklist == null) + } + + // read two epochs separately + val inputStream1 = shuffleClient.readPartitions(appId, shuffleId, Array(reducerId), 2, 0) + val inputStream2 = shuffleClient.readPartitions(appId, shuffleId, Array(reducerId), 2, 1) + + val resultBytes = new Array[Byte](200 * 21 + 2000) + var index = 0 + val buffer = new Array[Byte](1024) + var readBytes = 0 + do { + readBytes = inputStream1.read(buffer, 0, 1024) + if (readBytes > 0) { + System.arraycopy(buffer, 0, resultBytes, index, readBytes) + index += readBytes + } + } while (readBytes != -1) + + readBytes = 0 + do { + readBytes = inputStream2.read(buffer, 0, 1024) + if (readBytes > 0) { + System.arraycopy(buffer, 0, resultBytes, index, readBytes) + index += readBytes + } + } while (readBytes != -1) + + val resultStr = new String(resultBytes, 0, index) + val actualSet = new java.util.HashMap[String, Int]() + resultStr.split(",").filter(_.nonEmpty) + .foreach(content => { + if (actualSet.containsKey(content)) { + actualSet.put(content, actualSet.get(content) + 1) + } else { + actualSet.put(content, 1) + } + }) + + inputStream1.close() + inputStream2.close() + + assert(!resultSet.values().asScala.exists(_ > 1)) + assert(resultSet.size() == actualSet.size()) + if (mode) { + assert(actualSet.values().asScala.count(_ > 1) == 0) + } else { + assert(actualSet.values().asScala.count(_ > 1) >= 1) + } + + // the two epochs are read as a whole, there should be no duplication. + val asSinglePartitionBytes = new Array[Byte](200 * 21 + 2000) + val inputStream = shuffleClient.readPartitions(appId, shuffleId, Array(0), 0, 1) + readBytes = 0 + index = 0 + do { + readBytes = inputStream.read(buffer, 0, 1024) + if (readBytes > 0) { + System.arraycopy(buffer, 0, asSinglePartitionBytes, index, readBytes) + index += readBytes + } + } while (readBytes != -1) + val singlePartitionResultStr = new String(asSinglePartitionBytes, 0, index) + val singlePartitionSet = new java.util.HashMap[String, Int]() + singlePartitionResultStr.split(",").filter(_.nonEmpty) + .foreach(content => { + if (singlePartitionSet.containsKey(content)) { + singlePartitionSet.put(content, singlePartitionSet.get(content) + 1) + } else { + singlePartitionSet.put(content, 1) + } + }) + + inputStream.close() + assert(resultSet.size() == singlePartitionSet.size()) + assert(singlePartitionSet.values().asScala.count(_ > 1) == 0) + + shuffleClient.shutDown() + } + } +} diff --git a/service/src/test/scala/com/bytedance/css/service/deploy/master/EpochRotateSuite.scala b/service/src/test/scala/com/bytedance/css/service/deploy/master/EpochRotateSuite.scala new file mode 100644 index 0000000..4b60b5f --- /dev/null +++ b/service/src/test/scala/com/bytedance/css/service/deploy/master/EpochRotateSuite.scala @@ -0,0 +1,136 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import java.util + +import scala.collection.JavaConverters._ +import scala.util.Random + +import com.bytedance.css.client.ShuffleClient +import com.bytedance.css.client.impl.ShuffleClientImpl +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.protocol.CssRpcMessage.{GetReducerFileGroups, GetReducerFileGroupsResponse, RegisterPartitionGroup, RegisterPartitionGroupResponse} +import com.bytedance.css.common.protocol.CssStatusCode +import org.apache.commons.lang3.RandomStringUtils + +class EpochRotateSuite extends LocalClusterSuite { + + // set cssConf as common, because ShuffleClient is singleton + val cssConf = new CssConf() + val random = new Random + + override def clusterConf: Map[String, String] = { + Map("css.epoch.rotate.threshold" -> "4m") + } + + Seq("DISK", "HDFS").foreach { mode => + test(s"epoch rotate assert $mode") { + + cssConf.set("css.master.address", masterRef.address.toCssURL) + cssConf.set("css.local.chunk.fetch.enabled", "false") + cssConf.set("css.epoch.rotate.threshold", "4m") + cssConf.set("css.shuffle.mode", mode) + + val appId = s"appId-${System.currentTimeMillis()}" + val shuffleId = 5 + val numMappers = 5 + val numPartitions = 1 + val maxPartitionsPerGroup = 1 + + val mapperAttemptId = 0 + val reducerId = 0 + + val shuffleClient = new ShuffleClientImpl(cssConf) + ShuffleClient.cleanShuffle(shuffleId) + + val res1 = masterRef.askSync[RegisterPartitionGroupResponse]( + RegisterPartitionGroup(appId, shuffleId, numMappers, numPartitions, maxPartitionsPerGroup) + ) + assert(res1.statusCode == CssStatusCode.Success) + assert(res1.partitionGroups.size() == 1) + shuffleClient.applyShufflePartitionGroup(shuffleId, res1.partitionGroups) + + val resultSet = new util.HashMap[Int, util.HashSet[String]]() + 0.until(5).foreach { mapperId => + resultSet.put(mapperId, new util.HashSet[String]()) + } + val actualLength = new util.HashMap[Int, Long]() + 0.until(5).foreach { mapperId => + actualLength.put(mapperId, 0L) + } + + // just test retry to push data + (0 until 4000).foreach { _ => + val mapperId = random.nextInt(4) + val content = RandomStringUtils.randomAlphanumeric(1024, 2048); + resultSet.get(mapperId).add(content) + actualLength.put(mapperId, actualLength.get(mapperId) + content.length) + val bytes = content.getBytes + shuffleClient.batchPushData(appId, shuffleId, mapperId, mapperAttemptId, + Array(reducerId), bytes, Array(0), Array(bytes.length), numMappers, numPartitions, false) + } + Thread.sleep(5000) + + (0 until 5).foreach { mapperId => + shuffleClient.mapperEnd(appId, shuffleId, mapperId, mapperAttemptId, numMappers) + } + + Thread.sleep(2000) + val reducerFileGroupsRes = masterRef.askSync[GetReducerFileGroupsResponse]( + GetReducerFileGroups(appId, shuffleId)) + + var maxEpoch = -1 + reducerFileGroupsRes.fileGroup.foreach(reducerFileGroupsRes => { + reducerFileGroupsRes.foreach(p => { + if (p.getEpochId > maxEpoch) { + maxEpoch = p.getEpochId + } + }) + }) + + assert(reducerFileGroupsRes.status == CssStatusCode.Success) + assert(maxEpoch > 0) + + 0.until(5).foreach { mapperId => + val inputStream = shuffleClient.readPartitions(appId, shuffleId, Array(0), mapperId, mapperId + 1) + val resultBytes = new Array[Byte](actualLength.get(mapperId).toInt) + var index = 0 + val buffer = new Array[Byte](1024) + var readBytes = 0 + do { + readBytes = inputStream.read(buffer, 0, 1024) + if (readBytes > 0) { + System.arraycopy(buffer, 0, resultBytes, index, readBytes) + index += readBytes + } + } while (readBytes != -1) + + assert(actualLength.get(mapperId) == index) + + val resultString = new String(resultBytes) + resultSet.get(mapperId).asScala.foreach(f => assert(resultString.indexOf(f) != -1)) + inputStream.close() + } + + shuffleClient.shutDown() + } + } +} diff --git a/service/src/test/scala/com/bytedance/css/service/deploy/master/FetchDataSuite.scala b/service/src/test/scala/com/bytedance/css/service/deploy/master/FetchDataSuite.scala new file mode 100644 index 0000000..57ba51e --- /dev/null +++ b/service/src/test/scala/com/bytedance/css/service/deploy/master/FetchDataSuite.scala @@ -0,0 +1,139 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import java.util +import java.util.concurrent.CompletableFuture + +import scala.collection.JavaConverters._ + +import com.bytedance.css.client.stream.CssInputStream +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.protocol.{CssStatusCode, PartitionInfo, ShuffleMode} +import com.bytedance.css.common.protocol.CssRpcMessage._ +import com.bytedance.css.common.util.Utils +import org.apache.commons.lang3.RandomStringUtils + +class FetchDataSuite extends LocalClusterSuite { + + Seq(ShuffleMode.DISK, ShuffleMode.HDFS).foreach { mode => + Seq("lz4", "zstd").foreach { compressType => + Seq("true", "false").foreach { localChunkEnable => + test(s"push to single partition and read " + + s"via InputStream $mode via compressType $compressType via localChunkEnable $localChunkEnable") { + + val cssConf = new CssConf() + cssConf.set("css.compression.codec", compressType) + cssConf.set("css.local.chunk.fetch.enabled", localChunkEnable) + + // registerShuffle + val appId = s"appId-${System.currentTimeMillis()}" + val shuffleId = 5 + val numMappers = 1 + val numPartitions = 1 + val maxPartitionsPerGroup = 1 + + val mapperId = 0 + val mapperAttemptId = 0 + val reducerId = 0 + + val res = masterRef.askSync[RegisterPartitionGroupResponse]( + RegisterPartitionGroup(appId, shuffleId, numMappers, numPartitions, maxPartitionsPerGroup) + ) + assert(res.statusCode == CssStatusCode.Success) + assert(res.partitionGroups.size() == 1) + val partitionGroup = res.partitionGroups.asScala.head + + val resultSet = new java.util.HashSet[String]() + + var actualLength = 0L + val allfutureList = new util.ArrayList[CompletableFuture[_]]() + val futuresList = new util.ArrayList[Array[CompletableFuture[Int]]]() + (0 until 1000).foreach(batchId => { + val content = RandomStringUtils.randomAlphanumeric(1024, 2048) + actualLength += content.length + resultSet.add(content) + val bytes = addHeaderAndCompressWithSinglePartition( + cssConf, mapperId, mapperAttemptId, reducerId, batchId, content.getBytes) + val (allFuture, futures) = + batchPushData(appId, shuffleId, Array(reducerId), partitionGroup.epochId, mapperId, + bytes, Array(0, bytes.length), partitionGroup.getReplicaWorkers.asScala.toArray, mode.toString) + allfutureList.add(allFuture) + futuresList.add(futures) + }) + allfutureList.asScala.foreach(_.get()) + assert(!futuresList.asScala.flatten.exists(_.get() != 0)) + + // send mapper end + val epochList = new util.ArrayList[PartitionInfo]() + epochList.add(new PartitionInfo(reducerId, partitionGroup.epochId)) + masterRef.askSync[MapperEndResponse]( + MapperEnd(appId, shuffleId, mapperId, mapperAttemptId, numMappers, epochList, null)) + + Thread.sleep(5000) + + val reducerFileGroupsRes = masterRef.askSync[GetReducerFileGroupsResponse]( + GetReducerFileGroups(appId, shuffleId) + ) + + assert(reducerFileGroupsRes.status == CssStatusCode.Success) + // filter two empty piece and a broken one + assert(reducerFileGroupsRes.fileGroup.flatten.length == 2) + // for partition 1 mapper attempt 4 comes first. + reducerFileGroupsRes.attempts.zip(Array(0)).foreach(f => assert(f._1 == f._2)) + + val inputStream = CssInputStream.create(cssConf, dataClientFactory, Utils.getShuffleKey(appId, shuffleId), + reducerFileGroupsRes.fileGroup.head, reducerFileGroupsRes.attempts, null, 0, 1 + ) + + var resultCount = 0L + while (inputStream.read() != -1) { + resultCount += 1 + } + assert(actualLength == resultCount) + inputStream.close() + + val inputStream2 = CssInputStream.create(cssConf, dataClientFactory, Utils.getShuffleKey(appId, shuffleId), + reducerFileGroupsRes.fileGroup.head, reducerFileGroupsRes.attempts, null, 0, 1 + ) + val resultBytes = new Array[Byte](actualLength.toInt) + var index = 0 + val buffer = new Array[Byte](1024) + var readBytes = 0 + do { + readBytes = inputStream2.read(buffer, 0, 1024) + if (readBytes > 0) { + System.arraycopy(buffer, 0, resultBytes, index, readBytes) + index += readBytes + } + } while (readBytes != -1) + + var resultString = new String(resultBytes) + resultSet.asScala.foreach(f => resultString.indexOf(f) != -1) + resultSet.asScala.foreach{ f => + resultString = resultString.replace(f, "") + } + assert(resultString == "") + inputStream2.close() + } + } + } + } +} diff --git a/service/src/test/scala/com/bytedance/css/service/deploy/master/LocalCluster.scala b/service/src/test/scala/com/bytedance/css/service/deploy/master/LocalCluster.scala new file mode 100644 index 0000000..5f1e7de --- /dev/null +++ b/service/src/test/scala/com/bytedance/css/service/deploy/master/LocalCluster.scala @@ -0,0 +1,288 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import java.nio.ByteBuffer +import java.util +import java.util.concurrent.{CompletableFuture, ConcurrentHashMap} +import java.util.concurrent.atomic.AtomicReference + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag + +import com.bytedance.css.client.compress.CssCompressorFactory +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.protocol.{RpcNameConstants, TransportModuleConstants, WorkerAddress} +import com.bytedance.css.common.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} +import com.bytedance.css.common.unsafe.Platform +import com.bytedance.css.common.util.Utils +import com.bytedance.css.network.TransportContext +import com.bytedance.css.network.buffer.NettyManagedBuffer +import com.bytedance.css.network.client.RpcResponseCallback +import com.bytedance.css.network.protocol.BatchPushDataRequest +import com.bytedance.css.network.server.NoOpRpcHandler +import com.bytedance.css.service.deploy.worker.Worker +import io.netty.buffer.Unpooled +import org.scalatest.{BeforeAndAfterAll, FunSuite} + + +trait LocalClusterSuite extends FunSuite with BeforeAndAfterAll with Logging { + + private val conf = new CssConf().set("css.local.chunk.fetch.enabled", "false") + private val rpcEnv = RpcEnv.create("AnyClient", Utils.localHostName(), 0, conf, true) + private val dataTransportConf = Utils.fromCssConf(conf, TransportModuleConstants.DATA_MODULE, 8) + private val context = new TransportContext(dataTransportConf, new NoOpRpcHandler, true) + protected val dataClientFactory = context.createClientFactory(Nil.asJava) + + override def beforeAll(): Unit = { + super.beforeAll() + startCluster() + } + + override def afterAll() { + stopCluster() + super.afterAll() + } + + // mock local cluster for master & worker. + val testConf: Map[String, String] = Map( + "css.hdfsFlusher.num" -> "1", + "css.hdfsFlusher.base.dir" -> "file:///tmp/hdfs_css", + "css.test.mode" -> "true") + def clusterConf: Map[String, String] = Map() + + var masterThread: Thread = null + def master(): Master = Master.master + final def masterRef: RpcEndpointRef = master().self + final def heartbeatRef: RpcEndpointRef = rpcEnv.setupEndpointRef( + masterRef.address, RpcNameConstants.HEARTBEAT) + + var workerThreads: ArrayBuffer[Thread] = new ArrayBuffer[Thread]() + def worker(index: Int): Worker = Worker.workers(index) + + // val portOffset = new Random(System.currentTimeMillis()).nextInt(1000) + + def workersInitPorts: Seq[(Int, Int, Int)] = { + Seq( + (0, 0, 0), + (0, 0, 0), + (0, 0, 0)) + } + + def actualWorkers: Seq[(Int, Int, Int)] = { + workersInitPorts.indices.map(index => { + (worker(index).rpcPort, worker(index).pushPort, worker(index).fetchPort) + }) + } + + def startCluster(): Unit = { + val exception = new AtomicReference[Exception] + def checkException(): Unit = { + val e = exception.get + if (e != null) throw e + } + + try { + masterThread = new Thread() { + override def run(): Unit = { + val confParam = (testConf ++ clusterConf).flatMap(f => Array("--conf", f._1, f._2)) + try { + Master.main(Array("--local-mode") ++ confParam) + } catch { + case e: Exception => + logError("start master failed: " + e.getCause, e) + exception.set(e) + } + } + } + masterThread.start() + while ((Master.master == null || Master.master.self == null) && exception.get() == null) { + Thread.sleep(3000) + } + checkException() + + workersInitPorts.foreach(f => { + val thread = new Thread() { + override def run(): Unit = { + val masterArgs = Map("css.master.address" -> masterRef.address.toCssURL) + val confParam = (testConf ++ clusterConf ++ masterArgs).flatMap(f => Array("--conf", f._1, f._2)) + try { + Worker.main(Array("--port", f._1.toString, "-pp", f._2.toString, "-fp", f._3.toString) + ++ confParam ++ Array(masterRef.address.toCssURL)) + } catch { + case e: Exception => + logError("start worker failed: " + e.getCause, e) + exception.set(e) + } + } + } + workerThreads.append(thread) + }) + workerThreads.foreach(_.start()) + Thread.sleep(5000) + + if (Worker.workers.size != workersInitPorts.size) { + throw new RuntimeException( + s"started worker size ${Worker.workers.size} not match target size ${workersInitPorts.size}") + } + checkException() + } catch { + case e: Exception => + logError("start cluster failed. ", e) + try { + stopCluster() + } finally { + throw e + } + } + } + + def stopCluster(): Unit = { + val exceptionMap = new ConcurrentHashMap[String, Exception]() + Worker.workers.foreach { worker => + try { + worker.stop() + worker.rpcEnv.shutdown() + worker.rpcEnv.awaitTermination() + } catch { + case e: Exception => + logError("stop cluster worker failed. " + worker, e) + exceptionMap.put("worker failed: \n" + e.getMessage, e) + } + } + Worker.workers.clear() + workerThreads.foreach(_.stop()) + + try { + Master.master.stop() + Master.master.rpcEnv.shutdown() + Master.master.rpcEnv.awaitTermination() + } catch { + case e: Exception => + logError("stop cluster master failed. ", e) + exceptionMap.put("master failed: \n" + e.getMessage, e) + } finally { + Master.master = null + masterThread.stop() + } + + // stop dummy rpc env for test + rpcEnv.shutdown() + rpcEnv.awaitTermination() + + dataClientFactory.close() + context.close() + + exceptionMap.asScala.foreach(f => { + throw f._2 + }) + Thread.sleep(5000) + } + + def sendWorkerRpc[T: ClassTag, K: ClassTag](port: Int, req: T): K = { + val workerRef = + rpcEnv.setupEndpointRef(new RpcAddress(Utils.localHostName(), port), RpcNameConstants.WORKER_EP) + workerRef.askSync[K](req) + } + + def addHeaderAndCompressWithSinglePartition( + cssConf: CssConf, + mapperId: Int, + mapperAttemptId: Int, + reducerId: Int, + batchId: Int, + originalBytes: Array[Byte]): Array[Byte] = { + val (data, offsets) = addHeaderAndCompressWithMultiPartition( + cssConf, mapperId, mapperAttemptId, Array(reducerId), Array(batchId), + originalBytes, Array(0), Array(originalBytes.length)) + assert(data.length == offsets.last) + data + } + + def addHeaderAndCompressWithMultiPartition( + cssConf: CssConf, + mapperId: Int, + mapperAttemptId: Int, + reducerIdArray: Array[Int], + partitionBatchIds: Array[Int], + data: Array[Byte], + offsetArray: Array[Int], + lengthArray: Array[Int]): (Array[Byte], List[Int]) = { + val factory = new CssCompressorFactory(cssConf) + val compressor = factory.getCompressor + val compositeByteBuf = Unpooled.compositeBuffer(reducerIdArray.length) + val compressedOffsetList = new util.ArrayList[Int]() + compressedOffsetList.add(0) + for (i <- reducerIdArray.indices) { + compressor.compress(data, offsetArray(i), lengthArray(i)) + val compressedTotalSize = compressor.getCompressedTotalSize + val BATCH_HEADER_SIZE = 4 * 4 + val body = new Array[Byte](BATCH_HEADER_SIZE + compressedTotalSize) + Platform.putInt(body, Platform.BYTE_ARRAY_OFFSET, mapperId) + Platform.putInt(body, Platform.BYTE_ARRAY_OFFSET + 4, mapperAttemptId) + Platform.putInt(body, Platform.BYTE_ARRAY_OFFSET + 8, partitionBatchIds(i)) + Platform.putInt(body, Platform.BYTE_ARRAY_OFFSET + 12, compressedTotalSize) + System.arraycopy(compressor.getCompressedBuffer, 0, body, BATCH_HEADER_SIZE, compressedTotalSize) + compositeByteBuf.addComponent(true, Unpooled.wrappedBuffer(body)) + compressedOffsetList.add(compressedOffsetList.asScala.last + body.length) + } + val sendBytes = new Array[Byte](compressedOffsetList.asScala.last) + compositeByteBuf.readBytes(sendBytes) + assert(sendBytes.length == compressedOffsetList.asScala.last) + (sendBytes, compressedOffsetList.asScala.toList) + } + + def batchPushData( + appId: String, + shuffleId: Int, + reducerIds: Array[Int], + epochId: Int, + mapperId: Int, + data: Array[Byte], + offsets: Array[Int], + workers: Array[WorkerAddress], + shuffleMode: String, + epochRotateThreshold: Long = CssConf.epochRotateThreshold(conf)): + (CompletableFuture[_], Array[CompletableFuture[Int]]) = { + val futures = new Array[CompletableFuture[Int]](workers.length) + val shuffleKey = Utils.getShuffleKey(appId, shuffleId) + workers.indices.foreach { index => + val future = new CompletableFuture[Int] + val buffer = new NettyManagedBuffer(Unpooled.wrappedBuffer(data)) + val client = dataClientFactory.createClient(workers(index).host, workers(index).port) + val newPushData = new BatchPushDataRequest(shuffleKey, reducerIds, epochId, offsets, mapperId, + index, shuffleMode, epochRotateThreshold.toString, System.currentTimeMillis(), buffer) + client.batchPushData(newPushData, new RpcResponseCallback() { + override def onSuccess(response: ByteBuffer): Unit = { + future.complete(0) + } + + override def onFailure(e: Throwable): Unit = { + logError("batchPushData failure: ", e) + future.complete(1) + } + }) + futures(index) = future + } + (CompletableFuture.allOf(futures: _*), futures) + } +} diff --git a/service/src/test/scala/com/bytedance/css/service/deploy/master/RegisterShuffleSuite.scala b/service/src/test/scala/com/bytedance/css/service/deploy/master/RegisterShuffleSuite.scala new file mode 100644 index 0000000..abacd97 --- /dev/null +++ b/service/src/test/scala/com/bytedance/css/service/deploy/master/RegisterShuffleSuite.scala @@ -0,0 +1,292 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import java.util + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import com.bytedance.css.common.protocol.{CssStatusCode, PartitionGroup, PartitionInfo} +import com.bytedance.css.common.protocol.CssRpcMessage.{MapperEnd, MapperEndResponse, ReallocatePartitionGroup, ReallocatePartitionGroupResponse, RegisterPartitionGroup, RegisterPartitionGroupResponse} + + +class RegisterShuffleSuite extends LocalClusterSuite { + + test("register shuffle with same group") { + val appId = s"appId-${System.currentTimeMillis()}" + val shuffleId = 5 + val numMappers = 10 + + val numPartitions = 1002 + val maxPartitionsPerGroup = 20 + val res1 = masterRef.askSync[RegisterPartitionGroupResponse]( + RegisterPartitionGroup(appId, shuffleId, numMappers, numPartitions, maxPartitionsPerGroup) + ) + val res2 = masterRef.askSync[RegisterPartitionGroupResponse]( + RegisterPartitionGroup(appId, shuffleId, numMappers, numPartitions, maxPartitionsPerGroup) + ) + assert(res1.statusCode == CssStatusCode.Success) + assert(res2.statusCode == CssStatusCode.Success) + assert(res1.partitionGroups.size() == res2.partitionGroups.size()) + res1.partitionGroups.asScala.indices.foreach { index => + assert(res1.partitionGroups.asScala(index) == res2.partitionGroups.asScala(index)) + } + } + + test("register shuffle with diff group") { + // register shuffle same like to assign strategy. + // because there replica = 2 by default. so target possible tuple size = 3. + + val appId = s"appId-${System.currentTimeMillis()}" + val shuffleId = 10 + val numMappers = 10 + + // here use maxPartitionsPerGroup as group len. + val numPartitions1 = 1002 + val maxPartitionsPerGroup1 = 20 + val res1 = masterRef.askSync[RegisterPartitionGroupResponse]( + RegisterPartitionGroup(appId, shuffleId, numMappers, numPartitions1, maxPartitionsPerGroup1) + ) + assert(res1.statusCode == CssStatusCode.Success) + assert(res1.partitionGroups.size() == 51) + res1.partitionGroups.asScala.foreach(p => assert(p.epochId == 0)) + assert(res1.partitionGroups.asScala(0).startPartition == 0) + assert(res1.partitionGroups.asScala(0).endPartition == 20) + assert(res1.partitionGroups.asScala(50).startPartition == 1000) + assert(res1.partitionGroups.asScala(50).endPartition == 1002) + + // here use numPartitions / possible tuple size as group len. + val numPartitions2 = 1002 + val maxPartitionsPerGroup2 = 1000 + val res2 = masterRef.askSync[RegisterPartitionGroupResponse]( + // register another shuffle's partition group + RegisterPartitionGroup(appId, shuffleId + 1, numMappers, numPartitions2, maxPartitionsPerGroup2) + ) + assert(res2.statusCode == CssStatusCode.Success) + assert(res2.partitionGroups.size() == 3) + res2.partitionGroups.asScala.foreach(p => assert(p.epochId == 0)) + assert(res2.partitionGroups.asScala(0).startPartition == 0) + assert(res2.partitionGroups.asScala(0).endPartition == 334) + assert(res2.partitionGroups.asScala(1).startPartition == 334) + assert(res2.partitionGroups.asScala(1).endPartition == 668) + assert(res2.partitionGroups.asScala(2).startPartition == 668) + assert(res2.partitionGroups.asScala(2).endPartition == 1002) + + // here use one partition one group as group len. + val numPartitions3 = 2 + val maxPartitionsPerGroup3 = 100 + val res3 = masterRef.askSync[RegisterPartitionGroupResponse]( + // register another shuffle's partition group + RegisterPartitionGroup(appId, shuffleId + 2, numMappers, numPartitions3, maxPartitionsPerGroup3) + ) + assert(res3.statusCode == CssStatusCode.Success) + assert(res3.partitionGroups.size() == 2) + res3.partitionGroups.asScala.foreach(p => assert(p.epochId == 0)) + assert(res3.partitionGroups.asScala(0).startPartition == 0) + assert(res3.partitionGroups.asScala(0).endPartition == 1) + assert(res3.partitionGroups.asScala(1).startPartition == 1) + assert(res3.partitionGroups.asScala(1).endPartition == 2) + } + + test("multi thread register shuffle like all mappers") { + + val appId = s"appId-${System.currentTimeMillis()}" + val shuffleId = 15 + val numMappers = 10 + val numPartitions = 20 + val maxPartitionsPerGroup = 1 + + val waitList: ArrayBuffer[RegisterPartitionGroupResponse] = new ArrayBuffer[RegisterPartitionGroupResponse]() + val resultList: ArrayBuffer[RegisterPartitionGroupResponse] = new ArrayBuffer[RegisterPartitionGroupResponse]() + + // simulating 1000 mappers call register shuffle at the same time + val threads = (0 until 1000).map(_ => { + new Thread() { + override def run(): Unit = { + val res = masterRef.askSync[RegisterPartitionGroupResponse]( + RegisterPartitionGroup(appId, shuffleId, numMappers, numPartitions, maxPartitionsPerGroup) + ) + + if (res.statusCode == CssStatusCode.Success) { + resultList.synchronized { + resultList.append(res) + } + } else if (res.statusCode == CssStatusCode.Waiting) { + waitList.synchronized { + waitList.append(res) + } + } else { + assert(false) + } + } + } + }) + + threads.foreach(_.start()) + Thread.sleep(3000) + + resultList.foreach(res => { + assert(res.statusCode == CssStatusCode.Success) + assert(res.partitionGroups.size() == 20) + res.partitionGroups.asScala.foreach(p => { + assert(p.epochId == 0) + // skip Host check since all service in localhost + p.getReplicaWorkers.asScala.indices.foreach { index => + val wPs = p.getReplicaWorkers.asScala.map(worker => worker.port).toSet + assert(wPs.size == p.getReplicaWorkers.size()) + } + }) + }) + } + + test("reallocate partition group") { + + val appId = s"appId-${System.currentTimeMillis()}" + val shuffleId = 20 + val numMappers = 10 + val numPartitions = 20 + val maxPartitionsPerGroup = 1 + + val res = masterRef.askSync[RegisterPartitionGroupResponse]( + RegisterPartitionGroup(appId, shuffleId, numMappers, numPartitions, maxPartitionsPerGroup) + ) + + assert(res.statusCode == CssStatusCode.Success) + assert(res.partitionGroups.size() == 20) + res.partitionGroups.asScala.foreach(p => assert(p.epochId == 0)) + + val oldPartitionGroup = res.partitionGroups.asScala.head + var reallocateRes = masterRef.askSync[ReallocatePartitionGroupResponse]( + ReallocatePartitionGroup(appId, shuffleId, 0, 0, oldPartitionGroup) + ) + + assert(reallocateRes.statusCode == CssStatusCode.Success) + assert(reallocateRes.partitionGroup.partitionGroupId == oldPartitionGroup.partitionGroupId) + assert(reallocateRes.partitionGroup.epochId == oldPartitionGroup.epochId + 1) + assert(reallocateRes.partitionGroup.startPartition == oldPartitionGroup.startPartition) + assert(reallocateRes.partitionGroup.endPartition == oldPartitionGroup.endPartition) + + // get from the newly one, no need to reallocate + reallocateRes = masterRef.askSync[ReallocatePartitionGroupResponse]( + ReallocatePartitionGroup(appId, shuffleId, 0, 0, oldPartitionGroup) + ) + assert(reallocateRes.statusCode == CssStatusCode.Success) + assert(reallocateRes.partitionGroup.partitionGroupId == oldPartitionGroup.partitionGroupId) + assert(reallocateRes.partitionGroup.epochId == oldPartitionGroup.epochId + 1) + assert(reallocateRes.partitionGroup.startPartition == oldPartitionGroup.startPartition) + assert(reallocateRes.partitionGroup.endPartition == oldPartitionGroup.endPartition) + + // reallocate a bigger one since last reallocate. + val oldPartitionGroup2 = reallocateRes.partitionGroup + reallocateRes = masterRef.askSync[ReallocatePartitionGroupResponse]( + ReallocatePartitionGroup(appId, shuffleId, 0, 0, oldPartitionGroup2) + ) + assert(reallocateRes.statusCode == CssStatusCode.Success) + assert(reallocateRes.partitionGroup.partitionGroupId == oldPartitionGroup.partitionGroupId) + assert(reallocateRes.partitionGroup.epochId == oldPartitionGroup.epochId + 2) + assert(reallocateRes.partitionGroup.startPartition == oldPartitionGroup.startPartition) + assert(reallocateRes.partitionGroup.endPartition == oldPartitionGroup.endPartition) + } + + test("reallocate partition group with unexpected status") { + val appId = s"appId-${System.currentTimeMillis()}" + val shuffleId = 25 + val numMappers = 10 + val numPartitions = 20 + val maxPartitionsPerGroup = 1 + + // since do not register shuffle. + var oldPartitionGroup = new PartitionGroup(0, 0, 0, 0, null) + var reallocateRes = masterRef.askSync[ReallocatePartitionGroupResponse]( + ReallocatePartitionGroup(appId, shuffleId, 0, 0, oldPartitionGroup) + ) + assert(reallocateRes.statusCode == CssStatusCode.ShuffleNotRegistered) + + // start to register shuffle & mark current map task ended. + val registerRes = masterRef.askSync[RegisterPartitionGroupResponse]( + RegisterPartitionGroup(appId, shuffleId, numMappers, numPartitions, maxPartitionsPerGroup) + ) + assert(registerRes.statusCode == CssStatusCode.Success) + oldPartitionGroup = registerRes.partitionGroups.asScala.head + + val mapperRes = masterRef.askSync[MapperEndResponse]( + MapperEnd(appId, shuffleId, 0, 0, numMappers, new util.ArrayList[PartitionInfo], null) + ) + assert(mapperRes.statusCode == CssStatusCode.Success) + + reallocateRes = masterRef.askSync[ReallocatePartitionGroupResponse]( + ReallocatePartitionGroup(appId, shuffleId, 0, 0, oldPartitionGroup) + ) + assert(reallocateRes.statusCode == CssStatusCode.MapEnded) + } + + test("multi thread reallocate partition group like all mappers") { + val appId = s"appId-${System.currentTimeMillis()}" + val shuffleId = 30 + val numMappers = 10 + val numPartitions = 20 + val maxPartitionsPerGroup = 1 + + // start to register shuffle + val registerRes = masterRef.askSync[RegisterPartitionGroupResponse]( + RegisterPartitionGroup(appId, shuffleId, numMappers, numPartitions, maxPartitionsPerGroup) + ) + assert(registerRes.statusCode == CssStatusCode.Success) + val oldPartitionGroup = registerRes.partitionGroups.asScala.head + + val waitList: ArrayBuffer[ReallocatePartitionGroupResponse] = new ArrayBuffer[ReallocatePartitionGroupResponse]() + val resultList: ArrayBuffer[ReallocatePartitionGroupResponse] = new ArrayBuffer[ReallocatePartitionGroupResponse]() + + // simulating 1000 mappers call reallocate partition group at the same time + val threads = (0 until 1000).map(_ => { + new Thread() { + override def run(): Unit = { + val res = masterRef.askSync[ReallocatePartitionGroupResponse]( + ReallocatePartitionGroup(appId, shuffleId, 0, 0, oldPartitionGroup) + ) + + if (res.statusCode == CssStatusCode.Success) { + resultList.synchronized { + resultList.append(res) + } + } else if (res.statusCode == CssStatusCode.Waiting) { + waitList.synchronized { + waitList.append(res) + } + } else { + assert(false) + } + } + } + }) + + threads.foreach(_.start()) + Thread.sleep(3000) + + resultList.foreach(res => { + assert(res.statusCode == CssStatusCode.Success) + assert(res.partitionGroup.partitionGroupId == oldPartitionGroup.partitionGroupId) + assert(res.partitionGroup.epochId == oldPartitionGroup.epochId + 1) + assert(res.partitionGroup.startPartition == oldPartitionGroup.startPartition) + assert(res.partitionGroup.endPartition == oldPartitionGroup.endPartition) + }) + } +} diff --git a/service/src/test/scala/com/bytedance/css/service/deploy/master/ShuffleClientSuite.scala b/service/src/test/scala/com/bytedance/css/service/deploy/master/ShuffleClientSuite.scala new file mode 100644 index 0000000..a458cf6 --- /dev/null +++ b/service/src/test/scala/com/bytedance/css/service/deploy/master/ShuffleClientSuite.scala @@ -0,0 +1,328 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import scala.collection.JavaConverters._ + +import com.bytedance.css.client.ShuffleClient +import com.bytedance.css.client.impl.ShuffleClientImpl +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.protocol.CssRpcMessage.{BreakPartition, BreakPartitionResponse, GetReducerFileGroups, GetReducerFileGroupsResponse, RegisterPartitionGroup, RegisterPartitionGroupResponse} +import com.bytedance.css.common.protocol.CssStatusCode +import com.bytedance.css.common.util.Utils +import org.apache.commons.lang3.RandomStringUtils + +class ShuffleClientSuite extends LocalClusterSuite { + + // here only use 2 workers to test. + override def workersInitPorts: Seq[(Int, Int, Int)] = { + Seq( + (0, 0, 0), + (0, 0, 0)) + } + + // set cssConf as common, because ShuffleClient is singleton + val cssConf = new CssConf() + cssConf.set("css.client.register.shuffle.retry.timeout", "12s") + cssConf.set("css.client.register.shuffle.retry.init.interval", "500ms") + cssConf.set("css.local.chunk.fetch.enabled", "false") + cssConf.set("css.test.mode", "true") + + test("shuffle client singleton mode") { + cssConf.set("css.master.address", masterRef.address.toCssURL) + val shuffleClient1 = ShuffleClient.get(cssConf) + val shuffleClient2 = ShuffleClient.get(cssConf) + + assert(shuffleClient1 == shuffleClient2); + + shuffleClient1.shutDown() + shuffleClient2.shutDown() + } + + Seq("DISK", "HDFS").foreach { mode => + test(s"shuffle client batch push data $mode") { + cssConf.set("css.master.address", masterRef.address.toCssURL) + cssConf.set("css.shuffle.mode", mode) + + val appId = s"appId-${System.currentTimeMillis()}" + val shuffleId = 5 + val numMappers = 1 + val numPartitions = 1 + val maxPartitionsPerGroup = 1 + val mapperId = 0 + val mapperAttemptId = 0 + val reducerId = 0 + + val shuffleClient = new ShuffleClientImpl(cssConf) + ShuffleClient.cleanShuffle(shuffleId) + + val res1 = masterRef.askSync[RegisterPartitionGroupResponse]( + RegisterPartitionGroup(appId, shuffleId, numMappers, numPartitions, maxPartitionsPerGroup) + ) + assert(res1.statusCode == CssStatusCode.Success) + assert(res1.partitionGroups.size() == 1) + + shuffleClient.applyShufflePartitionGroup(shuffleId, res1.partitionGroups) + + val resultSet = new java.util.HashSet[String]() + var actualLength = 0L + (0 until 100).foreach(t => { + val content = RandomStringUtils.randomAlphanumeric(1024, 2048) + actualLength += content.length + resultSet.add(content) + val bytes = content.getBytes + shuffleClient.batchPushData(appId, shuffleId, mapperId, mapperAttemptId, + Array(reducerId), bytes, Array(0), Array(bytes.length), numMappers, numPartitions, false) + }) + // wait all batch push data request finish. + Thread.sleep(5000) + shuffleClient.mapperEnd(appId, shuffleId, mapperId, mapperAttemptId, numMappers) + Thread.sleep(5000) + + val reducerFileGroupsRes = masterRef.askSync[GetReducerFileGroupsResponse]( + GetReducerFileGroups(appId, shuffleId) + ) + assert(reducerFileGroupsRes.status == CssStatusCode.Success) + + val inputStream = shuffleClient.readPartitions(appId, shuffleId, Array(reducerId), mapperId, mapperId + 1) + + var resultCount = 0L + while (inputStream.read() != -1) { + resultCount += 1 + } + + assert(actualLength == resultCount) + inputStream.close() + + val inputStream2 = shuffleClient.readPartitions(appId, shuffleId, Array(reducerId), mapperId, mapperId + 1) + + val resultBytes = new Array[Byte](actualLength.toInt) + var index = 0 + val buffer = new Array[Byte](1024) + var readBytes = 0 + do { + readBytes = inputStream2.read(buffer, 0, 1024) + if (readBytes > 0) { + System.arraycopy(buffer, 0, resultBytes, index, readBytes) + index += readBytes + } + } while (readBytes != -1) + + var resultString = new String(resultBytes) + resultSet.asScala.foreach(f => resultString.indexOf(f) != -1) + resultSet.asScala.foreach{ f => + resultString = resultString.replace(f, "") + } + assert(resultString == "") + + inputStream2.close() + shuffleClient.shutDown() + } + } + + test("shuffle client batch push data retry") { + cssConf.set("css.master.address", masterRef.address.toCssURL) + + val appId = s"appId-${System.currentTimeMillis()}" + val shuffleId = 10 + val numMappers = 1 + val numPartitions = 1 + val maxPartitionsPerGroup = 1 + val mapperId = 0 + val mapperAttemptId = 0 + val reducerId = 0 + + val shuffleClient = ShuffleClient.get(cssConf) + ShuffleClient.cleanShuffle(shuffleId) + + val res1 = masterRef.askSync[RegisterPartitionGroupResponse]( + RegisterPartitionGroup(appId, shuffleId, numMappers, numPartitions, maxPartitionsPerGroup) + ) + assert(res1.statusCode == CssStatusCode.Success) + assert(res1.partitionGroups.size() == 1) + + shuffleClient.applyShufflePartitionGroup(shuffleId, res1.partitionGroups) + val partitionGroup = res1.partitionGroups.asScala.head + + val resultSet = new java.util.HashSet[String]() + var actualLength = 0L + + // just test retry to batch push data + { + val content = RandomStringUtils.randomAlphanumeric(1024, 2048) + actualLength += content.length + resultSet.add(content) + val bytes = content.getBytes + shuffleClient.batchPushData(appId, shuffleId, mapperId, mapperAttemptId, + Array(reducerId), bytes, Array(0), Array(bytes.length), numMappers, numPartitions, false) + } + Thread.sleep(5000) + + // because here only 2 workers. so worker head must replica 0. + // try set Exception for reduceId 0 epoch 0 & replica index 0. + sendWorkerRpc[BreakPartition, BreakPartitionResponse](actualWorkers.head._1, + BreakPartition(Utils.getShuffleKey(appId, shuffleId), reducerId, partitionGroup.epochId)) + Thread.sleep(2000) + + { + val content = RandomStringUtils.randomAlphanumeric(1024, 2048) + actualLength += content.length + resultSet.add(content) + val bytes = content.getBytes + shuffleClient.batchPushData(appId, shuffleId, mapperId, mapperAttemptId, + Array(reducerId), bytes, Array(0), Array(bytes.length), numMappers, numPartitions, false) + } + + Thread.sleep(8000) + shuffleClient.mapperEnd(appId, shuffleId, mapperId, mapperAttemptId, numMappers) + + Thread.sleep(2000) + val reducerFileGroupsRes = masterRef.askSync[GetReducerFileGroupsResponse]( + GetReducerFileGroups(appId, shuffleId) + ) + + var maxEpoch = -1 + reducerFileGroupsRes.fileGroup.foreach(reducerFileGroupsRes => { + reducerFileGroupsRes.foreach(p => { + if (p.getEpochId > maxEpoch) { + maxEpoch = p.getEpochId + } + }) + }) + + assert(reducerFileGroupsRes.status == CssStatusCode.Success) + assert(maxEpoch == 1) + + val inputStream = shuffleClient.readPartitions(appId, shuffleId, Array(reducerId), mapperId, mapperId + 1) + + var resultCount = 0L + while (inputStream.read() != -1) { + resultCount += 1 + } + + assert(actualLength == resultCount) + inputStream.close() + + val inputStream2 = shuffleClient.readPartitions(appId, shuffleId, Array(reducerId), mapperId, mapperId + 1) + + val resultBytes = new Array[Byte](actualLength.toInt) + var index = 0 + val buffer = new Array[Byte](1024) + var readBytes = 0 + do { + readBytes = inputStream2.read(buffer, 0, 1024) + if (readBytes > 0) { + System.arraycopy(buffer, 0, resultBytes, index, readBytes) + index += readBytes + } + } while (readBytes != -1) + + var resultString = new String(resultBytes) + resultSet.asScala.foreach(f => resultString.indexOf(f) != -1) + resultSet.asScala.foreach{ f => + resultString = resultString.replace(f, "") + } + assert(resultString == "") + + inputStream2.close() + shuffleClient.shutDown() + } + + test("skip batch push data after stage end") { + cssConf.set("css.master.address", masterRef.address.toCssURL) + + val appId = s"appId-${System.currentTimeMillis()}" + val shuffleId = 15 + val numMappers = 1 + val numPartitions = 1 + val maxPartitionsPerGroup = 1 + val mapperId = 0 + val mapperAttemptId = 0 + val reducerId = 0 + + val shuffleClient = ShuffleClient.get(cssConf) + ShuffleClient.cleanShuffle(shuffleId) + + val res1 = masterRef.askSync[RegisterPartitionGroupResponse]( + RegisterPartitionGroup(appId, shuffleId, numMappers, numPartitions, maxPartitionsPerGroup) + ) + assert(res1.statusCode == CssStatusCode.Success) + assert(res1.partitionGroups.size() == 1) + + shuffleClient.applyShufflePartitionGroup(shuffleId, res1.partitionGroups) + + val resultSet = new java.util.HashSet[String]() + var actualLength = 0L + (0 until 100).foreach(t => { + val content = RandomStringUtils.randomAlphanumeric(1024, 2048) + actualLength += content.length + resultSet.add(content) + val bytes = content.getBytes + shuffleClient.batchPushData(appId, shuffleId, mapperId, mapperAttemptId, + Array(reducerId), bytes, Array(0), Array(bytes.length), numMappers, numPartitions, false) + }) + + // trigger mapped end & stage end. so next batch push data will be skip. + shuffleClient.mapperEnd(appId, shuffleId, mapperId, mapperAttemptId, numMappers) + Thread.sleep(5000) + + // skip all batch push data. + (0 until 5).foreach(t => { + val content = RandomStringUtils.randomAlphanumeric(1024, 2048) + val bytes = content.getBytes + shuffleClient.batchPushData(appId, shuffleId, mapperId, mapperAttemptId, + Array(reducerId), bytes, Array(0), Array(bytes.length), numMappers, numPartitions, false) + }) + + val inputStream = shuffleClient.readPartitions(appId, shuffleId, Array(reducerId), mapperId, mapperId + 1) + + var resultCount = 0L + while (inputStream.read() != -1) { + resultCount += 1 + } + + assert(actualLength == resultCount) + inputStream.close() + + val inputStream2 = shuffleClient.readPartitions(appId, shuffleId, Array(reducerId), mapperId, mapperId + 1) + + val resultBytes = new Array[Byte](actualLength.toInt) + var index = 0 + val buffer = new Array[Byte](1024) + var readBytes = 0 + do { + readBytes = inputStream2.read(buffer, 0, 1024) + if (readBytes > 0) { + System.arraycopy(buffer, 0, resultBytes, index, readBytes) + index += readBytes + } + } while (readBytes != -1) + + var resultString = new String(resultBytes) + resultSet.asScala.foreach(f => resultString.indexOf(f) != -1) + resultSet.asScala.foreach{ f => + resultString = resultString.replace(f, "") + } + assert(resultString == "") + + inputStream2.close() + shuffleClient.shutDown() + } +} diff --git a/service/src/test/scala/com/bytedance/css/service/deploy/master/StageEndSuite.scala b/service/src/test/scala/com/bytedance/css/service/deploy/master/StageEndSuite.scala new file mode 100644 index 0000000..27db7f4 --- /dev/null +++ b/service/src/test/scala/com/bytedance/css/service/deploy/master/StageEndSuite.scala @@ -0,0 +1,102 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.service.deploy.master + +import java.util +import java.util.concurrent.CompletableFuture + +import scala.collection.JavaConverters._ + +import com.bytedance.css.common.protocol.{CssStatusCode, PartitionInfo, ShuffleMode} +import com.bytedance.css.common.protocol.CssRpcMessage._ +import com.bytedance.css.common.util.Utils +import org.apache.commons.lang3.RandomStringUtils + +class StageEndSuite extends LocalClusterSuite { + + test("E2E Test for 3 * 5 mixed scenario") { + // registerShuffle + val appId = s"appId-${System.currentTimeMillis()}" + val shuffleId = 5 + val numMappers = 3 + val numPartitions = 5 + val maxPartitionsPerGroup = 1 + + val res = masterRef.askSync[RegisterPartitionGroupResponse]( + RegisterPartitionGroup(appId, shuffleId, numMappers, numPartitions, maxPartitionsPerGroup) + ) + assert(res.statusCode == CssStatusCode.Success) + assert(res.partitionGroups.size() == 5) + + val partitionGroups = res.partitionGroups + + // make partition index 0 empty + // break partition index 1 for either replicas piece + // other partition just normal push + val allfutureList = new util.ArrayList[CompletableFuture[_]]() + val futuresList = new util.ArrayList[Array[CompletableFuture[Int]]]() + (0 until 10000).foreach(t => { + val content = RandomStringUtils.randomAlphanumeric(1024, 2048) + val bytes = content.getBytes + (1 until 5).foreach(index => { + val pg = partitionGroups.get(index) + val reducerId = pg.startPartition // because 1 partition 1 group. so sp is reduceId. + val (allFuture, futures) = batchPushData(appId, shuffleId, + Array(reducerId), pg.epochId, 0, + bytes, Array(0, bytes.length), pg.getReplicaWorkers.asScala.toArray, ShuffleMode.DISK.toString) + allfutureList.add(allFuture) + futuresList.add(futures) + }) + }) + allfutureList.asScala.foreach(_.get()) + assert(!futuresList.asScala.flatten.exists(_.get() != 0)) + + // break index 1 partition's replica index 0 partition. + val breakPartition = partitionGroups.get(1) + val breakWorkerRpcPort = actualWorkers.filter(_._2 == breakPartition.getReplicaWorkers.get(0).port).map(_._1).head + + sendWorkerRpc[BreakPartition, BreakPartitionResponse](breakWorkerRpcPort, + BreakPartition(Utils.getShuffleKey(appId, shuffleId), + partitionGroups.get(1).startPartition, partitionGroups.get(1).epochId)) + + // send mapper end + val epochList = new util.ArrayList[PartitionInfo]() + // initialPartitions index 0 has no data + epochList.addAll((1 until 5).map(partitionGroups.get) + .map(p => new PartitionInfo(p.startPartition, p.epochId)).asJava) + masterRef.askSync[MapperEndResponse](MapperEnd(appId, shuffleId, mapId = 0, attemptId = 0, 3, epochList, null)) + masterRef.askSync[MapperEndResponse](MapperEnd(appId, shuffleId, mapId = 1, attemptId = 4, 3, epochList, null)) + masterRef.askSync[MapperEndResponse](MapperEnd(appId, shuffleId, mapId = 1, attemptId = 0, 3, epochList, null)) + masterRef.askSync[MapperEndResponse](MapperEnd(appId, shuffleId, mapId = 2, attemptId = 1, 3, epochList, null)) + + // wait for trigger & finish stage end. + Thread.sleep(5000) + + val reducerFileGroupsRes = masterRef.askSync[GetReducerFileGroupsResponse]( + GetReducerFileGroups(appId, shuffleId) + ) + + assert(reducerFileGroupsRes.status == CssStatusCode.Success) + // filter two empty piece and a broken one + assert(reducerFileGroupsRes.fileGroup.flatten.length == 7) + // for partition 1 mapper attempt 4 comes first. + reducerFileGroupsRes.attempts.zip(Array(0, 4, 1)).foreach(f => assert(f._1 == f._2)) + } +} diff --git a/spark-shuffle-manager-2/pom.xml b/spark-shuffle-manager-2/pom.xml new file mode 100644 index 0000000..a56e65b --- /dev/null +++ b/spark-shuffle-manager-2/pom.xml @@ -0,0 +1,116 @@ + + + + 4.0.0 + + com.bytedance.inf + css-root + 1.0.0 + ../pom.xml + + + shuffle-manager-2 + jar + Cloud Shuffle Service Shuffle Manager for Spark + + + + com.bytedance.inf + css-api + ${project.version} + + + com.bytedance.inf + css-client_${scala.binary.version} + ${project.version} + + + junit + junit + test + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scala-lang + scala-library + ${scala.version} + provided + + + org.apache.spark + spark-core_${scala.binary.version} + ${spark.version} + provided + + + org.apache.hadoop + hadoop-client + + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} + provided + + + com.bytedance.inf + css-common_${scala.binary.version} + ${project.version} + compile + + + org.apache.hadoop + hadoop-common + ${hadoop.version} + provided + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + net.alchim31.maven + scala-maven-plugin + + + org.apache.maven.plugins + maven-shade-plugin + + + + io.dropwizard.metrics:metrics-core + com.fasterxml.jackson.core:* + org.slf4j:* + + + + + + + diff --git a/spark-shuffle-manager-2/src/main/java/org/apache/spark/shuffle/css/AsyncPushDataTaskManager.java b/spark-shuffle-manager-2/src/main/java/org/apache/spark/shuffle/css/AsyncPushDataTaskManager.java new file mode 120000 index 0000000..e420fb1 --- /dev/null +++ b/spark-shuffle-manager-2/src/main/java/org/apache/spark/shuffle/css/AsyncPushDataTaskManager.java @@ -0,0 +1 @@ +../../../../../../../../../spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/AsyncPushDataTaskManager.java \ No newline at end of file diff --git a/spark-shuffle-manager-2/src/main/java/org/apache/spark/shuffle/css/CssShuffleWriter.java b/spark-shuffle-manager-2/src/main/java/org/apache/spark/shuffle/css/CssShuffleWriter.java new file mode 100644 index 0000000..55e19ef --- /dev/null +++ b/spark-shuffle-manager-2/src/main/java/org/apache/spark/shuffle/css/CssShuffleWriter.java @@ -0,0 +1,290 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.shuffle.css; + +import com.bytedance.css.client.ShuffleClient; +import com.bytedance.css.common.CssConf; +import org.apache.spark.Partitioner; +import org.apache.spark.ShuffleDependency; +import org.apache.spark.SparkConf; +import org.apache.spark.TaskContext; +import org.apache.spark.executor.ShuffleWriteMetrics; +import org.apache.spark.memory.TaskMemoryManager; +import org.apache.spark.scheduler.MapStatus; +import org.apache.spark.scheduler.MapStatus$; +import org.apache.spark.serializer.SerializationStream; +import org.apache.spark.serializer.SerializerInstance; +import org.apache.spark.shuffle.BaseShuffleHandle; +import org.apache.spark.shuffle.ShuffleWriter; +import org.apache.spark.shuffle.css.sort.CssShuffleExternalSorter; +import org.apache.spark.storage.BlockManagerId; +import org.apache.spark.storage.BlockManagerId$; +import org.apache.spark.unsafe.Platform; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Option; +import scala.Product2; +import scala.collection.Iterator; +import scala.reflect.ClassTag; +import scala.reflect.ClassTag$; + +import javax.annotation.Nullable; +import java.io.ByteArrayOutputStream; +import java.io.IOException; + +public class CssShuffleWriter extends ShuffleWriter { + // Refer to UnsafeShuffleWriter + private static final Logger logger = LoggerFactory.getLogger(CssShuffleWriter.class); + private static final ClassTag OBJECT_CLASS_TAG = ClassTag$.MODULE$.Object(); + + private static final int DEFAULT_INITIAL_SER_BUFFER_SIZE = 1024 * 1024; + private final int PARTITION_GROUP_PUSH_BUFFER_SIZE; + private final int PUSH_QUEUE_CAPACITY; + + private final TaskMemoryManager memoryManager; + private final ShuffleDependency dep; + private final SerializerInstance serializer; + private final Partitioner partitioner; + private final ShuffleWriteMetrics writeMetrics; + private final ShuffleWriteMetricsAdapter writeMetricsAdapter; + + // shuffle meta info + private final String appId; + private final int shuffleId; + private final int mapId; + private final TaskContext taskContext; + private final CssConf cssConf; + private final SparkConf sparkConf; + private final ShuffleClient cssShuffleClient; + private final int numMappers; + private final int numPartitions; + + @Nullable private CssShuffleExternalSorter sorter; + @Nullable private MapStatus mapStatus; + // for overall mapStatus metrics update with each partitions + private final long[] partitionSizes; + + private long peakMemoryUsedBytes = 0; + + /** Subclass of ByteArrayOutputStream that exposes `buf` directly. */ + private static final class MyByteArrayOutputStream extends ByteArrayOutputStream { + MyByteArrayOutputStream(int size) { super(size); } + public byte[] getBuf() { return buf; } + } + + private final MyByteArrayOutputStream serBuffer; + private final SerializationStream serOutputStream; + + /** + * Are we in the process of stopping? Because map tasks can call stop() with success = true + * and then call stop() with success = false if they get an exception, we want to make sure + * we don't try deleting files, etc twice. + */ + private volatile boolean stopping = false; + private AsyncPushDataTaskManager taskManager; + + private final int initialSortBufferSize; + private final long sortPushSpillSizeThreshold; + private final long sortPushSpillRecordThreshold; + + + public CssShuffleWriter( + BaseShuffleHandle handle, + int mapId, + TaskContext taskContext, + SparkConf sparkConf, + CssConf cssConf, + ShuffleClient cssShuffleClient) throws IOException { + dep = handle.dependency(); + serializer = dep.serializer().newInstance(); + partitioner = dep.partitioner(); + this.taskContext = taskContext; + this.memoryManager = taskContext.taskMemoryManager(); + writeMetrics = taskContext.taskMetrics().shuffleWriteMetrics(); + this.writeMetricsAdapter = new ShuffleWriteMetricsAdapter() { + @Override + public void incBytesWritten(long v) { + writeMetrics.incBytesWritten(v); + } + + @Override + public void incWriteTime(long v) { + writeMetrics.incWriteTime(v); + } + + @Override + public void incRecordsWritten(long v) { + writeMetrics.incRecordsWritten(v); + } + }; + appId = ((CssShuffleHandle) handle).appId(); + shuffleId = dep.shuffleId(); + this.mapId = mapId; + this.sparkConf = sparkConf; + this.cssConf = cssConf; + this.cssShuffleClient = cssShuffleClient; + numMappers = handle.numMaps(); + numPartitions = handle.dependency().partitioner().numPartitions(); + this.initialSortBufferSize = sparkConf.getInt("spark.shuffle.sort.initialBufferSize", 4096); + + // CSS Configuration + PARTITION_GROUP_PUSH_BUFFER_SIZE = (int) CssConf.partitionGroupPushBufferSize(cssConf); + PUSH_QUEUE_CAPACITY = CssConf.pushQueueCapacity(cssConf); + sortPushSpillSizeThreshold = CssConf.sortPushSpillSizeThreshold(cssConf); + sortPushSpillRecordThreshold = CssConf.sortPushSpillRecordThreshold(cssConf); + + serBuffer = new MyByteArrayOutputStream(DEFAULT_INITIAL_SER_BUFFER_SIZE); + serOutputStream = serializer.serializeStream(serBuffer); + + // written bytes and record metrics report + partitionSizes = new long[numPartitions]; + } + + private void createExtSorter() { + assert (sorter == null); + sorter = new CssShuffleExternalSorter( + memoryManager, + taskContext, + taskManager, + sortPushSpillSizeThreshold, + sortPushSpillRecordThreshold, + initialSortBufferSize, + PARTITION_GROUP_PUSH_BUFFER_SIZE, + ShuffleClient.shufflePartitionGroupMap.get(shuffleId), + sparkConf, + writeMetricsAdapter); + } + + @Override + public void write(Iterator> records) throws IOException { + if (records.hasNext()) { + taskManager = new AsyncPushDataTaskManager(appId, shuffleId, mapId, taskContext.attemptNumber(), + numMappers, numPartitions, cssShuffleClient, writeMetricsAdapter, PUSH_QUEUE_CAPACITY); + + createExtSorter(); + + if (dep.mapSideCombine()) { + if (dep.aggregator().isEmpty()) { + throw new UnsupportedOperationException("map side combine"); + } + doSortWrite(dep.aggregator().get().combineValuesByKey(records, taskContext)); + } else { + doSortWrite(records); + } + } + close(); + } + + private void doSortWrite(scala.collection.Iterator iterator) throws IOException { + logger.info("Enter doSortWrite"); + final scala.collection.Iterator> records = iterator; + boolean success = false; + try { + while (records.hasNext()) { + assert(sorter != null); + final Product2 record = records.next(); + final K key = record._1(); + final int partitionId = partitioner.getPartition(key); + serBuffer.reset(); + serOutputStream.writeKey(key, OBJECT_CLASS_TAG); + serOutputStream.writeValue(record._2(), OBJECT_CLASS_TAG); + serOutputStream.flush(); + + final int serializedRecordSize = serBuffer.size(); + assert (serializedRecordSize > 0); + + sorter.insertRecord( + serBuffer.getBuf(), Platform.BYTE_ARRAY_OFFSET, serializedRecordSize, partitionId); + } + sorter.close(); + success = true; + } finally { + if (sorter != null) { + try { + sorter.cleanupResources(); + } catch (Exception e) { + // Only throw this error if we won't be masking another + // error. + if (success) { + throw e; + } else { + logger.error("In addition to a failure during writing, we failed during cleanup.", e); + } + } + } + } + } + + // send out last batches of data and call MapperEnd + // also update MapStatus + private void close() throws IOException { + + // wait for taskManager until noMorePendingTask + if (taskManager != null) { + taskManager.awaitTermination(); + // add flush buffer writer size & direct writer size + long[] flushBufferPartitionSizes = taskManager.getPartitionSizes(); + for (int i = 0; i < numPartitions; i++) { + partitionSizes[i] += flushBufferPartitionSizes[i]; + } + } + + // Send MapperEnd will trigger ShuffleClient limitMaxInFlight + // which requires all pushDataRequest being finished. + long waitStartTime = System.nanoTime(); + cssShuffleClient.mapperEnd(appId, shuffleId, mapId, taskContext.attemptNumber(), numMappers); + writeMetrics.incWriteTime(System.nanoTime() - waitStartTime); + + BlockManagerId dummyId = BlockManagerId$.MODULE$.apply( + "CSS-Writer", "127.0.0.1", 9527, Option.apply(null)); + mapStatus = MapStatus$.MODULE$.apply(dummyId, partitionSizes); + } + + @Override + public Option stop(boolean success) { + try { + taskContext.taskMetrics().incPeakExecutionMemory(peakMemoryUsedBytes); + + if (stopping) { + return Option.apply(null); + } else { + stopping = true; + if (taskManager != null) { + taskManager.setStopping(); + } + if (success) { + if (mapStatus == null) { + throw new IllegalStateException("Cannot call stop(true) without having called write()"); + } + return Option.apply(mapStatus); + } else { + return Option.apply(null); + } + } + } finally { + if (taskManager != null) { + taskManager = null; + } + if (cssShuffleClient != null) { + cssShuffleClient.mapperClose(appId, shuffleId, mapId, taskContext.attemptNumber()); + } + } + } +} diff --git a/spark-shuffle-manager-2/src/main/java/org/apache/spark/shuffle/css/sort b/spark-shuffle-manager-2/src/main/java/org/apache/spark/shuffle/css/sort new file mode 120000 index 0000000..8cbe45f --- /dev/null +++ b/spark-shuffle-manager-2/src/main/java/org/apache/spark/shuffle/css/sort @@ -0,0 +1 @@ +../../../../../../../../../spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort \ No newline at end of file diff --git a/spark-shuffle-manager-2/src/main/scala/org/apache/spark/shuffle/css/CssShuffleHandle.scala b/spark-shuffle-manager-2/src/main/scala/org/apache/spark/shuffle/css/CssShuffleHandle.scala new file mode 100644 index 0000000..32a388e --- /dev/null +++ b/spark-shuffle-manager-2/src/main/scala/org/apache/spark/shuffle/css/CssShuffleHandle.scala @@ -0,0 +1,37 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.shuffle.css + +import java.util + +import com.bytedance.css.common.protocol.PartitionGroup + +import org.apache.spark.ShuffleDependency +import org.apache.spark.shuffle.BaseShuffleHandle + +// Add extra appId +class CssShuffleHandle[K, V]( + val appId: String, + shuffleId: Int, + numMaps: Int, + val partitionGroups: util.List[PartitionGroup], + dependency: ShuffleDependency[K, V, V]) + extends BaseShuffleHandle(shuffleId, numMaps, dependency) { +} diff --git a/spark-shuffle-manager-2/src/main/scala/org/apache/spark/shuffle/css/CssShuffleManager.scala b/spark-shuffle-manager-2/src/main/scala/org/apache/spark/shuffle/css/CssShuffleManager.scala new file mode 100644 index 0000000..b5850f9 --- /dev/null +++ b/spark-shuffle-manager-2/src/main/scala/org/apache/spark/shuffle/css/CssShuffleManager.scala @@ -0,0 +1,188 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.shuffle.css + +import java.util + +import com.bytedance.css.api.CssShuffleContext +import com.bytedance.css.client.ShuffleClient +import com.bytedance.css.client.metrics.ClientSource +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.metrics.MetricsSystem +import com.bytedance.css.common.util.Utils + +import org.apache.spark.{ShuffleDependency, SparkConf, SparkContext, SparkEnv, TaskContext} +import org.apache.spark.internal.config._ +import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleBlockResolver, ShuffleHandle} +import org.apache.spark.shuffle.{ShuffleManager, ShuffleReader, ShuffleWriter} +import org.apache.spark.shuffle.css.CssShuffleManager.getAppId + +class CssShuffleManager(conf: SparkConf) extends ShuffleManager with Logging { + + private lazy val cssConf = CssShuffleManager.fromSparkConf(conf) + + private lazy val cssShuffleClient: ShuffleClient = ShuffleClient.get(cssConf) + private lazy val maxPartitionsPerGroup = CssConf.maxPartitionsPerGroup(cssConf) + private lazy val cssClusterName = CssConf.clusterName(cssConf) + private var appId: Option[String] = None + @volatile private var metricsInitialized = false + + if (!CssConf.workerRegistryType(cssConf).equals("standalone")) { + if (conf.getOption("spark.executor.id").exists(_.equals(SparkContext.DRIVER_IDENTIFIER)) && + !conf.contains("spark.css.master.identifier")) { + val cssParams = new util.HashMap[String, String]() + cssConf.getAll.foreach(kv => cssParams.put(kv._1, kv._2)) + val host = conf.get(DRIVER_HOST_ADDRESS) + logInfo("Spark Driver try to start css master if needed.") + CssShuffleContext.get.startMaster(host, 0, cssParams) + + // Set master host & port for CssShuffleManager + val cssMasterAddr = s"css://${host}:${CssShuffleContext.get().getMasterPort.toString}" + cssConf.set("css.master.address", cssMasterAddr) + conf.set("spark.css.master.address", cssMasterAddr) + conf.set("spark.css.master.identifier", "driver-master") + + // initialize shuffle workers num + val estimatedWorkers: Int = if (conf.getBoolean("spark.dynamicAllocation.enabled", false)) { + (conf.get(DYN_ALLOCATION_MAX_EXECUTORS) / 50 + conf.get(DYN_ALLOCATION_MIN_EXECUTORS)) / 10 + } else { + conf.get(EXECUTOR_INSTANCES).getOrElse(0) / 20 + } + + val initWorkers = Math.max(2, estimatedWorkers) + require(initWorkers != 0, "initWorkers could not be zero.") + CssShuffleContext.get().allocateWorkerIfNeeded(initWorkers) + } + } + + override def registerShuffle[K, V, C]( + shuffleId: Int, + numMaps: Int, + dependency: ShuffleDependency[K, V, C]): ShuffleHandle = { + + // enable heartbeat + appId = Some(getAppId(dependency.rdd.context)) + cssShuffleClient.registerApplication(appId.get) + + logInfo(s"Css RegisterShuffle in Spark Driver with shuffleId: $shuffleId") + new CssShuffleHandle( + appId.get, + shuffleId, + numMaps, + cssShuffleClient.registerPartitionGroup( + appId.get, shuffleId, + dependency.rdd.getNumPartitions, dependency.partitioner.numPartitions, maxPartitionsPerGroup), + dependency.asInstanceOf[ShuffleDependency[K, V, V]]) + } + + override def getWriter[K, V]( + handle: ShuffleHandle, + mapId: Int, + context: TaskContext): ShuffleWriter[K, V] = { + val cssShuffleHandle = handle.asInstanceOf[CssShuffleHandle[K, V]] + initializeMetrics(cssShuffleHandle.appId) + cssShuffleClient.applyShufflePartitionGroup(cssShuffleHandle.shuffleId, cssShuffleHandle.partitionGroups) + handle match { + case h: BaseShuffleHandle[K@unchecked, V@unchecked, _] => + new CssShuffleWriter(h, mapId, context, conf, cssConf, cssShuffleClient) + } + } + + override def getReader[K, C]( + handle: ShuffleHandle, + startPartition: Int, + endPartition: Int, + context: TaskContext): ShuffleReader[K, C] = { + val cssShuffleHandle = handle.asInstanceOf[CssShuffleHandle[K, C]] + initializeMetrics(cssShuffleHandle.appId) + new CssShuffleReader[K, C](cssShuffleHandle, 0, cssShuffleHandle.numMaps, + startPartition, endPartition, context, cssShuffleClient) + } + + override def unregisterShuffle(shuffleId: Int): Boolean = { + // In community spark + // unregisterShuffle is called by all nodes include driver & executor to unregister with external + // shuffle service. + // But in CSS, we should only call unregisterShuffle once will be good. + appId match { + case Some(id) => + cssShuffleClient.unregisterShuffle(id, shuffleId, + SparkEnv.get.executorId == SparkContext.DRIVER_IDENTIFIER) + case None => + } + true + } + + // No need for CSS, because we deal with partition Resolver in ShuffleClient + override def shuffleBlockResolver: ShuffleBlockResolver = null + + override def stop(): Unit = { + if (SparkEnv.get.executorId != SparkContext.DRIVER_IDENTIFIER) { + Utils.tryLogNonFatalError(cssShuffleClient.shutDown()) + } else { + logInfo("Spark Driver try to stop css master") + Utils.tryLogNonFatalError(cssShuffleClient.shutDown()) + Utils.tryLogNonFatalError(CssShuffleContext.get.stopMaster()) + } + } + + def initializeMetrics(appId: String): Unit = { + if (!metricsInitialized) { + synchronized { + if (!metricsInitialized) { + cssConf.set("css.metrics.conf.*.sink.bytedance.prefix", "inf.spark") + val metricsSystem = MetricsSystem.createMetricsSystem(MetricsSystem.CLIENT, cssConf) + metricsSystem.registerSource(ClientSource.instance(cssClusterName, appId)) + metricsSystem.start() + metricsInitialized = true + } + } + } + } + +} + +object CssShuffleManager { + + def getAppId(context: SparkContext): String = { + context.applicationAttemptId match { + case Some(id) => s"${context.applicationId}_$id" + case None => s"${context.applicationId}" + } + } + + // parsing spark.css.* into css.* to construct CssConf + def fromSparkConf(conf: SparkConf): CssConf = { + val tmpConf = new CssConf() + for ((key, value) <- conf.getAll) { + if (key.startsWith("spark.css.")) { + tmpConf.set(key.substring("spark.".length), value) + // if zk mode enable. + if (key.equals("spark.css.zookeeper.address")) { + tmpConf.set("css.worker.registry.type", "zookeeper") + } + } + } + + tmpConf + } + +} diff --git a/spark-shuffle-manager-2/src/main/scala/org/apache/spark/shuffle/css/CssShuffleReader.scala b/spark-shuffle-manager-2/src/main/scala/org/apache/spark/shuffle/css/CssShuffleReader.scala new file mode 100644 index 0000000..96df024 --- /dev/null +++ b/spark-shuffle-manager-2/src/main/scala/org/apache/spark/shuffle/css/CssShuffleReader.scala @@ -0,0 +1,130 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.shuffle.css + +import com.bytedance.css.client.{MetricsCallback, ShuffleClient} +import com.bytedance.css.client.stream.CssInputStream +import com.bytedance.css.common.internal.Logging + +import org.apache.spark.{InterruptibleIterator, TaskContext} +import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleReader} +import org.apache.spark.util.CompletionIterator +import org.apache.spark.util.collection.ExternalSorter + +// Refer to BlockStoreShuffleReader +class CssShuffleReader[K, C]( + handle: BaseShuffleHandle[K, _, C], + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + val cssShuffleClient: ShuffleClient) extends ShuffleReader[K, C] with Logging { + + private val dep = handle.dependency + + override def read(): Iterator[Product2[K, C]] = { + + val serializerInstance = dep.serializer.newInstance() + + // Update the context task metrics via callback + val readMetrics = context.taskMetrics.createTempShuffleReadMetrics() + val metricsCallback = new MetricsCallback { + override def incBytesRead(bytesRead: Long): Unit = + readMetrics.incRemoteBytesRead(bytesRead) + + override def incReadTime(time: Long): Unit = + readMetrics.incFetchWaitTime(time) + } + + val inputStreams = if (handle.numMaps > 0) { + val start = System.currentTimeMillis() + val inputStream = cssShuffleClient.readPartitions( + handle.asInstanceOf[CssShuffleHandle[_, _]].appId, + handle.shuffleId, + (startPartition until endPartition).toArray, + startMapIndex, + endMapIndex) + metricsCallback.incReadTime(System.currentTimeMillis() - start) + inputStream.setCallback(metricsCallback) + context.addTaskCompletionListener(_ => inputStream.close()) + Seq(inputStream) + } else { + Seq(CssInputStream.empty()) + } + + val recordIter = inputStreams.toIterator.flatMap( + serializerInstance.deserializeStream(_).asKeyValueIterator + ) + + val metricIter = CompletionIterator[(Any, Any), Iterator[(Any, Any)]]( + recordIter.map { record => + readMetrics.incRecordsRead(1) + record + }, + context.taskMetrics().mergeShuffleReadMetrics()) + + // An interruptible iterator must be used here in order to support task cancellation + val interruptibleIter = new InterruptibleIterator[(Any, Any)](context, metricIter) + + val aggregatedIter: Iterator[Product2[K, C]] = if (dep.aggregator.isDefined) { + if (dep.mapSideCombine) { + // We are reading values that are already combined + val combinedKeyValuesIterator = interruptibleIter.asInstanceOf[Iterator[(K, C)]] + dep.aggregator.get.combineCombinersByKey(combinedKeyValuesIterator, context) + } else { + // We don't know the value type, but also don't care -- the dependency *should* + // have made sure its compatible w/ this aggregator, which will convert the value + // type to the combined type C + val keyValuesIterator = interruptibleIter.asInstanceOf[Iterator[(K, Nothing)]] + dep.aggregator.get.combineValuesByKey(keyValuesIterator, context) + } + } else { + interruptibleIter.asInstanceOf[Iterator[Product2[K, C]]] + } + + // Sort the output if there is a sort ordering defined. + val resultIter = dep.keyOrdering match { + case Some(keyOrd: Ordering[K]) => + // Create an ExternalSorter to sort the data. + val sorter = + new ExternalSorter[K, C, C](context, ordering = Some(keyOrd), serializer = dep.serializer) + sorter.insertAll(aggregatedIter) + context.taskMetrics().incMemoryBytesSpilled(sorter.memoryBytesSpilled) + context.taskMetrics().incDiskBytesSpilled(sorter.diskBytesSpilled) + context.taskMetrics().incPeakExecutionMemory(sorter.peakMemoryUsedBytes) + // Use completion callback to stop sorter if task was finished/cancelled. + context.addTaskCompletionListener(_ => { + sorter.stop() + }) + CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]](sorter.iterator, sorter.stop()) + case None => + aggregatedIter + } + + resultIter match { + case _: InterruptibleIterator[Product2[K, C]] => resultIter + case _ => + // Use another interruptible iterator here to support task cancellation as aggregator + // or(and) sorter may have consumed previous interruptible iterator. + new InterruptibleIterator[Product2[K, C]](context, resultIter) + } + } +} diff --git a/spark-shuffle-manager-3/pom.xml b/spark-shuffle-manager-3/pom.xml new file mode 100644 index 0000000..28bb24a --- /dev/null +++ b/spark-shuffle-manager-3/pom.xml @@ -0,0 +1,104 @@ + + + + 4.0.0 + + com.bytedance.inf + css-root + 1.0.0 + ../pom.xml + + + shuffle-manager-3 + jar + Cloud Shuffle Service Shuffle Manager for Spark + + + + com.bytedance.inf + css-api + ${project.version} + + + com.bytedance.inf + css-client_${scala.binary.version} + ${project.version} + + + junit + junit + test + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.scala-lang + scala-library + ${scala.version} + provided + + + org.apache.spark + spark-core_${scala.binary.version} + ${spark.version} + provided + + + org.apache.hadoop + hadoop-client + + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} + provided + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + net.alchim31.maven + scala-maven-plugin + + + org.apache.maven.plugins + maven-shade-plugin + + + + io.dropwizard.metrics:metrics-core + com.fasterxml.jackson.core:* + org.slf4j:* + + + + + + + diff --git a/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/AsyncPushDataTaskManager.java b/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/AsyncPushDataTaskManager.java new file mode 100644 index 0000000..285acba --- /dev/null +++ b/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/AsyncPushDataTaskManager.java @@ -0,0 +1,255 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.shuffle.css; + +import com.bytedance.css.client.ShuffleClient; + +import java.io.IOException; +import java.util.Arrays; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.ReentrantLock; + +public class AsyncPushDataTaskManager { + + private final long WAIT_TIME_NANOS = TimeUnit.MILLISECONDS.toNanos(500); + + private final String appId; + private final int shuffleId; + private final int mapperId; + private final int mapperAttemptId; + private final int numMappers; + private final int numPartitions; + private final ShuffleClient shuffleClient; + private final ShuffleWriteMetricsAdapter writeMetrics; + private final int pushQueueCapacity; + + // TODO remove it + private class PushDataTask { + int partitionId; + final byte[] buffer = new byte[0]; + int size; + } + + private final LinkedBlockingQueue pendingQueue; + private final LinkedBlockingQueue pushingQueue; + + // remain flush buffer writer size with each partitions. + private final long[] partitionSizes; + + // Refer to Spark ContextWaiter + private final ReentrantLock lock = new ReentrantLock(); + private final Condition noMorePendingTask = lock.newCondition(); + + // whenever there is exception occur, we should stop the entire mapperAttempt + private final AtomicReference exception = new AtomicReference<>(); + + private volatile boolean terminated; + private volatile boolean stopping; + + private final Thread pushDataTaskProcessor; + + public AsyncPushDataTaskManager( + String appId, + int shuffleId, + int mapperId, + int mapperAttemptId, + int numMappers, + int numPartitions, + ShuffleClient shuffleClient, + ShuffleWriteMetricsAdapter writeMetrics, + int pushQueueCapacity) throws IOException { + this.appId = appId; + this.shuffleId = shuffleId; + this.mapperId = mapperId; + this.mapperAttemptId = mapperAttemptId; + this.numMappers = numMappers; + this.numPartitions = numPartitions; + this.shuffleClient = shuffleClient; + this.writeMetrics = writeMetrics; + this.pushQueueCapacity = pushQueueCapacity; + + // written bytes and record metrics report + partitionSizes = new long[numPartitions]; + + pendingQueue = new LinkedBlockingQueue<>(this.pushQueueCapacity); + pushingQueue = new LinkedBlockingQueue<>(this.pushQueueCapacity); + + // initialize of pending queue, put PushDataTasks that has already with initialized byte buffer. + for (int i = 0; i < pushQueueCapacity; i++) { + try { + pendingQueue.put(new PushDataTask()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException(e); + } + } + + // create processor thread and start + pushDataTaskProcessor = new Thread( + String.format("AsyncPushDataTaskManager for %s-%s-%s-%s", appId, shuffleId, mapperId, mapperAttemptId)) { + @Override + public void run() { + while (!terminated && !stopping && exception.get() == null) { + try { + PushDataTask task = pushingQueue.poll(WAIT_TIME_NANOS, TimeUnit.NANOSECONDS); + if (task == null) { + continue; + } + executePushData(task.partitionId, task.buffer, task.size); + returnTaskBuffer(task); + } catch (InterruptedException e) { + exception.set(new IOException(e)); + } catch (IOException e) { + exception.set(e); + } + } + } + }; + pushDataTaskProcessor.start(); + } + + public void setStopping() { + stopping = true; + } + + private void returnTaskBuffer(PushDataTask task) throws InterruptedException { + lock.lockInterruptibly(); + try { + pendingQueue.put(task); + if (pendingQueue.remainingCapacity() == 0) { + noMorePendingTask.signal(); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + exception.set(new IOException(e)); + } finally { + lock.unlock(); + } + } + + public int directBatchPush( + int[] reducerIdArray, + byte[] data, + int[] offsetArray, + int[] lengthArray) throws IOException { + long pushStartTime = System.nanoTime(); + int[] partitionWrittenBytes = shuffleClient.batchPushData( + appId, + shuffleId, + mapperId, + mapperAttemptId, + reducerIdArray, + data, + offsetArray, + lengthArray, + numMappers, + numPartitions, + false); + for (int i = 0; i < reducerIdArray.length; i ++) { + partitionSizes[reducerIdArray[i]] += partitionWrittenBytes[i]; + } + int bytesWritten = Arrays.stream(partitionWrittenBytes).sum(); + writeMetrics.incBytesWritten(bytesWritten); + writeMetrics.incWriteTime(System.nanoTime() - pushStartTime); + return bytesWritten; + } + + public void safeDirectBatchPush(int[] reducerIdArray, byte[] data, int[] offsetArray, int[] lengthArray) { + try { + this.directBatchPush(reducerIdArray, data, offsetArray, lengthArray); + } catch (IOException ex) { + throw new RuntimeException(ex); + } + } + + public void safeAddPushDataTask(int partitionId, byte[] buffer, int size) { + try { + this.addPushDataTask(partitionId, buffer, size); + } catch (IOException ex) { + throw new RuntimeException(ex); + } + } + + public void addPushDataTask(int partitionId, byte[] buffer, int size) throws IOException { + try { + // fetch task buffer from pendingQueue + // copy buffer and set parameters + // put this task into pushingQueue + // wait for the local thread to fetch from pushingQueue and do executePushData + PushDataTask task = null; + while (task == null) { + checkException(); + task = pendingQueue.poll(WAIT_TIME_NANOS, TimeUnit.NANOSECONDS); + } + task.partitionId = partitionId; + System.arraycopy(buffer, 0, task.buffer, 0, size); + task.size = size; + while (!pushingQueue.offer(task, WAIT_TIME_NANOS, TimeUnit.NANOSECONDS)) { + checkException(); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + IOException ioe = new IOException(e); + exception.set(ioe); + throw ioe; + } + } + + private void executePushData(int partitionId, byte[] buffer, int size) throws IOException { + long pushStartTime = System.nanoTime(); + int bytesWritten = 0; + partitionSizes[partitionId] += bytesWritten; + writeMetrics.incBytesWritten(bytesWritten); + writeMetrics.incWriteTime(System.nanoTime() - pushStartTime); + } + + public long[] getPartitionSizes() { + return partitionSizes; + } + + private void checkException() throws IOException { + if (exception.get() != null) { + throw exception.get(); + } + } + + void awaitTermination() throws IOException { + try { + lock.lockInterruptibly(); + // when all task finished, pendingQueue.remainingCapacity = 0 and break + while (pendingQueue.remainingCapacity() > 0 && exception.get() == null) { + noMorePendingTask.await(WAIT_TIME_NANOS, TimeUnit.NANOSECONDS); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + exception.set(new IOException(e)); + } finally { + lock.unlock(); + } + + terminated = true; + pendingQueue.clear(); + pushingQueue.clear(); + checkException(); + } +} diff --git a/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/CssShuffleWriter.java b/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/CssShuffleWriter.java new file mode 100644 index 0000000..c44c633 --- /dev/null +++ b/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/CssShuffleWriter.java @@ -0,0 +1,293 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.shuffle.css; + +import com.bytedance.css.client.ShuffleClient; +import com.bytedance.css.common.CssConf; +import org.apache.spark.Partitioner; +import org.apache.spark.ShuffleDependency; +import org.apache.spark.SparkConf; +import org.apache.spark.TaskContext; +import org.apache.spark.internal.config.package$; +import org.apache.spark.memory.TaskMemoryManager; +import org.apache.spark.scheduler.MapStatus; +import org.apache.spark.scheduler.MapStatus$; +import org.apache.spark.serializer.SerializationStream; +import org.apache.spark.serializer.SerializerInstance; +import org.apache.spark.shuffle.BaseShuffleHandle; +import org.apache.spark.shuffle.ShuffleWriter; +import org.apache.spark.shuffle.css.sort.CssShuffleExternalSorter; +import org.apache.spark.storage.BlockManagerId; +import org.apache.spark.storage.BlockManagerId$; +import org.apache.spark.unsafe.Platform; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Option; +import scala.Product2; +import scala.collection.Iterator; +import scala.reflect.ClassTag; +import scala.reflect.ClassTag$; + +import javax.annotation.Nullable; +import java.io.ByteArrayOutputStream; +import java.io.IOException; + +public class CssShuffleWriter extends ShuffleWriter { + // Refer to UnsafeShuffleWriter + private static final Logger logger = LoggerFactory.getLogger(CssShuffleWriter.class); + private static final ClassTag OBJECT_CLASS_TAG = ClassTag$.MODULE$.Object(); + + private static final int DEFAULT_INITIAL_SER_BUFFER_SIZE = 1024 * 1024; + private final int PARTITION_GROUP_PUSH_BUFFER_SIZE; + private final int PUSH_QUEUE_CAPACITY; + + private final TaskMemoryManager memoryManager; + private final ShuffleDependency dep; + private final SerializerInstance serializer; + private final Partitioner partitioner; + private final ShuffleWriteMetricsAdapter writeMetrics; + + // shuffle meta info + private final String appId; + private final int shuffleId; + private final int mapId; + private final TaskContext taskContext; + private final CssConf cssConf; + private final SparkConf sparkConf; + private final ShuffleClient cssShuffleClient; + private final int numMappers; + private final int numPartitions; + + @Nullable private CssShuffleExternalSorter sorter; + @Nullable private MapStatus mapStatus; + // for overall mapStatus metrics update with each partitions + private final long[] partitionSizes; + + private long peakMemoryUsedBytes = 0; + + + /** Subclass of ByteArrayOutputStream that exposes `buf` directly. */ + private static final class MyByteArrayOutputStream extends ByteArrayOutputStream { + MyByteArrayOutputStream(int size) { super(size); } + public byte[] getBuf() { return buf; } + } + + private final MyByteArrayOutputStream serBuffer; + private final SerializationStream serOutputStream; + + /** + * Are we in the process of stopping? Because map tasks can call stop() with success = true + * and then call stop() with success = false if they get an exception, we want to make sure + * we don't try deleting files, etc twice. + */ + private volatile boolean stopping = false; + private AsyncPushDataTaskManager taskManager; + + private final int initialSortBufferSize; + private final long sortPushSpillSizeThreshold; + private final long sortPushSpillRecordThreshold; + + + public CssShuffleWriter( + BaseShuffleHandle handle, + int mapId, + TaskContext taskContext, + SparkConf sparkConf, + CssConf cssConf, + ShuffleClient cssShuffleClient, + ShuffleWriteMetricsAdapter writeMetrics) throws IOException { + dep = handle.dependency(); + serializer = dep.serializer().newInstance(); + partitioner = dep.partitioner(); + this.taskContext = taskContext; + this.memoryManager = taskContext.taskMemoryManager(); + this.writeMetrics = writeMetrics; + appId = ((CssShuffleHandle) handle).appId(); + shuffleId = dep.shuffleId(); + this.mapId = mapId; + this.sparkConf = sparkConf; + this.cssConf = cssConf; + this.cssShuffleClient = cssShuffleClient; + numMappers = ((CssShuffleHandle) handle).numMappers(); + numPartitions = handle.dependency().partitioner().numPartitions(); + this.initialSortBufferSize = (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_SORT_INIT_BUFFER_SIZE()); + + // CSS Configuration + PARTITION_GROUP_PUSH_BUFFER_SIZE = (int) CssConf.partitionGroupPushBufferSize(cssConf); + PUSH_QUEUE_CAPACITY = CssConf.pushQueueCapacity(cssConf); + sortPushSpillSizeThreshold = CssConf.sortPushSpillSizeThreshold(cssConf); + sortPushSpillRecordThreshold = CssConf.sortPushSpillRecordThreshold(cssConf); + + serBuffer = new MyByteArrayOutputStream(DEFAULT_INITIAL_SER_BUFFER_SIZE); + serOutputStream = serializer.serializeStream(serBuffer); + + // written bytes and record metrics report + partitionSizes = new long[numPartitions]; + } + + private void updatePeakMemoryUsed() { + // sorter can be null if this writer is closed + if (sorter != null) { + long mem = sorter.getPeakMemoryUsedBytes(); + if (mem > peakMemoryUsedBytes) { + peakMemoryUsedBytes = mem; + } + } + } + + /** + * Return the peak memory used so far, in bytes. + */ + public long getPeakMemoryUsedBytes() { + updatePeakMemoryUsed(); + return peakMemoryUsedBytes; + } + + private void createExtSorter() { + assert (sorter == null); + sorter = new CssShuffleExternalSorter( + memoryManager, + taskContext, + taskManager, + sortPushSpillSizeThreshold, + sortPushSpillRecordThreshold, + initialSortBufferSize, + PARTITION_GROUP_PUSH_BUFFER_SIZE, + ShuffleClient.shufflePartitionGroupMap.get(shuffleId), + sparkConf, + writeMetrics); + } + + @Override + public void write(Iterator> records) throws IOException { + if (records.hasNext()) { + taskManager = new AsyncPushDataTaskManager(appId, shuffleId, mapId, taskContext.attemptNumber(), + numMappers, numPartitions, cssShuffleClient, writeMetrics, PUSH_QUEUE_CAPACITY); + + createExtSorter(); + + if (dep.mapSideCombine()) { + if (dep.aggregator().isEmpty()) { + throw new UnsupportedOperationException("map side combine"); + } + doSortWrite(dep.aggregator().get().combineValuesByKey(records, taskContext)); + } else { + doSortWrite(records); + } + } + close(); + } + + private void doSortWrite(scala.collection.Iterator iterator) throws IOException { + logger.info("Enter doSortWrite"); + final scala.collection.Iterator> records = iterator; + boolean success = false; + try { + while (records.hasNext()) { + assert(sorter != null); + final Product2 record = records.next(); + final K key = record._1(); + final int partitionId = partitioner.getPartition(key); + serBuffer.reset(); + serOutputStream.writeKey(key, OBJECT_CLASS_TAG); + serOutputStream.writeValue(record._2(), OBJECT_CLASS_TAG); + serOutputStream.flush(); + + final int serializedRecordSize = serBuffer.size(); + assert (serializedRecordSize > 0); + + sorter.insertRecord( + serBuffer.getBuf(), Platform.BYTE_ARRAY_OFFSET, serializedRecordSize, partitionId); + } + sorter.close(); + success = true; + } finally { + if (sorter != null) { + try { + sorter.cleanupResources(); + } catch (Exception e) { + // Only throw this error if we won't be masking another + // error. + if (success) { + throw e; + } else { + logger.error("In addition to a failure during writing, we failed during cleanup.", e); + } + } + } + } + } + + // send out last batches of data and call MapperEnd + // also update MapStatus + private void close() throws IOException { + + // wait for taskManager until noMorePendingTask + if (taskManager != null) { + taskManager.awaitTermination(); + // add flush buffer writer size & direct writer size + long[] flushBufferPartitionSizes = taskManager.getPartitionSizes(); + for (int i = 0; i < numPartitions; i++) { + partitionSizes[i] += flushBufferPartitionSizes[i]; + } + } + + // Send MapperEnd will trigger ShuffleClient limitMaxInFlight + // which requires all pushDataRequest being finished. + long waitStartTime = System.nanoTime(); + cssShuffleClient.mapperEnd(appId, shuffleId, mapId, taskContext.attemptNumber(), numMappers); + writeMetrics.incWriteTime(System.nanoTime() - waitStartTime); + + BlockManagerId dummyId = BlockManagerId$.MODULE$.apply( + "CSS-Writer", "127.0.0.1", 9527, Option.apply(null)); + mapStatus = MapStatus$.MODULE$.apply(dummyId, partitionSizes, mapId); + } + + @Override + public Option stop(boolean success) { + try { + taskContext.taskMetrics().incPeakExecutionMemory(getPeakMemoryUsedBytes()); + + if (stopping) { + return Option.apply(null); + } else { + stopping = true; + if (taskManager != null) { + taskManager.setStopping(); + } + if (success) { + if (mapStatus == null) { + throw new IllegalStateException("Cannot call stop(true) without having called write()"); + } + return Option.apply(mapStatus); + } else { + return Option.apply(null); + } + } + } finally { + if (taskManager != null) { + taskManager = null; + } + if (cssShuffleClient != null) { + cssShuffleClient.mapperClose(appId, shuffleId, mapId, taskContext.attemptNumber()); + } + } + } +} diff --git a/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/CssShuffleExternalSorter.java b/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/CssShuffleExternalSorter.java new file mode 100644 index 0000000..9220ed6 --- /dev/null +++ b/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/CssShuffleExternalSorter.java @@ -0,0 +1,450 @@ +/* + * This file may have been modified by Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.shuffle.css.sort; + +import com.bytedance.css.common.protocol.PartitionGroupManager; +import org.apache.spark.SparkConf; +import org.apache.spark.TaskContext; +import org.apache.spark.memory.MemoryConsumer; +import org.apache.spark.memory.SparkOutOfMemoryError; +import org.apache.spark.memory.TaskMemoryManager; +import org.apache.spark.memory.TooLargePageException; +import org.apache.spark.shuffle.css.AsyncPushDataTaskManager; +import org.apache.spark.shuffle.css.ShuffleWriteMetricsAdapter; +import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.UnsafeAlignedOffset; +import org.apache.spark.unsafe.array.LongArray; +import org.apache.spark.unsafe.memory.MemoryBlock; +import org.apache.spark.util.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; + +/** + * An external sorter that is specialized for sort-based shuffle. + *

    + * Incoming records are appended to data pages. When all records have been inserted (or when the + * current thread's shuffle memory limit is reached), the in-memory records are sorted according to + * their partition ids (using a {@link ShuffleInMemorySorter}). The sorted records are then + * written to a single output file (or multiple files, if we've spilled). The format of the output + * files is the same as the format of the final output file written by + * {@link org.apache.spark.shuffle.sort.SortShuffleWriter}: each output partition's records are + * written as a single serialized, compressed stream that can be read with a new decompression and + * deserialization stream. + *

    + * Unlike {@link org.apache.spark.util.collection.ExternalSorter}, this sorter does not merge its + * spill files. Instead, this merging is performed in UnsafeShuffleWriter, which uses a + * specialized merge procedure that avoids extra serialization/deserialization. + */ +public final class CssShuffleExternalSorter extends MemoryConsumer { + + private static final Logger logger = LoggerFactory.getLogger(CssShuffleExternalSorter.class); + + private final TaskMemoryManager taskMemoryManager; + private final TaskContext taskContext; + private final AsyncPushDataTaskManager taskManager; + private final ShuffleWriteMetricsAdapter writeMetrics; + + private final int PARTITION_GROUP_PUSH_BUFFER_SIZE; + private final PartitionGroupManager partitionGroupManager; + + /** + * Memory pages that hold the records being sorted. The pages in this list are freed when + * spilling, although in principle we could recycle these pages across spills (on the other hand, + * this might not be necessary if we maintained a pool of re-usable pages in the TaskMemoryManager + * itself). + */ + private final LinkedList allocatedPages = new LinkedList<>(); + + /** Peak memory used by this sorter so far, in bytes. **/ + private long peakMemoryUsedBytes; + + // These variables are reset after spilling: + @Nullable private ShuffleInMemorySorter inMemSorter; + @Nullable private MemoryBlock currentPage = null; + private long pageCursor = -1; + private long spillTimes = 0L; + + private final long sortPushSpillSizeThreshold; + private final long sortPushSpillRecordThreshold; + + public CssShuffleExternalSorter( + TaskMemoryManager memoryManager, + TaskContext taskContext, + AsyncPushDataTaskManager taskManager, + long sortPushSpillSizeThreshold, + long sortPushSpillRecordThreshold, + int initialSize, + int partitionGroupPushBufferSize, + PartitionGroupManager partitionGroupManager, + SparkConf conf, + ShuffleWriteMetricsAdapter writeMetrics) { + super(memoryManager, + (int) Math.min(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES, memoryManager.pageSizeBytes()), + memoryManager.getTungstenMemoryMode()); + this.taskMemoryManager = memoryManager; + this.taskManager = taskManager; + this.taskContext = taskContext; + this.PARTITION_GROUP_PUSH_BUFFER_SIZE = partitionGroupPushBufferSize; + this.partitionGroupManager = partitionGroupManager; + this.writeMetrics = writeMetrics; + this.inMemSorter = new ShuffleInMemorySorter( + this, initialSize, conf.getBoolean("spark.shuffle.sort.useRadixSort", true)); + this.peakMemoryUsedBytes = getMemoryUsage(); + this.sortPushSpillSizeThreshold = sortPushSpillSizeThreshold; + this.sortPushSpillRecordThreshold = sortPushSpillRecordThreshold; + } + + private void mergeSpillToCss() { + + // This call performs the actual sort. + final ShuffleInMemorySorter.ShuffleSorterIterator sortedRecords = + inMemSorter.getSortedIterator(); + + // If there are no sorted records, so we don't need to create an empty spill file. + if (!sortedRecords.hasNext()) { + return; + } + + byte[] giantBuffer = null; + final byte[] writeBuffer = new byte[PARTITION_GROUP_PUSH_BUFFER_SIZE]; + int writeBufferOffset = 0; + long recordsWritten = 0L; + + List offsets = new ArrayList<>(); + List lengths = new ArrayList<>(); + List partitionIds = new ArrayList<>(); + + int currentPartition = -1; + int currentGroup = -1; + int currentPartitionStartOffset = 0; + final int uaoSize = UnsafeAlignedOffset.getUaoSize(); + + while (sortedRecords.hasNext()) { + sortedRecords.loadNext(); + final int partition = sortedRecords.packedRecordPointer.getPartitionId(); + final int group = partitionGroupManager.groupId(partition); + assert (partition >= currentPartition); + if (partition != currentPartition) { + // Record the start offset and partitionId of the current partition when the partition is switched + switchPartition( + writeBufferOffset, + offsets, + lengths, + partitionIds, + currentPartition, + currentPartitionStartOffset + ); + currentPartitionStartOffset = writeBufferOffset; + currentPartition = partition; + } + + if (group != currentGroup) { + // Switch the group and flush out the data of the old group + if (writeBufferOffset > 0) { + switchPartition( + writeBufferOffset, + offsets, + lengths, + partitionIds, + currentPartition, + currentPartitionStartOffset + ); + taskManager.safeDirectBatchPush( + partitionIds.stream().mapToInt(x -> x).toArray(), + writeBuffer, + offsets.stream().mapToInt(x -> x).toArray(), + lengths.stream().mapToInt(x -> x).toArray()); + resetMeta(offsets, lengths, partitionIds); + currentPartitionStartOffset = 0; + writeBufferOffset = 0; + } + currentGroup = group; + } + + final long recordPointer = sortedRecords.packedRecordPointer.getRecordPointer(); + final Object recordPage = taskMemoryManager.getPage(recordPointer); + final long recordOffsetInPage = taskMemoryManager.getOffsetInPage(recordPointer); + int dataRemaining = UnsafeAlignedOffset.getSize(recordPage, recordOffsetInPage); + long recordReadPosition = recordOffsetInPage + uaoSize; // skip over record length + if (dataRemaining > PARTITION_GROUP_PUSH_BUFFER_SIZE) { + // trigger direct push + // no need to update writeBufferOffset + // since we are using separate giantBuffer for directPush + if (giantBuffer == null || giantBuffer.length < dataRemaining) { + giantBuffer = new byte[dataRemaining]; + } + Platform.copyMemory(recordPage, recordReadPosition, + giantBuffer, Platform.BYTE_ARRAY_OFFSET, dataRemaining); + // update to new direct push api + taskManager.safeDirectBatchPush( + new int[] {currentPartition}, + giantBuffer, + new int[] {0}, + new int[] {dataRemaining}); + } else { + if (PARTITION_GROUP_PUSH_BUFFER_SIZE - writeBufferOffset < dataRemaining) { + switchPartition( + writeBufferOffset, + offsets, + lengths, + partitionIds, + currentPartition, + currentPartitionStartOffset + ); + // writeBuffer no enough space for current record + // trigger normal push and clear buffer + taskManager.safeDirectBatchPush( + partitionIds.stream().mapToInt(x -> x).toArray(), + writeBuffer, + offsets.stream().mapToInt(x -> x).toArray(), + lengths.stream().mapToInt(x -> x).toArray()); + resetMeta(offsets, lengths, partitionIds); + currentPartitionStartOffset = 0; + writeBufferOffset = 0; + } + + // just write into writeBuffer + Platform.copyMemory(recordPage, recordReadPosition, + writeBuffer, Platform.BYTE_ARRAY_OFFSET + writeBufferOffset, dataRemaining); + writeBufferOffset += dataRemaining; + } + recordsWritten += 1; + } + + if (writeBufferOffset > 0) { + switchPartition( + writeBufferOffset, + offsets, + lengths, + partitionIds, + currentPartition, + currentPartitionStartOffset); + taskManager.safeDirectBatchPush( + partitionIds.stream().mapToInt(x -> x).toArray(), + writeBuffer, + offsets.stream().mapToInt(x -> x).toArray(), + lengths.stream().mapToInt(x -> x).toArray()); + resetMeta(offsets, lengths, partitionIds); + currentPartitionStartOffset = 0; + writeBufferOffset = 0; + } + + writeMetrics.incRecordsWritten(recordsWritten); + spillTimes += 1; + } + + private void resetMeta(List offsets, List lengths, List partitionIds) { + offsets.clear(); + lengths.clear(); + partitionIds.clear(); + } + + private void switchPartition( + int writeBufferOffset, + List offsets, + List lengths, + List partitionIds, + int currentPartition, + int currentPartitionStartOffset) { + if (writeBufferOffset > currentPartitionStartOffset && + (partitionIds.size() == 0 || partitionIds.get(partitionIds.size() - 1) != currentPartition)) { + partitionIds.add(currentPartition); + offsets.add(currentPartitionStartOffset); + lengths.add(writeBufferOffset - currentPartitionStartOffset); + } + } + + /** + * Sort and spill the current records in response to memory pressure. + */ + @Override + public long spill(long size, MemoryConsumer trigger) throws IOException { + if (trigger != this || inMemSorter == null || inMemSorter.numRecords() == 0) { + return 0L; + } + + logger.info("Thread {} spilling sort data of {} to css ({} {} so far)", + Thread.currentThread().getId(), + Utils.bytesToString(getMemoryUsage()), + spillTimes, + spillTimes > 1 ? " times" : " time"); + + mergeSpillToCss(); + + final long spillSize = freeMemory(); + inMemSorter.reset(); + return spillSize; + } + + private long getMemoryUsage() { + long totalPageSize = 0; + for (MemoryBlock page : allocatedPages) { + totalPageSize += page.size(); + } + return ((inMemSorter == null) ? 0 : inMemSorter.getMemoryUsage()) + totalPageSize; + } + + private void updatePeakMemoryUsed() { + long mem = getMemoryUsage(); + if (mem > peakMemoryUsedBytes) { + peakMemoryUsedBytes = mem; + } + } + + /** + * Return the peak memory used so far, in bytes. + */ + public long getPeakMemoryUsedBytes() { + updatePeakMemoryUsed(); + return peakMemoryUsedBytes; + } + + private long freeMemory() { + updatePeakMemoryUsed(); + long memoryFreed = 0; + for (MemoryBlock block : allocatedPages) { + memoryFreed += block.size(); + freePage(block); + } + allocatedPages.clear(); + currentPage = null; + pageCursor = 0; + return memoryFreed; + } + + /** + * Force all memory and spill files to be deleted; called by shuffle error-handling code. + */ + public void cleanupResources() { + freeMemory(); + if (inMemSorter != null) { + inMemSorter.free(); + inMemSorter = null; + } + } + + /** + * Checks whether there is enough space to insert an additional record in to the sort pointer + * array and grows the array if additional space is required. If the required space cannot be + * obtained, then the in-memory data will be spilled to disk. + */ + private void growPointerArrayIfNecessary() throws IOException { + assert(inMemSorter != null); + if (!inMemSorter.hasSpaceForAnotherRecord()) { + long used = inMemSorter.getMemoryUsage(); + LongArray array; + try { + // could trigger spilling + array = allocateArray(used / 8 * 2); + } catch (TooLargePageException e) { + // The pointer array is too big to fix in a single page, spill. + spill(); + return; + } catch (SparkOutOfMemoryError e) { + // should have trigger spilling + if (!inMemSorter.hasSpaceForAnotherRecord()) { + logger.error("Unable to grow the pointer array"); + throw e; + } + return; + } + // check if spilling is triggered or not + if (inMemSorter.hasSpaceForAnotherRecord()) { + freeArray(array); + } else { + inMemSorter.expandPointerArray(array); + } + } + } + + /** + * Allocates more memory in order to insert an additional record. This will request additional + * memory from the memory manager and spill if the requested memory can not be obtained. + * + * @param required the required space in the data page, in bytes, including space for storing + * the record size. This must be less than or equal to the page size (records + * that exceed the page size are handled via a different code path which uses + * special overflow pages). + */ + private void acquireNewPageIfNecessary(int required) { + if (currentPage == null || + pageCursor + required > currentPage.getBaseOffset() + currentPage.size() ) { + // TODO: try to find space in previous pages + currentPage = allocatePage(required); + pageCursor = currentPage.getBaseOffset(); + allocatedPages.add(currentPage); + } + } + + /** + * Write a record to the shuffle sorter. + */ + public void insertRecord(Object recordBase, long recordOffset, int length, int partitionId) + throws IOException { + + // for tests + assert(inMemSorter != null); + + // record based spill + if (inMemSorter.numRecords() >= sortPushSpillRecordThreshold) { + logger.info("Spilling data because number of spilledRecords crossed the threshold " + + sortPushSpillRecordThreshold); + spill(); + } + + // in memory size based spill + if (getMemoryUsage() >= sortPushSpillSizeThreshold) { + logger.info("Spilling data because spilledSize crossed the threshold " + + Utils.bytesToString(sortPushSpillSizeThreshold)); + spill(); + } + + growPointerArrayIfNecessary(); + final int uaoSize = UnsafeAlignedOffset.getUaoSize(); + // Need 4 or 8 bytes to store the record length. + final int required = length + uaoSize; + acquireNewPageIfNecessary(required); + + assert(currentPage != null); + final Object base = currentPage.getBaseObject(); + final long recordAddress = taskMemoryManager.encodePageNumberAndOffset(currentPage, pageCursor); + UnsafeAlignedOffset.putSize(base, pageCursor, length); + pageCursor += uaoSize; + Platform.copyMemory(recordBase, recordOffset, base, pageCursor, length); + pageCursor += length; + inMemSorter.insertRecord(recordAddress, partitionId); + } + + public void close() throws IOException { + if (inMemSorter != null) { + spill(); + freeMemory(); + inMemSorter.free(); + inMemSorter = null; + } + } + +} diff --git a/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/PackedRecordPointer.java b/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/PackedRecordPointer.java new file mode 100644 index 0000000..3362e2b --- /dev/null +++ b/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/PackedRecordPointer.java @@ -0,0 +1,104 @@ +/* + * This file may have been modified by Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.shuffle.css.sort; + +/** + * Wrapper around an 8-byte word that holds a 24-bit partition number and 40-bit record pointer. + *

    + * Within the long, the data is laid out as follows: + *

    + *   [24 bit partition number][13 bit memory page number][27 bit offset in page]
    + * 
    + * This implies that the maximum addressable page size is 2^27 bits = 128 megabytes, assuming that + * our offsets in pages are not 8-byte-word-aligned. Since we have 2^13 pages (based off the + * 13-bit page numbers assigned by {@link org.apache.spark.memory.TaskMemoryManager}), this + * implies that we can address 2^13 * 128 megabytes = 1 terabyte of RAM per task. + *

    + * Assuming word-alignment would allow for a 1 gigabyte maximum page size, but we leave this + * optimization to future work as it will require more careful design to ensure that addresses are + * properly aligned (e.g. by padding records). + */ +final class PackedRecordPointer { + + static final int MAXIMUM_PAGE_SIZE_BYTES = 1 << 27; // 128 megabytes + + /** + * The maximum partition identifier that can be encoded. Note that partition ids start from 0. + */ + static final int MAXIMUM_PARTITION_ID = (1 << 24) - 1; // 16777215 + + /** + * The index of the first byte of the partition id, counting from the least significant byte. + */ + static final int PARTITION_ID_START_BYTE_INDEX = 5; + + /** + * The index of the last byte of the partition id, counting from the least significant byte. + */ + static final int PARTITION_ID_END_BYTE_INDEX = 7; + + /** Bit mask for the lower 40 bits of a long. */ + private static final long MASK_LONG_LOWER_40_BITS = (1L << 40) - 1; + + /** Bit mask for the upper 24 bits of a long */ + private static final long MASK_LONG_UPPER_24_BITS = ~MASK_LONG_LOWER_40_BITS; + + /** Bit mask for the lower 27 bits of a long. */ + private static final long MASK_LONG_LOWER_27_BITS = (1L << 27) - 1; + + /** Bit mask for the lower 51 bits of a long. */ + private static final long MASK_LONG_LOWER_51_BITS = (1L << 51) - 1; + + /** Bit mask for the upper 13 bits of a long */ + private static final long MASK_LONG_UPPER_13_BITS = ~MASK_LONG_LOWER_51_BITS; + + /** + * Pack a record address and partition id into a single word. + * + * @param recordPointer a record pointer encoded by TaskMemoryManager. + * @param partitionId a shuffle partition id (maximum value of 2^24). + * @return a packed pointer that can be decoded using the {@link PackedRecordPointer} class. + */ + public static long packPointer(long recordPointer, int partitionId) { + assert (partitionId <= MAXIMUM_PARTITION_ID); + // Note that without word alignment we can address 2^27 bytes = 128 megabytes per page. + // Also note that this relies on some internals of how TaskMemoryManager encodes its addresses. + final long pageNumber = (recordPointer & MASK_LONG_UPPER_13_BITS) >>> 24; + final long compressedAddress = pageNumber | (recordPointer & MASK_LONG_LOWER_27_BITS); + return (((long) partitionId) << 40) | compressedAddress; + } + + private long packedRecordPointer; + + public void set(long packedRecordPointer) { + this.packedRecordPointer = packedRecordPointer; + } + + public int getPartitionId() { + return (int) ((packedRecordPointer & MASK_LONG_UPPER_24_BITS) >>> 40); + } + + public long getRecordPointer() { + final long pageNumber = (packedRecordPointer << 24) & MASK_LONG_UPPER_13_BITS; + final long offsetInPage = packedRecordPointer & MASK_LONG_LOWER_27_BITS; + return pageNumber | offsetInPage; + } + +} diff --git a/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/ShuffleInMemorySorter.java b/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/ShuffleInMemorySorter.java new file mode 100644 index 0000000..078ca58 --- /dev/null +++ b/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/ShuffleInMemorySorter.java @@ -0,0 +1,205 @@ +/* + * This file may have been modified by Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.shuffle.css.sort; + +import java.util.Comparator; + +import org.apache.spark.memory.MemoryConsumer; +import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.array.LongArray; +import org.apache.spark.unsafe.memory.MemoryBlock; +import org.apache.spark.util.collection.Sorter; +import org.apache.spark.util.collection.unsafe.sort.RadixSort; + +final class ShuffleInMemorySorter { + + private static final class SortComparator implements Comparator { + @Override + public int compare(PackedRecordPointer left, PackedRecordPointer right) { + int leftId = left.getPartitionId(); + int rightId = right.getPartitionId(); + return leftId < rightId ? -1 : (leftId > rightId ? 1 : 0); + } + } + private static final SortComparator SORT_COMPARATOR = new SortComparator(); + + private final MemoryConsumer consumer; + + /** + * An array of record pointers and partition ids that have been encoded by + * {@link PackedRecordPointer}. The sort operates on this array instead of directly manipulating + * records. + * + * Only part of the array will be used to store the pointers, the rest part is preserved as + * temporary buffer for sorting. + */ + private LongArray array; + + /** + * Whether to use radix sort for sorting in-memory partition ids. Radix sort is much faster + * but requires additional memory to be reserved memory as pointers are added. + */ + private final boolean useRadixSort; + + /** + * The position in the pointer array where new records can be inserted. + */ + private int pos = 0; + + /** + * How many records could be inserted, because part of the array should be left for sorting. + */ + private int usableCapacity = 0; + + private final int initialSize; + + ShuffleInMemorySorter(MemoryConsumer consumer, int initialSize, boolean useRadixSort) { + this.consumer = consumer; + assert (initialSize > 0); + this.initialSize = initialSize; + this.useRadixSort = useRadixSort; + this.array = consumer.allocateArray(initialSize); + this.usableCapacity = getUsableCapacity(); + } + + private int getUsableCapacity() { + // Radix sort requires same amount of used memory as buffer, Tim sort requires + // half of the used memory as buffer. + return (int) (array.size() / (useRadixSort ? 2 : 1.5)); + } + + public void free() { + if (array != null) { + consumer.freeArray(array); + array = null; + } + } + + public int numRecords() { + return pos; + } + + public void reset() { + // Reset `pos` here so that `spill` triggered by the below `allocateArray` will be no-op. + pos = 0; + if (consumer != null) { + consumer.freeArray(array); + // As `array` has been released, we should set it to `null` to avoid accessing it before + // `allocateArray` returns. `usableCapacity` is also set to `0` to avoid any codes writing + // data to `ShuffleInMemorySorter` when `array` is `null` (e.g., in + // ShuffleExternalSorter.growPointerArrayIfNecessary, we may try to access + // `ShuffleInMemorySorter` when `allocateArray` throws SparkOutOfMemoryError). + array = null; + usableCapacity = 0; + array = consumer.allocateArray(initialSize); + usableCapacity = getUsableCapacity(); + } + } + + public void expandPointerArray(LongArray newArray) { + assert(newArray.size() > array.size()); + Platform.copyMemory( + array.getBaseObject(), + array.getBaseOffset(), + newArray.getBaseObject(), + newArray.getBaseOffset(), + pos * 8L + ); + consumer.freeArray(array); + array = newArray; + usableCapacity = getUsableCapacity(); + } + + public boolean hasSpaceForAnotherRecord() { + return pos < usableCapacity; + } + + public long getMemoryUsage() { + return array.size() * 8; + } + + /** + * Inserts a record to be sorted. + * + * @param recordPointer a pointer to the record, encoded by the task memory manager. Due to + * certain pointer compression techniques used by the sorter, the sort can + * only operate on pointers that point to locations in the first + * {@link PackedRecordPointer#MAXIMUM_PAGE_SIZE_BYTES} bytes of a data page. + * @param partitionId the partition id, which must be less than or equal to + * {@link PackedRecordPointer#MAXIMUM_PARTITION_ID}. + */ + public void insertRecord(long recordPointer, int partitionId) { + if (!hasSpaceForAnotherRecord()) { + throw new IllegalStateException("There is no space for new record"); + } + array.set(pos, PackedRecordPointer.packPointer(recordPointer, partitionId)); + pos++; + } + + /** + * An iterator-like class that's used instead of Java's Iterator in order to facilitate inlining. + */ + public static final class ShuffleSorterIterator { + + private final LongArray pointerArray; + private final int limit; + final PackedRecordPointer packedRecordPointer = new PackedRecordPointer(); + private int position = 0; + + ShuffleSorterIterator(int numRecords, LongArray pointerArray, int startingPosition) { + this.limit = numRecords + startingPosition; + this.pointerArray = pointerArray; + this.position = startingPosition; + } + + public boolean hasNext() { + return position < limit; + } + + public void loadNext() { + packedRecordPointer.set(pointerArray.get(position)); + position++; + } + } + + /** + * Return an iterator over record pointers in sorted order. + */ + public ShuffleSorterIterator getSortedIterator() { + int offset = 0; + if (useRadixSort) { + offset = RadixSort.sort( + array, pos, + PackedRecordPointer.PARTITION_ID_START_BYTE_INDEX, + PackedRecordPointer.PARTITION_ID_END_BYTE_INDEX, false, false); + } else { + MemoryBlock unused = new MemoryBlock( + array.getBaseObject(), + array.getBaseOffset() + pos * 8L, + (array.size() - pos) * 8L); + LongArray buffer = new LongArray(unused); + Sorter sorter = + new Sorter<>(new ShuffleSortDataFormat(buffer)); + + sorter.sort(array, 0, pos, SORT_COMPARATOR); + } + return new ShuffleSorterIterator(pos, array, offset); + } +} diff --git a/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/ShuffleSortDataFormat.java b/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/ShuffleSortDataFormat.java new file mode 100644 index 0000000..8a0410b --- /dev/null +++ b/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/ShuffleSortDataFormat.java @@ -0,0 +1,80 @@ +/* + * This file may have been modified by Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.shuffle.css.sort; + +import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.array.LongArray; +import org.apache.spark.util.collection.SortDataFormat; + +final class ShuffleSortDataFormat extends SortDataFormat { + + private final LongArray buffer; + + ShuffleSortDataFormat(LongArray buffer) { + this.buffer = buffer; + } + + @Override + public PackedRecordPointer getKey(LongArray data, int pos) { + // Since we re-use keys, this method shouldn't be called. + throw new UnsupportedOperationException(); + } + + @Override + public PackedRecordPointer newKey() { + return new PackedRecordPointer(); + } + + @Override + public PackedRecordPointer getKey(LongArray data, int pos, PackedRecordPointer reuse) { + reuse.set(data.get(pos)); + return reuse; + } + + @Override + public void swap(LongArray data, int pos0, int pos1) { + final long temp = data.get(pos0); + data.set(pos0, data.get(pos1)); + data.set(pos1, temp); + } + + @Override + public void copyElement(LongArray src, int srcPos, LongArray dst, int dstPos) { + dst.set(dstPos, src.get(srcPos)); + } + + @Override + public void copyRange(LongArray src, int srcPos, LongArray dst, int dstPos, int length) { + Platform.copyMemory( + src.getBaseObject(), + src.getBaseOffset() + srcPos * 8L, + dst.getBaseObject(), + dst.getBaseOffset() + dstPos * 8L, + length * 8L + ); + } + + @Override + public LongArray allocate(int length) { + assert (length <= buffer.size()) : + "the buffer is smaller than required: " + buffer.size() + " < " + length; + return buffer; + } +} diff --git a/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/SpillInfo.java b/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/SpillInfo.java new file mode 100644 index 0000000..a19d836 --- /dev/null +++ b/spark-shuffle-manager-3/src/main/java/org/apache/spark/shuffle/css/sort/SpillInfo.java @@ -0,0 +1,36 @@ +/* + * This file may have been modified by Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.shuffle.css.sort; + +import org.apache.spark.storage.TempShuffleBlockId; + +import java.io.File; + +public final class SpillInfo { + final long[] partitionLengths; + final File file; + final TempShuffleBlockId blockId; + + public SpillInfo(int numPartitions, File file, TempShuffleBlockId blockId) { + this.partitionLengths = new long[numPartitions]; + this.file = file; + this.blockId = blockId; + } +} diff --git a/spark-shuffle-manager-3/src/main/scala/org/apache/spark/shuffle/css/CssShuffleHandle.scala b/spark-shuffle-manager-3/src/main/scala/org/apache/spark/shuffle/css/CssShuffleHandle.scala new file mode 100644 index 0000000..26c35fd --- /dev/null +++ b/spark-shuffle-manager-3/src/main/scala/org/apache/spark/shuffle/css/CssShuffleHandle.scala @@ -0,0 +1,37 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.shuffle.css + +import java.util + +import com.bytedance.css.common.protocol.PartitionGroup + +import org.apache.spark.ShuffleDependency +import org.apache.spark.shuffle.BaseShuffleHandle + +// Add extra appId +class CssShuffleHandle[K, V]( + val appId: String, + val numMappers: Int, + shuffleId: Int, + val partitionGroups: util.List[PartitionGroup], + dependency: ShuffleDependency[K, V, V]) + extends BaseShuffleHandle(shuffleId, dependency) { +} diff --git a/spark-shuffle-manager-3/src/main/scala/org/apache/spark/shuffle/css/CssShuffleManager.scala b/spark-shuffle-manager-3/src/main/scala/org/apache/spark/shuffle/css/CssShuffleManager.scala new file mode 100644 index 0000000..ce762cc --- /dev/null +++ b/spark-shuffle-manager-3/src/main/scala/org/apache/spark/shuffle/css/CssShuffleManager.scala @@ -0,0 +1,236 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.shuffle.css + +import java.util + +import com.bytedance.css.api.CssShuffleContext +import com.bytedance.css.client.ShuffleClient +import com.bytedance.css.client.metrics.ClientSource +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.metrics.MetricsSystem +import com.bytedance.css.common.util.Utils + +import org.apache.spark._ +import org.apache.spark.internal.config._ +import org.apache.spark.shuffle._ +import org.apache.spark.shuffle.css.CssShuffleManager.getAppId + +class CssShuffleManager(conf: SparkConf) extends ShuffleManager with Logging { + + private lazy val cssConf = CssShuffleManager.fromSparkConf(conf) + + private lazy val cssShuffleClient: ShuffleClient = ShuffleClient.get(cssConf) + private lazy val maxPartitionsPerGroup = CssConf.maxPartitionsPerGroup(cssConf) + private lazy val cssClusterName = CssConf.clusterName(cssConf) + private var appId: Option[String] = None + @volatile private var metricsInitialized = false + + if (!CssConf.workerRegistryType(cssConf).equals("standalone")) { + if (conf.getOption("spark.executor.id").exists(_.equals(SparkContext.DRIVER_IDENTIFIER))) { + val cssParams = new util.HashMap[String, String]() + cssConf.getAll.foreach(kv => cssParams.put(kv._1, kv._2)) + val host = conf.get(DRIVER_HOST_ADDRESS) + logInfo("Spark Driver try to start css master if needed.") + CssShuffleContext.get.startMaster(host, 0, cssParams) + + // Set master host & port for CssShuffleManager + val cssMasterAddr = s"css://${host}:${CssShuffleContext.get().getMasterPort.toString}" + cssConf.set("css.master.address", cssMasterAddr) + conf.set("spark.css.master.address", cssMasterAddr) + + // initialize shuffle workers num + val estimatedWorkers: Int = if (conf.getBoolean("spark.dynamicAllocation.enabled", false)) { + (conf.get(DYN_ALLOCATION_MAX_EXECUTORS) / 50 + conf.get(DYN_ALLOCATION_MIN_EXECUTORS)) / 10 + } else { + conf.get(EXECUTOR_INSTANCES).getOrElse(0) / 20 + } + + val initWorkers = Math.max(2, estimatedWorkers) + require(initWorkers != 0, "initWorkers could not be zero.") + CssShuffleContext.get().allocateWorkerIfNeeded(initWorkers) + } + } + + override def registerShuffle[K, V, C]( + shuffleId: Int, + dependency: ShuffleDependency[K, V, C]): ShuffleHandle = { + + // enable heartbeat + appId = Some(getAppId(dependency.rdd.context)) + cssShuffleClient.registerApplication(appId.get) + + logInfo(s"Css RegisterShuffle in Spark Driver with shuffleId: $shuffleId") + + new CssShuffleHandle( + appId.get, + dependency.rdd.getNumPartitions, + shuffleId, + cssShuffleClient.registerPartitionGroup( + appId.get, shuffleId, + dependency.rdd.getNumPartitions, dependency.partitioner.numPartitions, maxPartitionsPerGroup), + dependency.asInstanceOf[ShuffleDependency[K, V, V]]) + } + + override def getWriter[K, V]( + handle: ShuffleHandle, + mapId: Long, + context: TaskContext, + metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { + val cssShuffleHandle = handle.asInstanceOf[CssShuffleHandle[K, V]] + initializeMetrics(cssShuffleHandle.appId) + + cssShuffleClient.applyShufflePartitionGroup(cssShuffleHandle.shuffleId, cssShuffleHandle.partitionGroups) + + val metricsAdapter = new ShuffleWriteMetricsAdapter() { + override def incBytesWritten(v: Long): Unit = metrics.incBytesWritten(v) + override def incRecordsWritten(v: Long): Unit = metrics.incRecordsWritten(v) + override def incWriteTime(v: Long): Unit = metrics.incWriteTime(v) + } + + handle match { + case h: BaseShuffleHandle[K@unchecked, V@unchecked, _] => + // In new fetch protocol of spark 3.0, mapId is the unique task id among application, which + // is different from spark 2.x, using context.partitionId() instead + new CssShuffleWriter(h, context.partitionId(), context, conf, cssConf, cssShuffleClient, metricsAdapter) + } + } + + override def getReader[K, C]( + handle: ShuffleHandle, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + val cssShuffleHandle = handle.asInstanceOf[CssShuffleHandle[K, C]] + initializeMetrics(cssShuffleHandle.appId) + + val metricsAdapter = new ShuffleReadMetricsAdapter() { + override def incRemoteBytesRead(v: Long): Unit = metrics.incRemoteBytesRead(v) + override def incFetchWaitTime(v: Long): Unit = metrics.incFetchWaitTime(v) + override def incRecordsRead(v: Long): Unit = metrics.incRecordsRead(v) + } + + new CssShuffleReader[K, C]( + cssShuffleHandle, 0, cssShuffleHandle.numMappers, + startPartition, endPartition, context, cssShuffleClient, metricsAdapter) + } + + + override def getReaderForRange[K, C]( + handle: ShuffleHandle, + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { + val cssShuffleHandle = handle.asInstanceOf[CssShuffleHandle[K, C]] + initializeMetrics(cssShuffleHandle.appId) + + val metricsAdapter = new ShuffleReadMetricsAdapter() { + override def incRemoteBytesRead(v: Long): Unit = metrics.incRemoteBytesRead(v) + override def incFetchWaitTime(v: Long): Unit = metrics.incFetchWaitTime(v) + override def incRecordsRead(v: Long): Unit = metrics.incRecordsRead(v) + } + + new CssShuffleReader[K, C]( + handle.asInstanceOf[CssShuffleHandle[K, C]], + startMapIndex, + endMapIndex, + startPartition, + endPartition, + context, + cssShuffleClient, + metricsAdapter + ) + } + + override def unregisterShuffle(shuffleId: Int): Boolean = { + // In community spark + // unregisterShuffle is called by all nodes include driver & executor to unregister with external + // shuffle service. + // But in CSS, we should only call unregisterShuffle once will be good. + appId match { + case Some(id) => + cssShuffleClient.unregisterShuffle(id, shuffleId, + SparkEnv.get.executorId == SparkContext.DRIVER_IDENTIFIER) + case None => + } + true + } + + // No need for CSS, because we deal with partition Resolver in ShuffleClient + override def shuffleBlockResolver: ShuffleBlockResolver = null + + override def stop(): Unit = { + if (SparkEnv.get.executorId != SparkContext.DRIVER_IDENTIFIER) { + Utils.tryLogNonFatalError(cssShuffleClient.shutDown()) + } else { + logInfo("Spark Driver try to stop css master if needed.") + Utils.tryLogNonFatalError(cssShuffleClient.shutDown()) + Utils.tryLogNonFatalError(CssShuffleContext.get.stopMaster()) + } + } + + def initializeMetrics(appId: String): Unit = { + if (!metricsInitialized) { + synchronized { + if (!metricsInitialized) { + cssConf.set("css.metrics.conf.*.sink.bytedance.prefix", "inf.spark") + val metricsSystem = MetricsSystem.createMetricsSystem(MetricsSystem.CLIENT, cssConf) + metricsSystem.registerSource(ClientSource.instance(cssClusterName, appId)) + metricsSystem.start() + metricsInitialized = true + } + } + } + } + +} + +object CssShuffleManager { + def getAppId(context: SparkContext): String = { + context.applicationAttemptId match { + case Some(id) => s"${context.applicationId}_$id" + case None => s"${context.applicationId}" + } + } + + // parsing spark.css.* into css.* to construct CssConf + def fromSparkConf(conf: SparkConf): CssConf = { + val tmpConf = new CssConf() + for ((key, value) <- conf.getAll) { + if (key.startsWith("spark.css.")) { + tmpConf.set(key.substring("spark.".length), value) + // if zk mode enable. + if (key.equals("spark.css.zookeeper.address")) { + tmpConf.set("css.worker.registry.type", "zookeeper") + } + } + } + // With CSS as Shuffle Service. Mark AQE localShuffleReader enabled = false. + conf.set("spark.sql.adaptive.localShuffleReader.enabled", "false") + + tmpConf + } + +} diff --git a/spark-shuffle-manager-3/src/main/scala/org/apache/spark/shuffle/css/CssShuffleReader.scala b/spark-shuffle-manager-3/src/main/scala/org/apache/spark/shuffle/css/CssShuffleReader.scala new file mode 100644 index 0000000..d04bc05 --- /dev/null +++ b/spark-shuffle-manager-3/src/main/scala/org/apache/spark/shuffle/css/CssShuffleReader.scala @@ -0,0 +1,131 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.shuffle.css + +import com.bytedance.css.client.{MetricsCallback, ShuffleClient} +import com.bytedance.css.client.stream.CssInputStream +import com.bytedance.css.common.internal.Logging + +import org.apache.spark.{InterruptibleIterator, TaskContext} +import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleReader} +import org.apache.spark.util.CompletionIterator +import org.apache.spark.util.collection.ExternalSorter + +// Refer to BlockStoreShuffleReader +class CssShuffleReader[K, C]( + handle: BaseShuffleHandle[K, _, C], + startMapIndex: Int, + endMapIndex: Int, + startPartition: Int, + endPartition: Int, + context: TaskContext, + val cssShuffleClient: ShuffleClient, + val readMetrics: ShuffleReadMetricsAdapter) extends ShuffleReader[K, C] with Logging { + + private val dep = handle.dependency + private val numMappers = handle.asInstanceOf[CssShuffleHandle[K, C]].numMappers + + override def read(): Iterator[Product2[K, C]] = { + + val serializerInstance = dep.serializer.newInstance() + + // Update the context task metrics via callback + val metricsCallback = new MetricsCallback { + override def incBytesRead(bytesRead: Long): Unit = + readMetrics.incRemoteBytesRead(bytesRead) + + override def incReadTime(time: Long): Unit = + readMetrics.incFetchWaitTime(time) + } + + val inputStreams = if (numMappers > 0) { + val start = System.currentTimeMillis() + val inputStream = cssShuffleClient.readPartitions( + handle.asInstanceOf[CssShuffleHandle[_, _]].appId, + handle.shuffleId, + (startPartition until endPartition).toArray, + startMapIndex, + endMapIndex) + metricsCallback.incReadTime(System.currentTimeMillis() - start) + inputStream.setCallback(metricsCallback) + context.addTaskCompletionListener[Unit](_ => inputStream.close()) + Seq(inputStream) + } else { + Seq(CssInputStream.empty()) + } + + val recordIter = inputStreams.toIterator.flatMap( + serializerInstance.deserializeStream(_).asKeyValueIterator + ) + + val metricIter = CompletionIterator[(Any, Any), Iterator[(Any, Any)]]( + recordIter.map { record => + readMetrics.incRecordsRead(1) + record + }, + context.taskMetrics().mergeShuffleReadMetrics()) + + // An interruptible iterator must be used here in order to support task cancellation + val interruptibleIter = new InterruptibleIterator[(Any, Any)](context, metricIter) + + val aggregatedIter: Iterator[Product2[K, C]] = if (dep.aggregator.isDefined) { + if (dep.mapSideCombine) { + // We are reading values that are already combined + val combinedKeyValuesIterator = interruptibleIter.asInstanceOf[Iterator[(K, C)]] + dep.aggregator.get.combineCombinersByKey(combinedKeyValuesIterator, context) + } else { + // We don't know the value type, but also don't care -- the dependency *should* + // have made sure its compatible w/ this aggregator, which will convert the value + // type to the combined type C + val keyValuesIterator = interruptibleIter.asInstanceOf[Iterator[(K, Nothing)]] + dep.aggregator.get.combineValuesByKey(keyValuesIterator, context) + } + } else { + interruptibleIter.asInstanceOf[Iterator[Product2[K, C]]] + } + + // Sort the output if there is a sort ordering defined. + val resultIter = dep.keyOrdering match { + case Some(keyOrd: Ordering[K]) => + // Create an ExternalSorter to sort the data. + val sorter = + new ExternalSorter[K, C, C](context, ordering = Some(keyOrd), serializer = dep.serializer) + sorter.insertAll(aggregatedIter) + context.taskMetrics().incMemoryBytesSpilled(sorter.memoryBytesSpilled) + context.taskMetrics().incDiskBytesSpilled(sorter.diskBytesSpilled) + context.taskMetrics().incPeakExecutionMemory(sorter.peakMemoryUsedBytes) + // Use completion callback to stop sorter if task was finished/cancelled. + context.addTaskCompletionListener[Unit](_ => { + sorter.stop() + }) + CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]](sorter.iterator, sorter.stop()) + case None => + aggregatedIter + } + + resultIter match { + case _: InterruptibleIterator[Product2[K, C]] => resultIter + case _ => + // Use another interruptible iterator here to support task cancellation as aggregator + // or(and) sorter may have consumed previous interruptible iterator. + new InterruptibleIterator[Product2[K, C]](context, resultIter) + } + } +} diff --git a/version_release.sh b/version_release.sh new file mode 100755 index 0000000..14e35ca --- /dev/null +++ b/version_release.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash + +# +# Copyright 2022 Bytedance Inc. +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +if test -z "$CSS_VERSION" +then + CSS_VERSION=$1 + if test -z "$CSS_VERSION" + then + echo "Usage: ./version_release.sh version" + exit 1 + fi +fi +mvn versions:set -DnewVersion=${CSS_VERSION} + +mvn clean package deploy -DskipTests + +git reset HEAD --hard +git clean -d -f

    LjdH@Pn+A>5%?wk!X;|Mh(on1YVkm9MYJb(P59`&g?%0G@*^oGj zZr&kN#y8Wx4bx`X*HQW7XeoZP*Nj%|HM=$R)`<4t(Y2*vUZ8GYTe8`%t<@X6w(hsD z57Vsd?sdO=v)=c$EnCCdaIG=;@QJ;2SOa@)Xuw_@HU@V{tc?csJ4gOb_BR2+=HjTi zw!2l`zP@TRUth1_xxT(t0%_e3$NFaD6s-Fc)<=1R^{VwW8}ub5OrfvaB^#u_+#oOg zWm`yprN-g@imjs$XDnylU$xWMrv_eq*o<3;@4MD7cFm?R@F@%$s~z}ii|x}4 z1{uxtwI*8w-<1LMshsD)cV)n?RF{2I25d@&T?5~hL1R}2zAJ;qt_)TU%?1VZnb@Fl z90u!#b%XVtdJ5}?FN4M@894L@1&a`*2Af922Aq<5!dnKx!B)+@23v*zgRP9C8f2?t zG=r@$=5Aw5XEcNUT0P;u;o6{|$F!OCeUw3C1_LAA1NN|*@X+^Z*x09`vyowApN7k} zykX<;41Jr1>_*bq%|xMAdX*Z}p=K|O2`q@ja)*aY>^K|O46{-LAxuz|xv z$Khd(!^53XJ>ikhtU-Bp97sD2AniDib{at1aUkt9k$K0Vw9|mbjzeRo0gaI}_|Zyc zBYQYgEp`Uo3eIrYu5NJofucLk@gTkdz-=xSDjGU+4n0EZDPg(mu-sj%+5GOB9qHYT`amr&^#%InrC|m7C1I6s)tzhAo$D1kE-$YO5BQT8Z(r+)8l_!cE)ZTfZLAi1!`A{C`1Yin z(MCPqhERZeG71>bjGrzOX;Ux!Nj30RwVeKD`I$mSN=bf<2 z%VHYKyUP{xTi#tUq}{FQ#q#c&-@R4~Xv@2%bn&Oc$K5QG>lNz>diJe6YWyf>yS)3P zIk2gU%e#Z>2=ZNwoj)11?Q7aQTiVbL&u&9$S>7G`!?kv@yz2*Y7s0#S<=q{>d#48T zt}%w?-BGP>E${C7OLrR+-t|#-%L%V6uaw#+UnO$#XOOqfzL(2dSvHXHCu0t)U3DwL z>e5#A;?*UitgB1AwXU-26l8UIB~OS^`a{E#ohKs1-?aMvl%|xz){zVt#kJaovF6CS zwcF3wr#UaBQ*28xb!!zn|EFF~c%@s`+(3hcMJAl;YCVA^-UV!zR%?V_T6JE#T96)Q zvb45oI>A~?ecUoWwT|jv;0Tr(yM4WoB{a&q+Z@&h`E1!$5=Ju_Sy~_Zm=z`9>RS1R zjWyG#H`eo}Gt|bDirS(i3d{IYZ7Sc5C2s80On1X&n~hP0G3b=W?KegZ(R*X$YV0Vl zMvL5Xv1AJt6(_8#nVZ;6t0sePtre$5cdeB~U}?*35?kx_gtj(J#@yP-Z?G|4cG=pj z$T^y3!JYV)&W_$U*o||v5uCqoHTe6M8#T5tYE&DwN}qFA<)(BPYnC}PinmF zI|%!LFy5^(2H6;8V-9Td`n3SF)ORrThqVshcPRA>C~1&^qsgF@WOO-0!tQcB8Q_GA z6SB*`+Cjs-IPk>|8r*1Uux`g-P;es~Wz+CzP^{K2`|<|G`LoN$C6)$FEEo*!7z~Q_ zX&p5d41Kl3;!tSJp)YT^*3jsOYc}CwV|_z6pAH+FIb3%osxW}r;J&S#uW#7E z@}aM9*f8AR10GFmGP)OvVOKRj^lq(QGoPOnFWRH3zxhNf4K07-S$c1?6z z+Fi~EfV+2B3vrTf~yMg@r$vx#;Ssge<&Ur-MJ4)#&S19?HJ< zKOtP07b%Gq_N~FsVd#RxQbTv%aDKjl5>`TO!!3my4WW2xi|lXLZ__>%=oySycl2gEDrfL-5}+J!ibYS@-<%}r;%M2cso9+zIv zJGpS9Yc%=Ctyy2FjEcLH^n;?DfApFM7p^>@Zz$2K`0yDK4($ws&rEeEqdJAF7tVBR ziFklhvnW`o?$p}RR&NT$IFN0t+6`&pp*56c&03iId1@W9rvFAj~Kc5m@ z@%;L6?}c}w%O&n(?+Y8h7}>nxgc%T3s-LL2^NCk`2bJR8-yc5ieNUherD|? zU$}?4{dQ>$R@25JILxCR(uNfbW#!7p6boJ;1Jf!qrHNYg3TnDfSTjYzZN6W9fcP$1 z?1=h*`y#lwyE}T68a~m(gF50?Ggl!l=L&H#_xG)n^9uI{@G;(ZoYO=!Qtoj$5G~6ucrNWnI}400w*|hQ_dl zDobaC`>_bhVa2-+NUE|1l#RD}mvA37L4ha{2g8h&?-N>(OP?p86HP#HNl~*>&}(A| z!9JYCuqnEwz+6($Cx>HI($;39#uA3dKCAtxA7}Mowo>J1IvxEKjPjJYDp9iRJ$~Hd zJKje7iyJQdRTTod<1HG3bD|=m7pP`DRAdRJ3BItV$%n(!83Ia1XBgeuxfo_dCd!E-?Bb|MdUw5vk0OV3Zy{s5DC(~4fpR^D{Z=JoP-icimdXI zZ=ZOfxCk$Vnud3IowkHJ`GskA6bVqz>{M=k@^3dMR3eBq=5Bf6Rqw&m2X@XM{5Srx zfALg##L7d9>|sL6MA{T{rKi)af?bGE+J%OWVcvwEsh5$_#gwTl@&Gm& zHC1savPEbN7b+TBp-@fW;?$!5Ah>ZVM7&L?ETK|JBE4zyDnOv3W0auKU~vASWezWH zXWYn}CRW_Fhsr|vaKV1%4IJsa6;R<_BkSO68=B}%ud8x}L>Zlz3+o;9AxE59V_MS@_YR?$r1&vO#fkdHcifU0}LJS>DzWLM7 z!=3(q|L1T1GAy#@@-Z>D)GeU7gjK%} zjkofBelOS)80#?w#V!f3b$}2n{57LL(|=AZ=X%?}`~pj0UbYf&%Ofcbg;*!jf{k{1 z8e#UEGpGdBvUG{}z#tHj#ByaY303{oB`iF|6tsWh|}%fp6*PnYIKJ*?ys-8Ui?YCzS_;J1b@weaf zmfJ~F+Hb%&-Me2y3zadhYu9wd(!)bZ<{J!OQ}-h2UkubVW;m| zbnj~)He`YHh!!1w()oZaEb&!0ZshikJW7LruRAAE zb1JIpm>N>HHH~Ok$SFoP+6qR5A4#=7!6-;^2&&0-B5{zmU>wC^Xe|;3^ogM!N`ryW zRkw9u`XfU<=YJ%n8$@SKxP@1OQl2nO%C#|a-_Bidk#5$S{KINaGA-Oj?%G>K3n*)Y zNv@sGxR%`NB|GhFp{8oypHJVzsuAMiiSX=@uxq9i7*B7bRY8s$BS%)!QV}CoP|>1= z6}s>{)=&XI!dtM3hYsQpx_YbnQWOgc;YEDFD{BqU{4ufw zwCLvCn-G#TGd_J3tM|pSkUZ&(2O%_Yo&rvp`B*w)O?nF5x6qf2vt(_`%5SUtt=H2! z%gR$-WNR42AFs0ZzepPUpmcRFmRnsITbu1P*ERR!s+e2?&9lXN-^NwA4rx_rvlc9na#CC+V2D5~s=k`yk_ znDn;G?xur3bONb9#!4zH-j_wsFxzm@Z=gk?v{xjvkKh(jgRk@_R&jH33a9nnnfAu3 zki4vpHgBjw#>qVgIg2+Z@rSt|uT96f?X z8h6mCYP}Q!9nFy5WC{Kp+e}JSKXBDR$LY>?nBS8RNnX^?BRqxpZl;IZ6*Lo&*&0L@ zgqC-rdcK3j9HCB{rh|gD9#9%F@MbY z6qmDoV>?mg4@vUMdr7JF&V3SK4rw8I3D>mRs0hN-bH1dPilxHg99EH>8qAyz-FcTk z-zT(HZ?eoPW^RZZ441@sHMu%}aNVSXH)oW~(l~J+gR?m79wiWqWKtFU;<^1d0 zQQ!$#@H{-6T&RT*R(A(%QfZ`Qc|%5$`f_{_KXb7$pWP@(}mvZ!n%~=XwzO@*uqLj(z|N37ByMD z4r{d8IsN4x_45`=k`qkdAet8Ek(KNj+!ivK8%<`WdHMipqUOBGOA`gDxxKt!HxeE(A z4j?O1ZM}x4kxxP|lLCy892b{f9_p$1EV=rgo{JrC(~+T8rSU2JVxdcDwq8Z)KXONn zuOt2z&!%~yaR}d-aU8Oh3=o1B&E!9(+l)B@gJN+}LJ?%l0O+i_e4)`JnDP7ZM~YM1 zPFTNkDL?j3hGjBWS2q}HjO*WA=%7dL7)if-c@nr1DuncT?E$kWpjhbAS}s&m+FFpA z6U7pRiOd$VGrc4zL$iSIiyKoHY(hn98F{!UGT?Y?yU?DD7AwcG2$-n2;~ZZlu8W4K&8E64zGDz_k?vl6w$;#S0~alkG(GJZWNK^=6Ni<}?}8 zgZf%(7P14t_ZI|qS*6u;9*!pF;NduMRB#;FG~$SH1y;vhEhv4H^m7suq6o)WurJAT z(iahJ4o%)aH_Ut}zb=JFQ0wrV{X0Z+6ivw$HYKI!++I`E`7`{nc0BEb6Zo z3mr0gN|*$e0y~fgnw;CHi6%klZVqccd;_2KC!Nd5;f>YEt%kRd2q7&Zl3H%|mU0l8 zj%(Ei^rAeDI)reZ29%-`z^&;y=#9YzNvdT}1D1k$3XRZ73=*%N#*^N|<(1Vnr~E2=%L}3s~hofJ&U!#Bj zHSFQ!$wY#ZeQ!s!qe?_(p0kIw*F0?o99i_e$v59DUGwOuAANlD!wX0K?QbVSwgDuJ zZ;?k+fL!4@+%hWzpi778AX-Nf=rmfM`n_)^Uw=LMdj7=K;WSsSCK8E^ zyBNW8D~C|io-k~34@OyeqncrGA-dewa<7jBhXhLce#AG$$SVX+T|;CuvG*JpsUT@V zb{<@4oRBFYaW2u*F;M@_`g+4Bx-C~$F{Ea{0%o4{d=8teC6c*m5_Sv7ddcv)k;@NX z>HEUrKN=;fn{+49X*8HZ|2rBqenQEH&Wie_h3IQGg*jRx_g~K+(Lfu(51SnM-xL3V{#7+0v(o+Ezm=uVr*Gl(Ciox24gJ- z#3E8&42KJeY#AH?!l$22KFK@5jj){kMh|s%J>lfJy+}+)VIllISiF{&ryoE2Oc}6G zUvksT`k(GUy*|13-5txhmXTmgjYI=s#r4T&{twvm>fp_EfBp5rK1{BH&!VT(qf>6Q zlc^cza77Y#(`g;te@?$XFoUd)u)WgggX!70uCVY|ba!s3eY!A`^F? zxC-scE2dvCAcP>Y1V0TqZt^(EeaH+vkb|3sX0w`uOgx-aZP`ferHY4KLrJ#HkQgwv z79zbHe()@KxqR8MRqXf!49B;NxZ=kY@qO~mWc$;}=O4EWMfgsI)7|@bEbG`#mKe#; zvlEAx(qzhpTFRTpI5IE})88Udb-eomJK21FXsRO!_$?be8Dd+SnuwOsc@3?&{>Za? z8bw=Ey*fAz?<6vetPAt(?*g3A(DP*4jUfGW`_=g|$cYO2!rT@obZZ1fdH#?DPI3U@ zVY`#pboL`*P>Q|*isA8f7cn%iewbcfwO!Llw(cK=wZ8jo1_C>F?a$1087s9jbwjA4 zQ@dy^A)#){(H2u6&q&n%c;lmuk1#`&Qx*d^BsB&LkPc_D{Lfgz%9gf;xE_$CSl*M5 zSC8zyobIU(zxW6UZfVs9+K^2bj`( zv8luM6SlA^bUhVUXnaaafn^}fweuH%N%5<2F1qsp{Y{77uq3xsExp3_QUHS$5>J;b zd0df>OHAR1-hrcW3OQ?$8}a9ZeX+w=Q_pZ?wdMq{udh8?V`G{aFsfpVW-+#}NCT$ZXD}mEcAbNm!_CJhA7OUn>9h!_ z4F^7-Ore1XH=45O9gsfy>cAI@%pbVd8hqglJ&x9x_6`A2Som>qEu!Cs24}%M{ z7%JKw_)3yJ*Ao{ZFiOI=J#@>9o`EZ9@&|;npdHLzPfWs%bHiZ0_=9)4{xwwI!ep~OLOoBOULllaqwlf0(% zxJL6cCM{G4!$#Ht03jZ8RU!n^o43es>JD?&@Zcvoda>@|ez9-6`9-c~r9Pg`2nH*R zCbCqy#`#gMt2kf6TC^UrkK|J^1;k~1S%*<0dFbk8rqMjt-1y~Y&F81TWa8``S#>V z1V_!TqY&#PGC9F>cT?kNh2dM`0226Asn!Y+9~|LV#E8asWr4~Fs)}!eJ)Ax7Da9m( zDC?8thDqctav4-A;iGa=0*{TKto?ZF!2trsg1t&?D(LH|Hct>8HJgNz@X* z_{=q>mLSRfcpg@9>Bya^buCu>Vw#8x2iX^LV%R;F>~67|h(}fR8`5WJS>2IZl^y{> zB!(5wm^*lnWuf;84O|ho6UNui)jfMA6?Cj69wve=#L;-bgU>LE!OI5m4JKSSc3RbA(_KJH&hTDxj(aI97sLT}coC&Z1v^C-5hme_W6z4p zHKiK91GFa8+sYD1t}dYr!Kc zD@2S839*FWX|O@`N=QktAxOhS0uA~KnVFBM&3W=l>O0<<+$qT2#7enT_xG@LT#NFb%zC3x81CCVF?o3Z;k}7c z7dTd-!O0BOh!1u`$V<_~i}M33$gFW!qXEK>5$Zr4H!;NRrU1T)q6%m=>YZ zNnWQZwq?e2KJ~d0Yzm*?lueMz#l13T4kIHCR zfQ9b|_KH|AwLmV3Dm_()DF@=%UrwYkyqwm*a32M4pAbsB9JK7pyvWyS!8KCfe0eAg z=n>(~8hx9~@R|J0>bC)RM^Vbo zSF$OJIfYztN0lz2G?fM?JylUtJUyJ47`Q5}=0y9Z3qz|ymZ)z&T1GZ<*NT!~H|b8_ z`s=8X8qF? z@%eanc6cy0Z@Gk}pQ+6A{(CD=bcZWgvB04XiUwb-KivECIYlC4lYX}$+`7+=y8W#0 zw7fYf2*4lp_$R{>75D||*+WPI z;r$2yHTeU2(-9uEc@Fl?`Hu(iZ8YsSZCcmm766mP3G`_9=iRX>w)bbx1}tYG219b( zdh>Gf?G+Ip5sXo>!3F4V`!B>2&g(aFCiXJwSa@_4JHw zNfMJ+ya&r%u4Wk0-@fBH%SbDlEMNuJIQ z@k)|ehde&P`)r!m$`c54L=-dw85`duhBW$qz)<66>nmz<+vMq<@1w49-`xrLGQW}g zUqAe1Q`}o7_|QWqjx}DJ*H4uS&{Oxx-3!9gzq3?R7t#s(FRj1OhRDu5eL1$Z%cARL zMbsSVmf(|8NGFe`(|sJZyo_!A1()C0LLX3*w1+m)u4^Iu9;s5`X!1Rs zszwp?Lh4ww!f#G?kN%$-w(5UjdQ5lE$5y(H)i^PWZFV?&f$O|qbkg2EyDk&{NbcTh zg!iAbqZe-vCjSS4gt2+Q1;q{rDo_idRW*!0ox&7~$JThnR0T!t*ip^#H&?-ckw|ke zmn9Z?cwEd$x#)UQnEIyf5#7A;iG0R#1g!u3;mwWxGg@%*qrTw!(4D*PnVdw4vm#-a z9@mTDYev?28_fZQnd0oa&?`;3s#`*ci>2&k|{&DwclVdvk z@c2}h5d)q0lv6+V8eQRhV92-;1uTC5xBvft zzYqeiKid0v1JwP6!z_KB>v}%r`j~&4OeUXx@^Sa?Wb)C?-{1J;BYwI0>8BrkL^|b- z$%NZyKK=CHCO4Lj`YZj2-LQKynf%+2P{`VRsf(mP=bEHXghbbWp4{BHsbb&HH*enDq>}6A z?{D3>d5dQ6FVA1hh&&%p{(a)f*QMK+NO|)3yRRRRYq_?*xrySjxv55mPx*oVsNtUe z3k{n$H}5^-x@_=hEB(TFa58==cQ8t;sLS>_JpIT#yh~q+JHQP} z(U$Bjy6I$1(CEg)M8Y$&_6RS$Txs2J4 zH18K5Jbd`Y2Y2s&(Y^QJ-d`qb?9spEFs)%EBrtn*@OKk}r+GZtcnyXu2h{G#i{D+d z&28`OJh=aKN1R&OmJ@F#`{?1$xBq$X(d4&J?)~|@`%mt1$>D^UiFy{TTj~2%Zx{Cu z6{8-#_{Xyu4))!nf2eYQouJ-7X3u^kKl55UfWtBxz_EbIy@!u?{;OLA-)f?Fonhjs z43n#~R%~qQ?K|rpuoynkUWOu%H8&ptAEvJr4ZW#yc5dK)?YB< z>j<1L;Wa2E~@74$Whf($CZA1U959}Yl^N+WMvs;e2w>Rwr8bA!;qX9bYV3`WE z^JLS9*c{(i$2(9-&W>+wZoYo~dLw$rU-W)zFSZD|S!hw8D(AZAwlKvab@`&v#X(JP z{wGi43Ym1{G7)_J7lL-nn?1+T`)B}IjfJdSbhGVagv!xB9hFdIIvuo`q;Z1elO&9pZ@yZO;Y(=L4R z64QPD)t{$^6fg>JplBT#T&KCOP`QJjD2wjj%D@KVREg@Wc4ZZ4&`8 z)~GH;=OP&Xjvm@+rjt;o_E&hzf%m~wmpl-fkQjLwIriF#XDwy+m@@J#i6fHF$`GW6 z;hps%WRBh?V|))6sB#K-ZP4W=Jf3n|N}nB^zuJ849iKLTi!?iBIcr-(u_lhA?jR&F zXms|do)~3cT6Pp1S^79jC{48Z0cZMwawSNyl|P*oV##}1mSrQ>XDy@N0g!_$7IuawKJybEQ( zpF#?hUC$tUbns!gBYmu{5rr_aKQ4|JGPPK(zJ zhozbokio8|tC69~JpC#K>=do+h#{77!<72yybDDCI>sItR$&_Ofat?VR}h{f%Gx5vm|iLe+PtC?z;!JLujemyuiGEUP+inS-*KhkGCFJHQYc3U;}aEvvx9PB$)H`sT+ z3Fjq&dP`|tR(?Mo-@sM7;VPPLwo-`%LQrDwj2k?*%yzPTbrSK9UYx4{MCeCMo@9A7 z8nYBa%k(2EVUtuWbR!Cm4poyAZoD6fW=%oNuWG#Uua5=!>4_}|xt!CRh_qQ%cVcEu zF5zJR%Qd_lTR~VrCw`fGYWnQxInRClh+6gbA^(sAOWw*9WWi=Xorb)=P4BDetF5O{ z(~~<-F-%|B?(=by+0_$T`M>@k9C)+5yP1#Ws1{jpW7XP|*aj(IxT#fJ)ZzZoSqVp! zK!iz#au8w>fS&w*p(!w@DZLtE0b+!e63-F9VoUJa=<;V!K2?m+vYd5(X-~U|Iwg=M zO@U$pX&=J-9We!-L^>Ob>2Ds<;nA4R=pm1IvxjPaDB=g4*ic z#M0}I$OLk98sgGm$cGO!XQG#9(nq%kCs`5Q$w51ll?mxmmjn(979G3xs8-ukqP^8% z2_OLH;Ft_GQwdm|dtj&qFAe;m>+r`Y2H>j)qv2xeJO@|l#$+ZzIpQD^x^EDN6W?UP^thz*-GJ+~ zSYX9<4mdA}99VGM3n5Ne8ETI;f-D<)q1H?MPDe77d9Ps9&Z#;|DV3JWUg^Z>ljz;LV)rd;O&zVAAKLWg#jvzv9RBM3#c{nsdO9>-0jVL zJ8%4HI72oc_A-xzrz`u`CO)X=;&c@QF3Lb9tnzya$bcHt4WVCd&h~q|8A^x)3z+lhff~ zFb3A`9f6>s&6cMd2T`H0ZP~bxMyl)`h}D*|tY({fmHI^?B|#PmM&Q|a@NIC?u;bCr z`aRm(BEvT7=$(*6WR*Lrx7c zqzXLeA~Np2%#PTPxa;e95l80 z`>Ol5^5o~V8pHh~e-Sw*5;)k{bC?gthf@W^@)^nmS?6;PX~QvQ<)%%omlxCSp-Y(- zZ--ZLtQ)W=f*l6V^Gy(%QDlYp6YG*|nw139f2%15JjjO6w*-|T1D)HqG0^b-?7|n1 zkBv2{i$r&EmzG=(}`S-%2>$TYiKs#u-XRoVSmE$ z(V0=q20Jnv(X>3Er}u_Jj?I0IGj)We!ou+%hYWv4ZJKRc*w~zE?p^)NrGfb@*f`~f zNti8RwMgL7;I$F6U${4$^R*5fj8T<-sK|5{$4Q5~9O6KBC<2Cfw1D~{MyX^Y$OI-o zr=ytM!vjq$ixN#3iVqa3N^V8xrvYm+j2d4tVk;r9@T{+4;ro7{0>&>>djf?{^D7*K#~x+ z_5u4D9!Q0Jf+g)IcxfI+u6^W)Vy7~Q=2pkiwD<&dG7Y@f>{?ja=`I=FG~+=potr!n zVHtjy4Wzz2_~xYm`&wLYW59x%rYGV^=@#Qm0v0(Bz?O*T^grH%BSj*Lv_PB&#Sjgf zmdn(4U*Y@#03OQ`gfrj}s=1Xg8=c$c%Mc;iydAX$CnqGkvPaSn^z@Xj>}G`eO|TrY zA7D%@$`PT$p>e3;57HgtUX|~cn5!`rmo1s2z?x5*M}f9*q(B94*WBKMz`|g2Z(HNQ zY+PyA#&MYR7C$jl9knAC?-kXsOAxACG9r&~8=L~i2y7)L&{eba!H%v0gw)(SRnhyD zaqyQJnYvAZ3`78`O@URxC{!7u6Rg~y5lWd(v_SV1aV|e;3~xcDH%(OtewM?`o$C(1 z!JlVB*g6*0whA8rSf_q4?-4wMPZjff@SHAKMbrqLJ!&WRI!%g({TMsXV|YD zL!3^e>bb?c3@w*Z(?J(<$eZh~HXzj)2~;e*f(>@NFZ7+ik%~OYW4YXZF7vN*WIxmDYhA&n-Ifx+r9%=r3?61Rnw+QF zhz;P}FsoheLG8tIxbzL2L~0P14M2fXn^HlC6ch!P#}$Le_2pzg#^6JKxgKUt&1Aw- zLt6b4f|8?8qsz-u7NW0ok`jHg*qyb8pv)g&h_<$hF5MWc;z2tmOkS8kES=^)c2u@# zvBG*?LdspM=~&eAdT&TL%W#L}f}~)sZMC1=J<>R8 z?z_fV9UZ}{9KYnP4pQJgwcBrW%cy5+d;lUnl3VPH-!kE!_2=4_{eu@07mR5Qi(hzj z$KUXO^A%$kQlH%y+TCTY`NG%c@=k+?msE?fbvz~YA_gU)Q;}Hj;qI&%k;LKj zP3JvS<1R(KfQkMxXG@aYx|qeOrGr{LEODFW3w+;4#Nd}y3(*TRUp4}A&`Pcf3a@6U z`+47tA0Up(-9zI9h%tPPDR?tDC|4Q2$Catg`HpEEoV$1aGW)n2;VQ&nAaNq<9Z44t;79Wx8NgG9GXHJ0?=e7kK1?c&l&%* zH;EOw4qrWD?(uA{b6zy(?}ihq%}tCBMQUNcWOzBAui|O#H>ElBi;GMxu{yInAg z&~DlI3PWJ`L0Cd$>pZRoNCe)53gtHEIq(&Q?$#}N6^1)BboXOxe=cub{=CYow{DG| znax-pGCbw0PJ3IvBk?aBOsJIago*X_cbMDq^vK${bqnz`gr%^>L0Ny4cN5!zSAM~4 z>$4M7(8K4Wvo*-HC!@{l^rC@h!liereQ85#Ipx8+CD4%lp@Lugr8uzy(@3ElR!0$w?qEJO7* z@~p7d@V_jl3gaEOZlU^d;pxd{g;hr+5{+(5iRKow>-u_vI@cW&B<|nmQE2WyWqwel=XlLp-<(FfCW-y z88Iw6nsVGGbSyTwNSazKN>iITQsy^D?`1e?w-g$bou1_(NbC=xGgw#qNbEdRHnVE5 za~-{G+#D+)$FW6*jg;8b%@9u&n3vv6gFZi`_w(wtFK^Pdvlr?DoN2y^(k)8aD0#9i zr(1rg3nr&d4Xt8y1^VeS z4A|AC)8nGFdmR@Ha{03?{*HBB4uSSwxaufg^(^&BzIb2CfK$d`mtPP5NG=Ly9q+Op zUjF%DcAmc4mDc3i?sA0hSjTfZe4R{N=zRR)5b6D-u@%qWqRT*8dDXeUoJ5-P#CZaS zbi__CSWVC$PbUY9owkem{onG(un*fG%Yys!?3Nm@C#R5Nglk4w^a`Z@L+V~Io^4uxKMP{Al z^p5Ma1+ns+tK%grYGs21VDn4XKj1avBDA#R=8RRonT|VMby?CEI;Y*O8s+M~7yAOB zWb&)$A9a6Sn5m7yZ7&!>P*y^V*>`<{XG2xQZFA()v+1$oUDB2#KI-sDGb&bPLV1@Q9&70i%)on$IXx1?b9z z^3eraoyYkR0oUd@>Igz@O*Jw=9%Iuj6{FM;g(Z7UgT~#Me8V!sHBBHtIfQwW@iV1f z)>>i-zJ?{XKc}KNf(Nu|nbpd1Fi-ZO1;cNj*6fJqfq z>&XN$AzCYV82>)wxNB>b1(~bOLotQ1h$ZxE>B9U5*VnW2L$Ee|V!mswCW&dtV)FY; z4p?=2ph-hrm61}1vupj*mW38?E9m}0cLq$3_g??rE*IyI5BYUe|=^VNH~etIKW%NplVBwef=MgVsRDavFzvS%+^2JG9p@j4R1yrNRD(5a zha_p=S=$=>wjYW14l}!1y4}#&Yqo1`W9&!DE9=6pY+mV>HnyE5oBWTZyDY)Gv1Mq( zOCMNa@8heDLmm6E+k1N?PvN81E`l2<$eLKPt+8R3Kxi_A@ai))@EM=*sjEpQmy9l~DqAYd&DJ7SZOA&WFVbwIphPu<;|nq-EmO{t zH|DJx4>EJZu>yJvE4xx*?bmNXYb~|gfF;!|T5p#aUz1>S?aDjbJ>d#s)m>!J9w7#B z$wYpBe!^_JJWT002BePpFM=!bfy(D4<^(aNC+(-$hfcfP+T!T!54WmIK8rd>U>ol*dxpwv(9sz^L!}&R z6^LfP7`UvSZP7uQHmE^_R06I&EtxJwSXBmb)u|`iSji*gm|Astqg;n-$7QRw-<(W> zz>ah$8PP9gT$tKWUPtM%;?K}+%6?GikgLsL$5(2()YO4AW{8 zO#w?Jm-;bj_02e1v<3Fv#!CWsQB4uoiu9da%o9;^#dhkbZ5;v*C?gS~1_8SmVllup z<~)jpV($6u9;eS3Vy8Raig)WQiO{)n4#!T>PR^jBvtrV@v&o!hsI(*p1cNII9j#mp z&0BIPGda2}n#iqr!`w0xDVy%OHQtfFjb`vLph{mCPE$o9;saV_E$h}I;D6$Qbi3o3 zntxbcu6#+B|MDRUT{Hu|q7$U>6#Q3~P~@|Sc(Ce}g(rWG#WWqlLd**(f)}0Ms@(HX zOd&66-$B^SiDaXHd(vc-DYst*pGY!h*pib4(m_h#av9q*Jt@?ddAS2tAVEl?%i-sE zPQ#&uR|65}LpXDxs$J>ES!r51x(r0;>17p~Cd+B?$3V%fNTYX|W|V7YrHO;vl23KB zINod6V5xz1MFiAQ*O0!Rx{P4QDrszKkf|$q50F&fCr>*K=^YId4N~-WeORcgZi6pR zqi}*;#qOi&eDIXU+IR$u+X4C3w(_r$haM-$*0#!@_84hJOYE4U!29YOh17S@s?tRP z?W~u6FRa|mmiF7FaCl^Dmq4JuY{io+w{BfoZPgvU)g0J=~QsNVBo_kEZc~Ak=|KH zuRK2`$ov%b83`k20Ih81n#xvoutHpOO81k~yZou@E&tXj9ewLae2n<2mQyDQt2JYiYBXyg|7C7|cEvBLn*~e%K{>Iu?zNvJ5qzo6@j?|7t z6$r9L*)I43FJ4h5nJr#ZJ|UnBME%hUbh2WWrK$nFth==LbC&?yyMHX(WP8HKoNWky zgRX!6M{?~vN>2yNRryJ_)p*|Agso`t#jkW74{FqM_dc=|MCHkg;p)Z(@;W2_;{YxR zIpujOd@N%j!QClbR?BAq=)fXWN9+{blmHdd7X&pbG@!sIOob}*jjm4;@zwDeMfMkB z^f0sGf_)+aJriQ3eRiAIDeMwKSZSZBDdu#8ZAz14xYhSr;I(*vf;vN*_J!#Qb6A=R z1>r=EkXIcF75h_Vuf>CFZAPwRd}mi4Rqa=1CCKO|_3~@tLa-+Lxkrap*i}OBpRnm? zUd-?gyH2uRxM#z5DcN!cB#-tlFGlEsUE_P!ddnk=5hR@KICoCOQLO^p6XH+>TvEy61Z;jdi8 zbSmCwGV=H|4RVFKB$Uv%b%aqqy8@bo#Cz8TR3u(-GTCe@A2bNc+#roM@>oNP*@hO1 zrq-WE7Bao&-RCrBto0)68vk@*s`<6 zRr>AE%61&;&7J$s7^NH&>ZpG)K8V8XdvqObIuzMNCVA? zHP-3VW`kwOM_T?PxJ$9F@{7*0xRMYa+79K1#Gv^Hl84XzV*2acxAoAL{IqbkBH-p; zF2AG?Pb)T#jJ|(Nt`sd`{)H4d3nvpg)z_-glvXsDI%X#6^eJ}5TVe-h z#?m9{Nit#wCX*?sD5rMRQo=#Yy@TW3Lu&wy>yj+N7v; zdLzqEOw0N`NBREni26E0cQ1BL`L6kJ9N`$KHN59Z*7kwL=vvVSU32Gj$aALKild~L zwtBJ$_1()rHt=<)C^9biPHt^`b3yuSnpnLqE-F4E*yR!pu2zaCb1b7Ylg=pk9_2UC}?#s7e1gdbiqVaA(zM#BjT#HOMe1#=HiEj zgGZXxM!(WnpFPP>`^u+#4xJQB)&vE}1 zq%vy95P)pp4{~-%9db0luyqxfH>J25G_$QtyDblm8Z(HqkliBO3IHh?mV&KQX4|ip zbO0G^=-~h?6~aOQA(e<*MgA#ig>k~A#FS(QiC1c+6PcRA?Nw0)XI*J_QMh1YMo<5X zP}5KffGof$K;jO3`xBw$idcYA&PiTdbGLWqfT9{5VxB8fs&kWTap16-$^9 zTc!8WA;{s_6ZF!L(Grc8UdC8XHXQR)dQrjlvwu=qYD!)8puhoN>-UGtjR$PSPKE~8 z+;WaXtV!LZ^XO~!1&>%dB2^=J>|e^?9)4Z$hY`jklQS8X&8tgeHH%hr4j=gvZkL7UUkbyU#0a2YK|wG7Dt zp+j2?Njp-|Qv@Y#Sslf8&rW9w)6{X(7W9}{iHCLe`y=M@04ArZ@L+z?+3)RU9@qleoKxUyLcTZs0F)hvEo zzAKOeLL?9UL883|a5MW^R4qcCAIwp#6URgVnla)-BfM!3C>b z?Ab1>tvc?a9 z0VF_M_sR>fZO#yM_YG$}m%3GkbZBOxl4CslFsO*s_rkI6S)Z-$qjOxgQ*(T5iN;+t z1+aDXsikXjUzOM++^%h|^mIn}>W2O8%9_+f*Ga7mkzA_MeL=hiEz5V&2!3|daH~+# ze8HiL761)lu4Owh_B%^ll&b)i-lX$%AZxO91W<=yf{zhNj|v?x3|uY5k41k8cOo_s z9asTkmo_%^Ps*->=3)^t=4rUpD!X%5iB!`xQc!94?8s7gC|Uvs5MObHjToZx zTTPYT&FgL-?<3x8jrmKNQ#jW6ObkISe$Ml(Fq z5+Qg-v%4Z{!}vXqsiBw$5TK z>LJ)?&$*f-Rx4yCPpxlUujlps!z;0BAG+3uI`8#te@&lT<1(4o!z7>D zG6J=>QK-}LsrX}ZCgJtSxGu3|4OD-I;e7V<6`C; zYJV**Ak>V@Lg%Yd@R$tAm`*yDF0*=_;SZ(mp-VZxsNKtDcVJUpP!k>=hZeI8qegw{ z%g^O^o9!rhFv6+IzVg`m3e(FU&L*lHwYAm~><8MPWZ#q0?4$#1rdazrmNYvl{udiK z7n^<`f1R1*hvndi)oqSKrJ`#Kv*HJ>LF+`Z@s<4w#5%sd%nq18xqfMml0~n!dUp78tDbL ze6@&d_ps*@wY=3??8f-9)jh6?R+pG^bji$P)fMx3w-@YxWAiID7ajwHgS7p%@bSWy z-gREV<%h52Bb)CpRCvZH&w|M>iB7aw5wxmFa(iV1WS3@leK`>PzN)~|rZ-X@W;p{Q3xUh(r_&$4LLcB_3>EQ8a(%T+1cT7^(b?R} zVGAzreAXPHBdZ1Z6G>S#GO~M~OSjZAd5%(vOsi1_fJZ!YRqFA9W zc=cMom#)yjmh$ERIUHcCjva{8!b21X#N<<|nFNH?d>iePsd$IWu_(Y-35%FLyEdCgrFW;azG1# z7&+O#6N{P51F?>4R!&(}kU-_+6{T?7m0a!4l@@#Q4K?<)l6^Q0c=gLIUcS9)e@#?2 z4Bv}9p73;c1LhGnyZ5ZRI=FfDD7S?aiTd@b*NGY})D131A+I-p9$qzLNCB-c=Tce> zC}xYHlcH3vE-~-?Tosa`_0eUY!BE`278+N!;A)wyaxND@h`;LsqJ9xak40yaMSzy|R=vI>;mnO*9g z)q$kAW|tIz|Gspl`o#R(t^2tQHr*fZmjumOadhekMNVStiRK|AA7-y#~T#Q zuXyaaZpE!reeB`ot(FG6RT27FSEu5YU$7g0Z+`XqzFX~;jT%#k)R2wltBdB@CJf(H z^3PA$JdpXthle&eO z)C21^DSg_GjYPHje9vsknRM|o4|fITc!X3;f-i)REj4|6;?MfOb_C1G`|-&E_cy-n zW@x_a(6OCp?zk6WK$$*BXOKMQ6Z@WYlihqCRy*rxs?}hI_W4H^2Bs;7{o8}X*V8`z zO{fHSP=nUC+E4wPl@jytIEjY#I%3LUCK`1qQRR~hrdP^tVx6Vz3um*NR?WU0+t4HM zzHUgLoH!NYjUJ|oFF#L}DCnS>Fyr=Etm(MRr*#vl-2qQMv@c10aV&i#gl@Z}?MRcP z5l8%GOVMSd44lD!d%I>-I^&QRjT8VP)Jm_3hEKT)OYy*FRm&Q5J%!4(3{y^HTjXBJ z$HKRyH0g(!kil{)rO4e!ftHyxNSu%40Jc;f7N>fBGE)~I%dh8^{Egl09vPQurwI;N z_MjbwG9MWN6=F4==2CcJ8}q@GRG>y&^T4Xy$I}Qib(Q^8m}s$rxclyE9zp|J&_3b6 z<&P~$LfEd55zyL6kPDh6z+18%npq}w5`SuRlK07~US(zv{l;bVj%s^rk?q7z#C<{ZjqRwYfdTL)4`O9iLi`2Y>;esoj zqhS$+CkGwZFOvK`=c_F{3nQ-5r3SCK&{l;1BTc2*F$v^cdIs&hI;KInk7_-pWzA5l zPcLI0;W9*6sjF7mTCHrjoug&SCHOI)aBT;GEey z%p`<0EHcM&vFFWY0dnjhL7FabcrnLP_iu^iM}vAyoaqG|?n3Mekh@ zprd(#(EvPq_|~oGvsrZsAs5B!9V^;=4xTD*t#s9KAnusTwA13ui!^8RRbA%*_I5cz z6DhVd9WDWY?&G&&m*_I1QNFNV>4*Rtn5rYIzkjvg*|lc8 zxveEI>fX}bA&bDbcJY?jh`{ZBtjli6{0csT5f#+Zc(Q~VzOD2mn^Qc#D&HkdOCVbB zex7Qn%U-HY|9dJjO+^q!>u4+dx&g+?rXSI)m}^_*PkU!_or<>o(H`ZC!w_pGM1#iuD;l8A6X$jct z`Y@x)x&!e4RQFq*t(-~yYB6H9R()sAx6K+abtbEcw*te0UKeR&p=>wG*w?e!Az_lC zt;|&tM!M#_m7bmIyolJ+qAWm9LJ*g~m_J9=ILeGiDZ(-&{MuDdqQ^VinvqiPA=XC! zfU%~kVYcSLv<)U{k@7B0U5c1ZZVF0SW3g-AIg2~9-%Y4$SLB;at?nzcdL+zZyTe5E zVDxqeOzMdmNOnuvCT6c^Q`wa>a)v`c0Cem)J@xKF59z4W`JS#d@ge%pHpn%(EbOi7 zXXqbGAAAn8U~Z`&D86-+?frdpg`cNBb^9qM9#WR{3*5=>tbwjt`5aymOaEvnP^!78 zJS`s3`;L!tiY{Fu&sHs>fpbiz&XTauWJ)(B^uV~bub=Yi#S zrr!*|_~7vDi}GD6%KCEf;s-D4bEaoHFVxrZBtU&?Euzgex8gR8fD#fcKUa`TYNjen zH|BJ3!Uft3pao-E3UN8sWBI?{?i)q&KJ_;+iGoIMoBCqSdDx(Wl^l79zwRcI60!#Y z96}+?rqvK!hm&{5P)0S}ic1{R8=`Z4&L_y>(ygE6k<;@%)yTO49>r3UmZq(WkK zVKO~MOz)Plm%(MDMXf^ki$707!m6@NAiK~Wz^v@n9A$!rmf_{C`VG6hR&Wi$X^Ap& z4;ZIj2u-)`6IhpoQiF@|Bws=JC_0fQW(8bMx=RzIZiutIqS1> ztm(F~igQ%RRZd0Q0o3nwEa41>BXC^rcq}anH}03!kheLOz+E_us$!`je+Sq);W}+q z1PqUN0UGqbBFH$QIaE#^aFjEQp|TDLuMl{xB~_g2$#@C8@FbtGiHo( z6bS{xKbxhjZUbk=5rL@ z=GdG%=)b@Dat)X9TE0`9qr;@ke1Z9{{`;HX_*>WFwOhuOrB9JJ86sk^P2M*Tx(aqP z@zX@Oywiqbb>Z18%0cHhC+GatzwOu*37_MWhTofjLQLth)ts&KlVZI{xGbM_#IyZt z^7hVDX%zSXbecGwrBW@5#!|Uh5+th?O$c+76Pi=Q*^&!@j`v4UnqS5U3R0|0qp`43 zeyl>{rsJosD{P7j4=}3jctXK9iZ?SUnN$nAOL3L3Ds04VLO>EML>&Ku*{hW}+%248 zp4X1!csku*DRq9QE5GcryM%`j|Si1JM6IC%Pp2uSG;n zadYFwa=2XvtJdd5DlJq148t>IF9>IN$!w4|ncv?`;2Y`_1X$E#!@c`f}|npG2UVaMA15 z#s;j&pV<1_-{>rmxzSD3ZIu{JX2$^)g)=fqZb4{c`)EI$SpJ9kuZn|v-g%y*=dhN} zHaqFVGRp@oUh_66Q(lQ>$LEF4|JGwj^XtZ&c+n2ZOl1n&!|u_3zlnCtWiyO!r++#2 z)PA{eV=Fi)mmE(!#mJ&3SHA2w`A?qhH@{%IQMeA*8<6e4UI^RPQA=)Kb&D%ftNGR>@U_bL&ui2AzA;RR#7Xcj?jHj`3URai zxfE%sNftY7`D=EVLnXAz&K#}POL2&WG_9J^aFV_}mOYp_t+HPB9NIf7;B#6;ou1Cf z4pJbcjkkT%i1yGNKALDDlMnr$n<FsCl`zmq8_a;jP|r5+#7*enfPk7sls4;2ypZ#%Wc667^i z%C8RQCWz_YfprD{L3tpa`9pBpA{E-_0H%`xwHc)ldbmnvu1O2Dd%^wd>H(#E;I^6ttgKszV z=Wjp%@WyAfSll99W7oKRyz$}2M{xSto?T4WZU0XXo?`Qxj>vY;Mb;#sxvTM~DO##) zlkrUfqzcpb=}_DvqqqfU$?d><9*HA8u{gDa+*JEzgiba^;74)PV7D|`+ST3Y&y6Pe zil#yh*X&Mn|1)tuX(IaeeWNr{~W|Pcv}Vcau$eqfM#btbg|UG}cju z!^~HuNi+lXMdpu~S(#mh1M)x+pcnOYN6jwle2DHhX(Rg&voyH zeaC21VyQ`!1^{`8mXcQz0Rd+EG>tU1#Ihh+%%#>qzr0fjfunS~aSPMWxq*={s9A_% z66%y^3QpKVo9au|NN%N|NVxoQQuq9kXN~U-EW#w$in8qluVyJW3%LT4nl^eeeMO5{ znwLhq1&;QF(h1|s9Iwv^$7+R8p{tbXw|j~Hd)j5}Pd?0z<#Z0K6jq(2935c&7mTpJ zbPemPEFy+jw2B2|v;sd_M;0av>CEqm*V+)!4C()oa4HA@EBOX3ONt z|Lo0x`5D%>=D+*Jv#$U0S&L=R`X1(&T7W+P(?ZaHcT`FY%byo*d4BkXfa{W<;=Q-u zhL|mHm2Yh3?wztIW!1(~f6et>fnNFpD%1S4W`yltcE(WzXcjt=#FDS9e7EGlOEPqy zrabV^Tp8eP8F?XC$|CKLi>jc3`-f(mL}aSkCQr|layG~@jV~1~$SaajlCh)AxJBAa zXIJK&^}(tkDsfNi(u!h{aVy!vnFyuOq&romG(N$ruZu6ZIt(afT+Xc%epg72Vm5B% zJ$i9Sl(r6-K4)ukNQe)fdQ1XPXnXNI@Rlq=GkC_PQDK`AqTNo8Tt*N}kM}00sHB+s zW;wn8-qq77+L1>NeTCmWx_{@}yZ2Zn4P7#4trItV$$0E}cXIO+nPu<4mxF6%KD39? z+V4rDEn6+Np@T>sZIwnW+ENbFuDuN|oK3#_pWpC5^GRc~?aDHF9;+)muZzQrA>)X9 zld$KgU*UPXQR#?!Zh7+IkmeH6KGiPpeD{U4ecmyw(lliUXx7%exkaPJV4*sOep#-V z+ukk5%0msHkR{2K87(}3mPYq}3SJeRolT}ktXsi7%;4c3v@S!Ny)%rCGYh^@`^rI4 zYvI`DGnIp=iexGnUSW^btBbKM6D@(SLqe7DWhe;L1cbF5tLHf9{&x!yDez-+d;Kv; zN2al*6wS_GSf}?kP^^JTF{M8mw)$BiNdr#KKaU)$=53W#dMv5_pCAfe2WBtZEOq+r zsR8bYe%no53OUVAc}pQkE(^<#Z2J}S>}cowP1eOQFyy1V>dURe(>JOxQ#8lzk<+}1 z*h1=32Ag#y+Ze{0%VKTVGg{tm)f z;FhwcBghAjVIq}1V6^I&S2*bUPV_og5O{z0*@1GUO@9qrxof(hQ>@x&hhw0$1*T95 z>ElP*gp7Hh6_w@T^t{!lH%pcMzL@325xUQ=T13-_=Oh$s*!_J573)>!p<`>>L?&_E zOCrwpTrp5}q39B+HS&%?<&)T-Hhv{ax9&=yuDGuP!p=P1p_7qBFW#%j<8It?A*5Ic zWVZY2*#TS)-Y0xqZCU9hUh3y1b%PP%5wIdBD{2MdqJHGUYE9@qkor9j+hJMqRVIIA z&%5-;mQH#3#nrH;igWgd{30AST%&4cm*v6_(%7r|n131UBh?^V)|FP>0bWtj-s}a~ z&5@e~Y>PryR88Sfz=7E5!pjB6?tjZz@!S#?Y|LxHb94Gsk!#HL?k87y6FfxpqC<+M zj^7nGy4I^f>rjUwKMZPm0{dq)8M9}e%E?tCJ$a3i+cn{mI~1A|2aR0ga?DEfA32^{ zy6_0?WQVLk`h~<_PS2vzo(L`|s1M$0t`Kuqrc)$w z0ME85u5}#Czn0@yleQ=p&>WkTOLT^%N4Fwr$sgaJWJ>^RTkWTy_rb|0W(;sd%3sGO zFTjxNV>~ATgS9T}*z@w)630s;j=~Hq-*jf6dP#7n_%-`4 z_jmC3-|W8|{$Ky+EBkQW{(I}zu)kKz0fa+4s0RQ%{xYBp49V_|aWY==Mjm@bfQ#n1 z;CPqJl}9@`HGcn);lC$Ai5ZH zDZY&`h*;*_FWp+N{5v$Z`f8UiB6)wM`XUS(^10GLJM!}uEnrkFy>g4&Jy##&&yI8U z5BZ60Y5gb))dL6s<`+HO+&g$y5htih?OW%*ga?n@eLMHq5i+rUIl8)`svT?WzwTB# zINdLPi^BetR-B7qVbfmZvK=(8H191=t0_(9s+-}C?jcAjnLP0RbG@s4)%u!l&jmi zU6<^YUKcI+fb5Nes+E6a`}dN~x2T?57t|QEVUM$C;d}{Y|PL!A#JLmbgpHSx%zj6fV{R|wO-k68C%8hP;tE;v;>G05L|w# zr*CoW#Jv`*Mw(8^Y0z!0a`9W|zi@*?1oYLgFpKhS4gPW`6WpqKW#)%v4^4EciD4>o zaYh=G9JByU_8wk+3tIMi)j;%L`@*Tb;=$r%w1T%G2rXkJ!x<2o%e3U7Ixg0NlfJTuF@NSs29EjUy{U68VLuz%Ic1he2Tx;6SbvW1>SQp?@dcy`TFrFdSG?7I z>^Xz+6+@;#jY71^5FE>-?^w|qkL7gXFy22~Pc-Es2h!&1?e;kLy6fVpsd9YGS)3bL zZYDEOEK1d`hflLcPI#``P+Hf`&)aQmZP7l>+38~!ZlUOa)FOgu)}f>0g<|S{;ga17 zPxittIa46MQbHz6X^n9jAhl%5yJ z_HK@gFpIOuB~8A2az9zeyFxwRZ;)$5UHDgHm#f>3%C@p!j9Y=O9qfKFWlCZmogW^K zp>A-46)@KbbP?*BBwHDCGwopq4irR`1Qc;%cz@B63NVD*qj;A$Ejd|E-|S6~t!klU zV~3t;5x~3y%)c{o!eLkYj5a2ZW?qq)F3F-b^5|9^4l;@YFajzG#1_LN8q#YBk3?gp zCd_+HI9Y`)Ffv2a0H$Fkm7XrEvAhBCIh#tRTQt>Qm-vJl+U$7vb7Z#`i!56$l_B6tlb5$x{lP|Br2cTg zt^Q*~`zs2O_KJfA{I=;q6G*276GK4XTscyA0JL)H0yVpep;A6$xCJt)=fDMGXpk3H z{B=)#S>L3?wR6ulqRY?ba>8phkBN^wD!e#5ndL~Y(pJ{ATz|#PPWN-3E=_0J?hrW! zg%LGlNloyhNi%w$+54;)`!jUf#oR*C>!WUHXF4ULtwRtm}|kyesQ znJIFWfkd?P3Ic#N5Sc;OTEO<>no1!!nQeTNK1KL zFC+;Ka|;p_S!N<#O3xm*P4@g$D71MtZrCpGvCsx&9Vx0Amn9x zIw5mGin|l~V4f3OIe2k2x#|a;wVBrtoo=UBe9gmMgiPlA^KNU)_LvK`%-gcXQ1=L% zjqTC(1aS?nYz+wcHcj`y2WBnlAYl4A?kmk9lS;Lc{tdS$`~~Y_=g)ig9go*Duo*4^ zvrw|)+<_oIiztIOPbHdHCf^NDX5n(-zjE|sM)2DbE3u_#oGA8uKrO{SrSi(PjilQ7 z4RPtj^sSnu3N(V(V2}cc9&V|r>KCpkgh>=ag3{edOJRAuHPl!cXrXpkwScq=%<*6DTNOeqbdI$y28UVd zEuNg5-UUek893spa&n55l=yQYgU3P_&jScip&l!-3z>(rK$_hnn)kEs=E} zBF?MEn8ZDNX==jMJyabRz`ZLJ`s>pNJH(b0HTan#47TCeEQ<&G3P$xu*Lht3_Q$42#W3TjtIMwq{dZa~4DNJY z_%hgH&};bC$M!S)ZrL7a+>rAu--w#tyY$m7dg{|w{3Z@9P)b8JhOc+08$2c*5v!rd z#ZZ31#K8p&4KRS9JV&ShR7m6`tAfFG>3Wc<^5!@uN}>h#7z}EzL#oJ2VL-mXsV;3X&rI2DU*K&{`L(1Zchj$!B%nS*2%ZbCE3kQx`1bOTRDKUyY?#URs^Jv1~q0=IYwb&p-cs^Tx-UH*T(7 zU)x+Ok;32#`Jx_cWuJF0Y4E;|Z2HH)=E!ZTi%uNiCXS>}bsWBS;$gcPZ0#}7M9mPn z>AjpdVN{NQx|YnG2(IF$?1BSl94jgeA5p&IO8WSMN>|tBz!zcL_#JCy=j#Tlm5-JnKVOJIiDj!H(E{`V8-e*yFm~U%L`Q^oAM zIKtd{B5ut3@m3ZYAaBy*0(Ts<8ge!*>~MPYf(D&6i7uW-4>_r5cmWD<_#b}s@$c8J ze|Y_)A(@9tw8jJ3Qy9;8=#Bj>sk@cMSb(*#3-PpqbuMS$ZR1kZg!h=Z8US6-*d^aa z$lNJt!I>8#s6tPeu$Z52{@(FDVckaahH@N8A(=>|8%OHr0{y2 zqC3j<&t4+s18lXfB8d=IN-ht8r^Mw}9i5uVPbW9s5ixu+@uz$CL+LS83}ZBynrh6u z>guv94xT?pk#^%WyR=MdlNQc6FLvYLfoga{c3vll2$oZmF=Q)xXj?J zb(L5ZH|AAnyO=V4at&GqKh)Sne7Y7s+ar(CkHe9Tjw~rlnggM{nWZU!Q!q@yU=0Bqq0vZ;@%gaPS+~ zCvyz741Dv$WGPi|f?y33;A7?%dkdJKQ@ zJd1YGdYjsu9_QNYE^!BHNaz;5YT+-eT~4(y#u<~e^wHYlJ{xSJ<8P@actom`JD8`v z*ue;J5#Dp9MZO>L194W;y*)I;hbG}OjTDmw?$dfryu}&(FAD^KaWZaXjok+vC66 zU*DOtA44fohpUvSUvFoZ(1LcT>S!fgNLt!1U9ek&6+xW(0=wk>u8-7Uk1T~PHoVj^ zZ*sN0Tzrvhfcr4i8>U_mu>fM%Os^-3P@E2Sj^>-Fi%p=<*_n3vq-dOmhqSv^YV4RF zs5yZ3_1)dw^>sf4{WEq73Hr6|?d_DPmI>T~+?GsKNrn^aWe?Uz+Y_|XFc~0i>{j^Y zd#U?|z?C_7a8zo+#1W?|;A$jdK>-* z)$|Om<`Y?9E7CA)QPe}dte!uoQ;+TBbar@dbqG?)E!E4#uJwZEoN47#8*{PSZDJYf zMl?Y*+cQ!Sy!KDcdSEaXqDXaLq3BFou?WgwiaDceQ4Y-4ewFdXC2Q&Q4`CTqYx!;K z^<6FXCEgt6ilL!+NUER~_BG-#-jkUUPQ9i1FQp)kQ?jriwv7aF-uI-!6vcz>p5kDC zBwomF%pkF6VSp$T%iI|6``(m~ET3t1v~JGfMapqWR$VxZ?%*-UkrvKy${|=d+OHG1 zOrQ&KSBYE>m$OJMhi(Qs-xxyWcNc1U%WEU^_!Z|YNM#W}#eK{lI(I9zHZtcF*FX!G zy<8Vis5uFj0?Q!bs(|WZ($P6BHuX-EMw~q78ku!T+)JJO7S0L~hfZ78zE~B~U>9y* zRVKEeh>A-qvHR^_;(jtgOQlV8G)!kptNw{G6bB7jPs+GAuMThd0z7D4k_O>2ZtSG) z%mVmE?EG0XktJHxncFE$nJF0M5(<{SgY;_#&{Oi?G?p0C6lqvAw9YdXG3|cs$hea6 zXbA(ofu|r;abI=wc}c~E5FvSl=8LSpUCM=TaZ2{m(G*2R!CBOVQyT6M4#`sjdD1H6 zfHOgPJCTJ4RCEDX2T||TWU>K5i1;e6{hTYtr~U1S9@evuX8kIyb7z8d<c z+_H-G?iw!D6;2XYzSOK1@{v!i`*!QLo|MSoC;4^;FW!6PFAlz*9<86txnD2wvH?`& z&lr8Vzh?`6lX}>q^x^O?^@>wEofecQuBJozAfqf5ID7hVC+%E?3K!JM8dT`qZmf9Y zVY6PcX+rD1aGlKUO_5c3rb%k-p6MIF7)ZNBN^(7{jh_$MI7&dC*d#wi9I4GpqXU!E zhiD<0($(&MI#Et}#k|_++(I%8~;M1x~GsJ?Kt<^|Nq zYO!?7U7|~-w&e_Fu+5t!3Wm<^VsZe7_unHMctl5aEoQ6NQ{6c0)>ZUR(BpzQfbVP+ z{mcob28VU*G~KBlFcp&V<=t|RXR^nl`;4Rt%JlR*!C_bLo`e_J|Es97|BG_)iey{D z08U>T50+(0YeJR1h?1~HjWL9Ysf;{}RMP2tH=Gk<^#TqhIo{XJK*j&`pK%$2OHLGLnX^a|oOo=4bXtue zZdOMLjciZmFmyVaSGa4Lg*2164{J(W+5~=M?Jof;g-RVj&yOMiST()jZmi%axI+Ue zjzJCF%~xRJU66Ep|MV@nW^Zi$gO-ht=Go(%p>O(k^kCukq=EA8d+RdyvW5bow^Vy) zuu)P^Ak7NCw{GM$8Z;J{FE9BC^m@$ryGTuTMi$nIy-rb2G%tbKmzv zF|!-h?O1cT5M#}KfAKEL^m335mlTyyEg;4qQKoIi=)M~)V{D9yxITy~GdeQvl2q1j zD5(L|pg(?I))$Md$57NHSi%%bXX_gzJMUmasYRIGY+!j6HWKd@4h#KN_FCA~VpI;c zBUDZQi2g1kx)_CFvuyCz(pk{@>NJgD9Wuw7Yd6SZ8+O4si!O6_6MTO}&*s+Ne^SxW{fswaNrG%za<_@=-iL zdmR$mW;NHvqA`3BdrnA?U{fCj(p|8KtSARal{`=icGwIAjxCHiiivTq3r_kXX?G|I zUx{MU;OO+_!LdtF#`lTBQqh{Ih%1%b>PQ!L50S(6*ZYsY`4)MvrR4^#bp5uJDDn+B zU!t(EW5m-L4>m$%tKBl7bePJaF0WBUEOaS2d)6A#G~>1R-?JVIx-IZ@@`Qlpk~?M| zZ9lyCbTIP{h^Hgc(3XvFx^A1zJ%wy;}NCbnP79ZReriGR2EXJhz>ny!vwO zsT*m3**%ow`(S@0KraTQmI62q)pAYID=o(Ok9*oYaDbMVE>H0x6kJU)(jhHw01YM5 zR7I3}4Gtw}Ho3~|b2(^dcCC&m1Q$h>L;I;>;xbe?Fs-&rFPd*%c(n*GOk(CaAXn8;AT5Uk86QLcIIKo_urX z!;e1ygae4Ht6ohpc`41a!6Lo*`N{4JYk2B66FonWn@O@aCUgZI~8?2IvBWN*C3Tt zZ}6l8hOkCYpfK!W%_%}_$kaWgh!gCrte_RJh0KDY(&rLA+=BY<&h~Jrz;V9y&Ta_D z;IjbH8YXanTr^|sHHTnrrR^K^m1rn^h;0b;#=kfSeE_04NuNOziiQoQXWe0=*TbQh zSr&#D`d)NVyg{W5(UR{qqr0Aj0JezvrwaowEAo11!>M#;y_gXtHvaSS;BX%*7H84GWi<{8`!6h{8;3o`ujcMx zCnX0+ObqC7*PB%?uAl9i>n#Cxb$*XTbz*Zkv33CLK&>z~STQYjqLhcSDx01>3cCx; zz-{dr*;OnjpB%EhzzyMSAdXq@LpTkW zgH);PWWg?wsIJYNNO?J(4jU#0u|4r$CwGG$-hb~~r#dIE{JBR5pU#wQooLJ;AkVX` zd^2c#D?hgc3u~1kKBrk6Z#^kWtet<4f6k}^31RrzYSo)^8is73nL`2S7TV|J6!JyM z4qU&bFo1judQ^ubHZG3rRPniy*``viUQM`W}}qk-Fua^e0YShop&Beb7qpq`L*OkLzea~(ABFxqw&0K5Uem>dtznZsdrT! z^mL5{*yisRy_D-kb{N<<)I6FvD!2yVIeONhTs-G>d18JnZsEfKQsXu8twj%>cYtCTU5> z%mh7~#0YX^L_{j7A4e>ptS(tqIp)xIc?0Y!K&tu_La!Qdrp1|E9HxXVqCTv!M|qkd zrS$pZp;0(D4mrwuhcn%NMxBt|Vy4|-sLV^#*jTHE2pq|%ib~fSSIhcZGZDUq0)sd) zxj}Gr)IJ1SPtH;Gw^b%Fv@@k8>l#KB&<0CseOpo1v0nT%ky}d*?Eu*2fjjpd0;O)N zpjh(Q`Mq>X@4`m7b&BfvvSb6JlQ0vSav{Y?X+^;8DuQCd9yZTUJ- zCASRvZlgdZq8rgGZ3s^FuA38vnguPh)Zu|@>P+%7t^0z~Vrd8qd{qY+Kc(qns81f^ zSNc)y1R;&`QQC&k;i}#z&*CboirGfKjsl}A-0dfJdwI);#2PaL!HtkhwI$7j zE(%)Jh*5+NN%_%}CuLV@?(OOtTy!1vo2{lwMY;4$+>iXN*;!H0w~vou7mU2U@sUWS zvsHm8eA25Fu*~;8xi>-`o@O>!fF*=yW-Vdob?3Ili~kJQjEmb&+F7 z6v!6cwnt`<{G}tP$5w$BlUZOZ&Fxm9R3do`{QRI!cev!BjXoS0ymUWkSg>dB@@Qw& zHPkgJUW5qa4K)a9ZsMvf0b{6ykO(bV&9fH!4;_t?tt!nLU$oQ@r7j+wp)B5$4Wg!j z;nJ|vG^b6)G}>e@L6f?cH!2G0vT0CpJTA2ZR^QJ5iA|%4&v$wu-3GQOd#5 zf<$_(!1TYbL@I2~Snv{|ggW(+iHUk&MiCL;<=q1V`LI3ESyf^X4vn}npt_bOSMT4| z^&1uzaj<2QW8iZonhZbE+KmWC<;-_??#k^YDo^^je>aGZp&!8u@Z`jn%?&_9x%54h z#p%FgA!M^M?hWDP`$hy}@PX8zE3Sr_Isq*iRoWO^mX|BXZLVsF92B-o6wxmd&x!#( zUDhUIJvbsA>|pryVA5h_R>guyQ^|v?6j0c-RQ0lefeJDK z(#85pfX=&f%;@S599Vh;#OASVG3g>7x%YZZEh)!7R7|!D6CF`a?&)r+giNbmlvc*{ zG$3LnqAla-IT{HA;7Z0p!E>@V?7ywnY=ga+su$LjNh@Nda}yBalXfJ&KKL;W->UX= z@L-tFk{g;ot)v=n#Q-C$odR*Rr!`ql(Ga5F=QHxDg{9q+!itI6=jc#geF&}fEH8HJ z-*Y{(xBds9DNcay4j1aaRLZq zqok9d9cAorFh5LMf0@dM$1`Ce)Z=ozue0g^_GC!tY|;?Xc5IoTxLK*`dAS8JZg1LO z^~IM{S}=xCE-*57dJfT-wYwS3EEQ2=?P}=wRsvjyEiURVYrAkQnQ1PcekR;e4VY%< zFRXi`&&|4e(9Mzh@z!>#h+;>D@I+)>Vo&(14!{zKiWKF$Jd8khUR=H*&r`9EkOk5s zbbxs#k_PGVHMXV12!m`YtKI7iB*M+MCjA|@N)|Rg%d4lbkkJk=o_X zgUMxIqZQh@Hp`18*HxPHh#T8bf~7OiGUg|jrlk?GOA zd%v@xLsOVHpUc?n6s?SU3(P9vuB>2oVu@6!U-h|gBio5WtsKqi9tKefgr)9twKV>B zq3p~a1vbTj50uueh9a;ZnpYu^D2n04-X$^g8i=l}Q(K)cBWbd32$!s$ShANOx!4&t zz4OO+@hXUAs%>GN-r)Ylq^CtS-Zqk;xl!-n%%Gd&wE5ZqQ(wnoHqVUJ{LgBUQH@`H+*HGSKUX|Q?YcM4i zH0PN(-b1{x%#PHd&^!fSxR-4HCW8gjNN8mxe9}({$U5EVOrte+XS6K^sg;;$b*d65 zUT3e|?w~ElVJE`z^+q+Q*31G6{gy~b<&341KrWtF#Cg#ea$9Mu7<}LYqoasgg@mSV z1QvDMPTo-uNvqQ~l1^{om(OqR+Xhz*ssi za{p!#Gg*Ppx1y%0VC|Euw})IH;TOS;YOJkhZ+Svy6MGYA3~oh+1Y^Hy<~Iap9cUt8 zHCzop2jFcXebn%psqW1y%7W#MxSCBCP&HD1Y0PHGHZ)PcFU_9P$}jojmO3F?1Q#op z2x8ElUEj%EGVuGX_~SF~H2F~#42K6lp((Vhu-%#>ngZgohgG3dUFwj5a$-u<=5YI)3k zmSW-@GJPDy3^$*i6O;v!>vrW~th0A({Ac0+_;gt9TodtPdbR+S4TtmUk#~+aHhb+L z?FSNwjFM}C4k7jUVmZ3Q!`W+2)k)e{&Jjt^e~aA8zLDrL_uXg747!44@9^BNG{I?3 zast;iO0`dh`yU^?I72WP!>&h^%yWn3NRMvLG1Z7gc!uwnw{TL>PR+CkM@v-f1^Y7f zXHMM)5U#>EQ``DF#u~&jB%3>tH?^EjA`(6%jQDzv5b|+hex&Xu>KiXw3G)TeWEe7u zM(Sr;i?vHnQqhCzq|5{bK4x3#V>nv!*Lvz^y6_;HyI*zd^6!JgvoC(LUi!y`Y*nGC zUhrqu*GCaUv|Hx^t%^13+St42236#%?67BsJ4!?tIntUtVATRp)C;N@k1ggb_OI83 zWXb~HMx~@^7A+ftdr$VZ-Z7s?G3ti{Sg5d!g6Fvo8ELZ4wxI6jdapvRQqhn(dtCel zbQMANW5Hu=?7EEX1_-$P?yRPa#adiK_8S!hn$?G_w&lkSZDu8vmn=O{5UQthyk$0L zdBiMmneLN2cei)&7eu+F<_|XxHb#BiMiY-v)_jAHKiKO(2si)sw+8oQdkWe83f&s+ z_Kc~V!k&;8&0I0)?yKW37%X|)Wb!r63-{1Q&8k+7tB2ztEvhah)z7+S+g^cfu1Nk3 zJ2UNsftKwbWkXA>ll&(3o0GxElO^}tx2Qri`!sIvNqZ)Gy^6>xZj6Boc+G%0ogk54 z$8gqZQ;o)Q(hquY*-Q-}+aZ9Su5hM9)8X5S7b!BnSL#DcBBm)s;7saCg+Z>V0NSwj z^~u2**+mmB8P;hceoTxkd#`A1M4WaZd@|{{&_TLu-j74KkILHydsUaT5R-aFN}(Y} zcb;>y=XmQ*@(}q)boE1WM$lng*2&gxVybyl9!PDXnwRLhEHPa<)&r`;)|Gj8^upXQ z#bapBFiKnCUFN}})5+n%-xZymO@8}kTcB%WCWd|z<>Ox5EC+u0U{Gn>vWtH%uNT|* z+HSFZ{pHE6cf5L7HfFE#dWA83MyY0Y%iQq^S96{m5Y%J$xKfCeU4>-gyE{ouIlO z{LVR&g=qQVt01@!^3#TA*N_*Y0f!0r`o%fuHn zWi-U&=~+yGtTf-KI&*PR#oFYj>Dw>YvQO9AmTaPfGfHx$0^K`=DRKX#ryP=+XrE_C z3YOnH;+pcQ-6ZJw88*gKBH8=9T(Bcf@imGu$Pd-+ynL19^#O{$MZi zIq-=;dU}5|2H2mluJxOrfA)Kz#pxanR-RRyzViIEwX-q#nuJIewEkd-hYpd>kqj?Qc0A^7+*9we8L9+7sBwq4p52D-0X4{i}c#9$sbhtrb{@VTJNB; z===9}zL_xpL`9_=lc#c2<0v5W4Q&5%?d93o@vY6x*RNl198AxiZ_G|!Y@R&d)BoEG zXK&8dwpMuP_9la>HfaPFaGy9q{k8S|?1avM--~a4KYccV%umVGJ2-!}5!<~-a0I?e z&hq>3J={I{>HK)|AlrRjWr%Cs`k^~zJk93&kM0b3+tOXW{>b*Y^JJ5o&pz9Ha_`PM z{whxL^UrR4c=JZL*$bZaJulj_Cr11Ei~l~7D`vQRl!v+P+k@UA0eQr$7i|^d89&jw z{@kW{Ll*3@duD!bn@OOh8)8r2-x^n4owo1%ar?=`Z|??X6q)M1JBDy-v4ri~$q!yJ zVwSGe<5vefe)z70c6Bh_zcC);H9>PUZ-|M-#?}H!=MBO{&OR+F$tG zl5PL#!PD5b{@VJ}#^evPQyD_nCtq(&{xm~flN9>{>pPp`*eyJ7M}7diu&U60x7otr z7I3s5cYogH0;{79lz~l5^iu@b?5ESs9}iDAB?aGXygYk#*z9n%7ksNgdW6s2fBNms z8=rjqNo<&pNQ*lY|89$qAP4`|NsoLx9WPN>t%)s4uicow=@ciKTaJv32cJCRKqx90xdyQ=@9iOgkdQAI1R5K$+W&X8$<2YT$R>T_< zIj6IYy_cJ+2;1Dh`Pru*-}F;yvF>2}nPmsDbMJnff&SXMv+-Vw$Ydl$yWc3}+M{g5yy(zT~C z!uxN|Hhw-hK^HmL-I(s5Z?di-{w;ZG*6vHFY4cIqvQsz4R)t2fbrmyU^4$={Y# z!1rvYvUXHbJ6_xOWgm-?AZZ&FMmBCx%2vj z@{Oku(cViKh!dgC3JOIfFO2`{U;q4o#?sDcgrR;qJ(aa2+T0(m!rfcC`7!y+9m>{q*OqsdG5t)VpuVhh9U^n|uy zQUPKSAt4oUr`{qiLTTOdmsYW%*%{t}aL*364VM3knC#)J;qH}K=Q`t-0nv1Ff6-j$RLRi2I6Pms2iU>l z`Ltl9WXSV4X!)|Aa%L}5|K^_q9CMTiCfyIGg!cpoo&vIUp&Y)>07t}wHhnE8w%i|Y z81W&3z3unCl}y9uhtCoFuAqqKQ7VBI3fjb0ono=Z|E}Oc1M+s03Y^>X`?4arX8Tur zVwPPX$e#Sw`@g#N^VDaMrrpp^wsXBPN`TzZ!ls<>(bJ>i%Aa|+cONNoySKBSo97hh zI2a?WclBCmiKp}JHSQp*G@>NQ?unn1W+2AU;UMEv(~=J)CmaD3Q04emnSXlmzg zcJ`~rwZxe4`--2ecAHB!zoxlv46LR=oAP%^8G%+DX%n~77`1U+i1p|y+a6wTBmKv0 zTogsYEyNHjF;+q_%Xl1%(W2+LSlPa9Lql_c7A^h{Sn*^%3yT(e zK&Su91$QTq(HtSJB9Dge1IxG~+0Ku%hHhxKt`3w!amfsA1hQvBDK11s79fEu|3uF= zbEGzk%b3?js@qmjd1PJf_xk2745yDd2OPxGAFeSSXIpRs-uI2@eainlsQ#VSd=+Fb z?s5Ih4Y7y$!4jRR9yr*Xje%^&JGmbtX+e0khIbI4Qx!tmHYCt z^_HN#kxlmn4nXc;5ic^7lil^#*Y~rL+&6@W=ZhzV346-yJMp}hr3br{P1en!l(XMI zADAI+1$baK-j;iS6I0*|7=}aZq|HDUtT~(Xg!k87Eupj z_ifXbiZ@CE$=o0cAW}0Q+yXMM$-x7 z?9YZ|$Awc1gZpoS#v_S(oyPyKAo6}*#N5|OeU1cU&Ob4y=1hxJfxSwj*i#M&zx~~R z{m=h75P0>oM?b%bL+FRx(XIF2|K%@#zWTe>YW2%s{KMqG{PGV!``Isk%71VD@|XPe z4?pMcTR(mO*86|AdjE%9RE_nfn~ z`r==H`|16URy(_Ud-y{4_EcFriK4Im^`|%A-`nFR%(veHjTZiAXHR}g=KjaM_kXd+ zYQ(K}zy9g_Hy@ur-dX)YtH1a8gZmGD?}C9bp{T+uUsjfl8&?i*f3B36)z(jT&2y(8 z5;3_+Ve!Iu1`!dJy%uI!`||FUKfQZ=bP&BZ+mgTf^poFR_WdW;&!_YL(*v@2mX}N- z(raZR+#-TX{ubfP0itiz{bWbGn*976W%Jo;=X2V$$hkdv_St8AaBK3@VKC^Vw&5@MF#U;g3K2?86^_{PBmA?>@cvud5xfbn;_FBNom>n|pb1{w&_U z`{2Q+_rH3udR>l+K z$-9PHP2dMYXd%z)^Uogq?tk3-eD$L*@BQzgn)zsb8b zzmdMCTk5`vD<|I_2Z%$;b+CZdz0baQ@cYRk_*E0R9O_%lu!@HGs~cRO`VGM#rq8BF zB;^=Ia`pzy8$bHOnacr-0#XO|^m)OmKUZE*9!3MzWTwLp(#N+O_Y-P+Hx56*FDpa) z>B;GvpHMvWLG#A&FCODkJAd=d;Ri~B;6FJ8e>w2Sf6INqpO2oM96dUG7aMK@%J$VT zF`S5s+{0S%C-?vT9+_b`7Cr?Y97vyExf%Nv_v(I&`fn?J~kYgL<2rah06=qNmB#P@?Os~!FNy1?W1 zbo=T5AOJJ7SX`B#{9q7q<;6M=53cot%~~OorHS*|_A@{((3x*_L$?ERZ^SO%*Kg`? zxnRHH2mP+U{|W>esT@nY@ss~ldnu`85f)NokXb3H{@)pcsgJbNzp=B46)*3mmi_L0 zKk?Du;BJySQXkl*4-xf0Zw~frQg7sk)v`(`3-5ac&mTo58nqxk zsXIwCahsy1@0D?Oc&fiB{W(6|$IM$bGjEbjNAa}Z;ML4s+W*%~ox%>ky;nc-95)SD z44oZ4jSDIEe~f+e;jYr z{vV_2&)c@;_J8dE@LT-zwhXZSFvD)|`5!nJbBGv?vlT4(&omimniUvB>@99K8@K65Z6(F(rU2T6Wg8R$&|IxX3NynRX-k1%jaS^3icJS^ar_^1+|}^yoK# zQp>16tGsAufBJ7lhImik?)3@$XRmR^3_rgRS@D(eLWT>C9oSpq?s(OH%L|zJt*pBW^jf-dM8V4;*O&qHjnI9ItVh)YvGi#jN|a} z>A@>ySgDkjtZIHyKw4@Fm9==ZoSe;yRi=#w_ncN?^axPv7c>SCibk6hQy-taQFL5! zeuh1KK|v6S8y>2(H0twdnaSoA+GUf<$2@o#jWjcxghE)y<{LV2i#z4e3h)2<#mOs@ zPft%dM(xWv+0#`D*d=<|5g5ghZd40O?&98ERt$ET4CfGZz925H`gJ0MVxqaJkLs2( z0b9VDZXyxm3}EUkMyyP#gM;nTOa--IPVRU<3*sHb zRis?59S%Q;INV&l5hQIaLan~pySt7pOhJm$F{viF?iAf*H|u-3d)6 zhy_x5@a6jA%tfYz(O5Z5NNqyI7^k_-TN!x3=C1m-z=kMPLXP-1x~0c&^Hr#pugNCv zT;>oSWoL$zRMQ1U-$(2Z)4+>d-NOig$)~w31HM;FyH*aWPDT5F`8cG0Y}K zL#&_wp9F0&>y>^@3hu)TbOBoGXx4zu8*L=4plGNp2o%Rl`Je^pHNZoz?__fl_|dsW z6I)u*;^y=S$w@0&(_+EF(yYKIH2@hy*J4Uy)a7J^C7*$*KqbYK1*>mT?|T^8gQwP6 z>9Y0)Sq3lpJ#-VHgeXoKW>{A}P`8uEqWT<(J~wNz;(^+v0w_8Ot@6(1-Io`qwBmB9 z5ZQ*cqmaXOK8*e)@kBNjEgajbdIuPDAPIfM|Ks34ld-h<$o8U8iFT_h^X%;YlXZLyVl-1eE8xC@BQ}kPo^)$7d0>@*pXxk zo3+hP{k%J++&*(^uF35$@N)+1d5>EsgKho$#`;Fq4C6vonEVElS~9tNF^6gLU4+l~XGpSPc-3_QTcrQSfpWKv+F{rA%g z#6OSVI1dkxPt-8SP*I1p)P)U>HdA&x*F!~&`M*0ms3P3s+a}%Hf49J*PPA6s6(|JQ z%Ni^MCv@-@z2YHXI4>v$jg@H1PWI8YFZ970+KE{yN1r--Gwu^w8fn~3QmIg|ceCxJ z?u7R#E_R#~iio#~c*K=4J1*#!A2pEm5rC8dlRbOf6+d`0Fw>0&>XR3ZdKms# zs=yPPJw5at_J@zNY16^!!y`_R2t?mbeuolq`1qPw1?(v6Io4!@mI4c~b|{K^c=+f5 zNhR~b8+t^6?^KR8gc8aa?Go7stpo+WDKF7$uYDD0SQaeUD9Y*@)3_los%aywS6u2r z)>ow7h+O&r!jD2~DJ^x))X2KoFb!>a5s!rpF9~Z4Zw9Lh30cKw7cX_$W6=6@0wpw6 zIeXrS)3*c9#lTk&e*fvcxVprh3(ZaB7I^`=h3?cxdZQj)hQLtjqM)jlLqRr0FZ2D$ zYk;0aku+W8xXBg&U2s7{n|L=IT?>rC&Nz zj)VW;>llD);LtA4H(#n+~|Ke2L z0CXQpV+<3!WeeAp9xE8((|aWIqDD(@1uSVJV79!lxfCtK@B&VrT4Q)37*ye+RNPcJ zrjAQGaT*eDLq{tRuK3mEXBS`uKB zt4_*XBC`*X@(3{ZHaoDeuTLD(pg(7bRwktaJ%tIFAMFeDJ9*6xp9I1(_q-=m! zB2BqxlENN@!|hvYOSBplT9m}}33j8)X4W=_sy*&(a5LTyE?Nh};>L+Kf$lw*EE+b! z&kJa=A~`4X+L7W~hGK9m@eTLDXiC=L5i@KhA*Lhx{@r3PXb8x5j)sPl&rZU7K}pb& zk35*$^lNb<-~I?(O9!>-zjBE*!u38-=|K{vn%ULLZejmoJ4P+cq{cax3^HiwrQ z`SOw*aZh~_mw8BPf}NlF{F?$4y4q`8O&`ie*C#Kj%M%w7K0472&Pk77anG(e;lr}d zvBdA&NZ~5L58r1Wm`Y^ah}Iv`RzAojocfcd$(>Jy@`R-Ad>YjKL{1#{Tm^OUWJ-SI z(jW}W{9yswz3E}3Tux>1jpQqR(d~ymuNz5^0v5QPJi%iUXAz+Qqf(Rmb#~3IHq+MX zc_XJ%qemtqQk`y6%|a%sNlG@L%NFG>Rf3?a>|%ii06Uhy#p(ophKX_Lgu-mdY^@@c zr7m&>Y;`k5O>*;6t(a3$zxEE2x@BuJbO60>>D=va&$X9JrAPRavN3w+p{z-&sccwH z?9c?QtkV)LQ#>8XzQL;u@sGl}jAwy7@}2!KZOh{|wMuiZj044t@XRMPTpGj%e8=tdtnZE=q{AMZfYPRGjvINVI3Y`%^S691 zOtcf?&JKvRD?LHz@!{(6g?W+&?I)fRO&mK&TNiqbs|-tNeSNWbs71QHx)=#g8biPZ zw2J>(M;TReVBTKEsB^=awM~jo(W&sm=+YEWSX*%IBJXpNK3xRAuwt)feGfPV=@p|b zypm?fnA|WTp7+rAdiZ1pSVX_6zLcq@YiuA4%jV^Z4``O-PGBy-C?X6QZMC4>Y9w~|D?0z;8N|96luzfeojfx@6q}l;T3*+b=d3=jShN0g^0#+W5z))dc}P5wL-ir z87p~3)HC%Ct=plz7^)f2_dE#bQ#*N2+*3aKlRq!Km8Q}Nz_t_noqd^3YIMdJo0i*0 z>s-GL5%oCft&-Ln3K?hMeVBLKL;28s!IIxMJzYAljrt7l>EsNVX4}Vkf#%&r7Gy0n zrLQMg$SM%rXz{0z--JjFJ~_2H+^(4rC-9xN4*)4MuKw-6LzsfIsVZlUZtKI;kO#cv zLaj<$NHVYu8qlmgp07pf?Q%_ZAEza}S2A8S6Rqp=kV-mLwkfL(H;Dlc&6Fl)GD%S; zEl0pxcfqo_`XD?M69CoV6Zph(KD+?BLTiy7hlZZ@lLqx)wQwKdPccc*WsGB}0ixyk z%$Kd8O>{ey2kZ0lagU$Iv3Mhmh^$A8C+otny!t|KPzG3ca`ZKicpvX7YpV`Y>^}#R z$)sW{=z(8c9>`N$2q6|Uo7w_OxzdYeuPMeIN7jhA#_~{|)H7-qcc}@q-IHI$6r;9k znwe&nth?h%sb@kh9c*!eeDQJ}Zc34cbMke0|DNljHWuoA$bZ<*fM?AaIO_=c*{9K; zoh3oIaxgxn5i|G0moHcg9d?+`X1NsF@HhLVoXrq zc@N61lxkW=OG~BL!cvmHI3nhmK{_`#ONAGQS*`vRF594~^@M;6pn|u@eld{^Xyx_v*v_R8w(HRGUO|a^)eu*j0z20ZuSm zaPGs_Que^8>WCPV1IPYdc z6NAA5+kuoF6dIR6S*u<<{bsa)S50Nc@}UuLSmWZAW-9GO)=&*&Wte+{cV;2ps@Lm( zqEe7!f;C9Fw?dE10GYjcN%oA@^U|qxKl|fEE;FBp50z`;bHejy5on*8G&rTvD!%=a zLPqQ6T5eOF;Y#)-u$8i~5{Z33nDM~96$Fk!OHiguHnir?rrC8K*;see?!M~M_+FsO zsNv0Pw|tCucKU#-ztUX|z4|&|IpWrkNSNngE3NGh_u|97;mb`!H33?yr&+E9imsK} zIK}zNpE(~yl-GB)X`I8dn|j#~GDoVLI~;G+kl)#%hJ4F5qD8~DZB%HpgQAEh{K9v3 z#>&{8Ispeub6g)_6tN2J@9dnPyxcwZT_`!$2*_bVGX7`9eXIh^oBx`VHn;MBVa*88 z#*6}22Z~6z{a=DLvsgPqzxm!_ghXl(8eM_xJSgqO7KE~^W9rLCUFlyoG;D@u*o`VU ziP{So@yUxf&n-v#cWX1wl)Wu)Hi2fXBO+?hiQDDo>DFQzHYmx(R&P1mx~&>Iaj2*k z9G!_LW09ZY6Epy)cg#Kvw`jCp2$zRbHexld6!p3o!O;Uv5HE8gwvhg!TX0Yu=3gLK z(K=pnv&`A9e6WZ<2hv(kPlAmk5irc*3x(JsY%;gjY6zIqBoX5Q>$nn#(W%i09GchK z`^B{O8o5twu{F#P)oI+WUhuc8>1&@TvO@B7Lf-l%zFK%W@*uDHwqk=Xcb&#~#V@RH zpMFh$9yJ%g;zww*FE5CM_*5_30XGg_@l6G4xq~xnUa$C3(~)A+L1VZY6sA+dTFCYEbNlWavx_Je2aQhrf#go4qe2hnR8qLeU;d&kE| zFERKr27-ES-ehc_@4LmyoUDgM)(*4<#!pSN3bTee;^Yf@V3G4llECWjF$n?}PoD)t zn=ULvR*o^b?Udl8ye9c@lhP@_X*^O1SDi&jJ8G?)`O?GiU$!enzc@wb^@mUQb?>m8 z7=6sDBZ`)Hp_TRu)oQuGH+N|GmF`fYmUSz#l~j!w+$L+-0H8Ir{xs99qq=aM_go-sab#*&8d>WRSBi*ZNPH##O-s4Y8V7!c-JBe|tAFN~p=Mv%9l08ujq*^Z ziQOQ%?v^CZIe+VWbWJv_#V_fE0lCMgj>)DFlzu!Yy<<0Vz-0rid1~6GBw72fgMW7+ zfu+4C&HufLqPk(;gNO3cM#=_ZXtFoRSu&P~U~mrRHAZrcA$;UvGaiN%xsBr>91kKl z%*zpxb>iOmRZHd5UFMH2ls*uV2QjZ5*;Q9YcTiJw1N=Zr|B?uLS;|lgZ$SrFl)Ol=@r)gU;@Ho zh-Z;k^!8hSQsk3L+T22(b#ciFW@W53UoGEWzG@4Tdj{>OAI{7cD*idFUMZvyzKX)y z-U7?6sEMZ~YE>SppQJ4h&vf9+`}xqwF!>PovqIgWGqiF7_Z@7+t*pRAE&Eh)jSgWv9@SN;O8Or^)PrpAOYhpLBI~q)csrX`-Fy8KZ!GnihpN#4 z;_h39<&h1maL>37)B4Npti#k?>JdV^$?@v$b4t$7KOod)rR(^D27ga7HR{d|l1}4u zz8Zh;T%T~dv1tunrJZ})_gv`_G)-};HBqoM46ixJH1mK3=~bBmPHT?;*$TpX=`vt? zx9#kmb$w@t3+_b4+K9E+zj<)8TIFhbZrhgY#z}{^;`uyPsb~C+zqyP_fOwlJwJN-jIFBYgS3C`Iq>}8HCBEI$zm9?7nk@C5Gl@)P@O~0IchIf z%-Fe?;a>qkS)O?VR)#!yx8?JmoJN{p|M(g0n4Htv^pW-TqIOlRoqZAHxC+4!R+lWU z3tUeMRfGEGEtCEC#L4a0fEn$i1Z}^eNNTq}ukM8aok`sYOkF?soRDwCK}|~1BZYe- znu&dbzhZaw8g!cGVxtEuoX~muK{ODt3}dKGP`t@tg$ciqe4twiB$bn-Q5xDA^>MVw zs2bnZEPYiBDG+R)z^W{dM7=0@o9C}eA;lJbboBDb`lA4n|W=(nk5>F^e_{Whi35c>@VRoRnhMAABQ;d7oAJJYE|^}| z4x2y3)dDzWD@$2fv5|^Jf>tWYIFj8nNiT^9@_%=BH00xMx7$?YO9z%Fuczj(2MV^* zn1OTabRd7=();ZVc>(J~U3flyI~K#wIkXa%#&F2e+h8af9LZD`*|8-2-w~#ep^N~r zgHR{6G(f%UW$D;@x}GskDnicyBh^!_A)W0s;^$6>9G>%Id7?nxq!nUaGCJDyq*QUsFILX{$XJgFY`)MAZ#oSEG?ww3nAZjDO0UEooYyY^he z22#h;C9n_FCZYj_^I|`s_OpH1R^Fo3S=T&OYC9olaOwOiA-@ zPYa7ctZR|i;wVU;s?h+C%hF6xqXYEYGM zEj#|zGj&A@MmUwf!y+zMG(}jTIp;Q4|C&O~sl!2SQ(r|gg0fm!#l)j;4f#Ca6!Z}b z?&`d<_BJe_)HTnq%s#SZCDN|0l_$hmJ^KU3Jw33EZM}C8Rh?ef#i|%0NIGG-n|(=& z&#QCMvZbcn)$ah+RRNUs*xcw4RRbSIc7I5$?c|3O-(lB6O90)ZrTABo;+XLu$_ZYY z2?lIp$%SeLY|Ol-qTf+hBxjKde9#ILgKYeSqLb?e{iU&|)46>Q5ra0tmTQMLnw3QE z>{--Bbw>)PmpMRMRFrcl`Orxa*4z9Sk{^Okg6n(+K;g|U@7I<^qrS-<*H#Th*~${A z>-$aY!lV)pHLRT2bASlbD zSGad8WkP&6IF#t(fVQ`abnX1B(DxwHC9ARiuZYE>SQ zphGul%qf)Zg#e;hTQoFGk4FP~q5Dk28~+nQ&tPkeEC|hgEuue}bk{r?sQA+LCQ!QRvx` z0y?Tii^32X*Fq$A5;t-D&f@J-vi=? z8HU1F@7kf^3j6hMz*41FZrLzgcI$o_p$;xKyiEy|IGk2ANkUpBz|lc@rrbJTwJ_Yw z2U+5x_qAxHt}=$yD1;$Kh||ra=JvxlENumQ|vvF4A=uhF+@?@JM4f^`^#?b-A2pDEkh@G@AR3L$YaQvYnFZ z<4jjP^31BRN$k;*u60Xh3V{DJw+BeAxO<#B)G2sIV-{FOKhuUHpkW-C^ZNCn3)M$x z0v40&l`)EkUcH!#wl`!5q~o#%T`CR3PCb94NjDWZxwzNZ)q(3UL+%bvY`BK*IJ^_y zm#GUZznLjdQ2`&yll6nJN@j0mDoE*Kf=9j4Mak4(1=E{&a8aV7THNh5r#L7gktZPL zb|N;e-($jv+)JWSU|)zdCB=c^?FnCL9JN{TTj_VbY%}(G3#~_ozXsXm2KF^lbjz^iT=`U%YRjY`f##UX5|4N}bi;2H*a55R zrI?K*V( zz|AU3&%=mu_rr%>x)Q#esr~2sjb!%W=b>&4mCuGw`Z8FR+Z^^0n<42pBxok%scr8O zw$+L%bdE!2pIa}7ss_qrD`x2mk1yPplm3Lmp#mW%*0q5QE0If_x3oH>&qKNa(9}Gf z5Nf*>xDXqdT4M5Fby#s)#KZP~%e{vDr_g57^xYJE#XjO z4Xg#MlKXQ!sE~^=g@JjPiPowU0$K`k1f;BX>U}vm;yZZ++ELT0CX=ZNL}pp8oHaTi zVa@m3aTtHyQ6?H1A5*EXIG7){L({Cl1G-T6K5f;{3#zO=5o>y~c^Wlq9ZoL7^f?-q z(tdx2Cc+?H6^r8SrU`%5^xxHeKe~z<*2b_F)|CYXEV-xjZ%(la4fk4lRl>sIqmp?d z6aT)(@x->JfU|$dNCvXpz((O&sz3z)^K_+`5F&R)i$&eQ1Ff;oWuJvE@~*97z-!%# zrC2<-9XYrzT^&7PDm;aF!z7|GEj$K<7W;TgC1hKc@Ofi{-P3O2PC|$-_|xSVuTq?K znG}PSDw*QwucT}mFj4jQ{V1DDc<^0x&BgJ)UscmN@X&nAv|K!pUY8YtsX!d1l4x$eZ`5T!qN_3GvR zCrFGeRK121`TcL>M23ZT7jmY`xPkuh;Y}Wtsk`>h!=tB9NROSr(}clLH7Rv53P~#d z<0;##S?;Le>a`=1@Q)sa1GT*umb~LZ?YAtr+~R8GOxA0ugCurkYgIM%eIv^1fqo)- zsAPtQwHq;A?Fb&Jy1>ju(*n~xsYDIOS?694Yg|%IHcv=Jq76p$j zebQaL9~*J8QaFZXUfdtKdK5B2&g%7lg)KRmMRn0RbkvVahtUWlRN{r%nq!9q@oSSe z)tPSc*_5CwQfkU8hXA658RY`Y2Kd@Ek_w89W75e%F*3)IXZvgIrQUi-le6>Ptb;wJuh;1rfqi&>R}f;>?AOIVrY}v-M$3)# zysL=}d4qEKhzv7Utj-Azk~1hTPCN2@Lg=o{08NzYRCjPF(bC$eymkb9qcN&`va#wa zP@0nN(d1gtkztcN!`(V(bdSiE2n%tLe$1iym+2OZ9W7Cq#9#u2RyZ0HffpRKPi|uX zU_N2lmg^IOje~`EH^X0#C_A;6BS$>$%?-IBi9ABXyR371WE{(PsUAg4;bX-nu4)ZW62pvuFDpI34 z(aNQ>Xqyp@j!I4;TT9)?i4stlXy{0X@EX26!Q9n9g5co2O(@we8Tm_$Gw0d%7_^Tj ztA(o8tO#fuz#x@$B}O2g7{%I2tUibs6<@p8I={v{xr*VVsV)_ zc?odi;*L#RVMu!Wt0CXQVx>33g$9n!KGV`E@fSBX_!1*={kSquj7Kj_PqkwcyZ&|# zIPV(lPXF8LF3#NO?g>*q3daiG6g=hlz>t;`FI>ds9CnP`klubPz3vT{*k5z8*>~&R zb6jHd?u6`AG;9KzCWJ~P&WU}mlXHwZ#tKDhT^Zrc{cuM`UMd02^2luCxDDFnPzJZb zY7u&^y+gNXl^}W_(>J~B#>=fhpOdxeu$a5=n*RJicDX%ks39y|Jx8++q!;#AT9?Q? z-@(TySvjy;SJ)iF(Q4TlQb@YByIkiK{(K`ld~GceAif@znFRGn5g{1`@inp=96_5R zqW6gUa(6G}{8jS^%_5?os|LZv7Fnq^Y!uKQDB1`Q!<9R!#xfpWS;(fdyNsgldg(Xh z!Yr4BkqNS2#V=#gjtNS1KyicJ& zBwl&1!ljFOTiXSK=ei%&?V0yRVHPLes@0f_Q!&983a*?&!m3|a>c;-jV@${Pua;Zk z3N|}NTEYQ?MOKwjD=7Bk2kfGP?VQE^kJo4~qDw8Qw9NW!jODHo{otP;4-FUc;F}g8 z;v747Pp%>W3+j5t@~@Yh#K2d9P{Ra3lhzq*6@}y#g>8-M)1Nnq^9qcd$`Gd2j`lC*JQyBF!AjTg!!R0-O8|&4vQao`L}j{Ex-2P zfBd_T9{m1`d-ANTzWV)F5AJ=o+Trh;kI&s$_BMYv|F@7~TAQyMgmSo^R#thMvWf^p zMd3dEkJjH<%-Y-Z+LM!LlJLpzbjQJ@7s9$%Mq6O5_Q?2Y(>JoX|LF0Jcb=S_yz_3j zS#Cc$`QRa^pd@oVE3Xb!rFv)Q;mI*z2oKD(k&T)j)HedqTH(DQHJ@|}y`1^)cXumK za(8!esCf;)rps*I>fvx(1#QkBVN)8|nR!@M{OmDXV}>};Njs|23hoY=4K7NNz-Kcs zXt7<>anb=EH2uc;NK;-0N6uNz#Jqlob}Tq;sii$za~Is=sBf~!UVxe9O5TW`&G zee5G#`wqdepBl&1y~o$U?N%O2$F(nYo#p3Dx%%kY(eYzkKd;G-OGHmpAGgm} zaH#>yjnx_j2}~i)UPIV6-grGxLHkH4IQ(L=Pwj!HSFo~G%R)H2O_@RSsR)MI$od;g3hnCq2(+fod>+@nVK!eriHXOv+c&KQ#Haw zS0aC~qp}c|Uuf={4H_4NTJc7hLpH$rR=mUM<*St|FOvur?$zOOba(~ABYk|R`>hq_ z_g4K;-l3<@NOl~)Q(rHaey7u*(wRsx;JVC?OfUBpHVRtkH$)2G<4#&<831IH^4ik6 zY7|yqM-x#?z7nE#hD6ECxj!Bp<=|(X3v|4JWwxAr z4waxw;cp;pIu&x&!IfCa;;6{%ihUrhQh;jziIcOTKMd1P6;ab!7$<~p0jmZm374#H z)8jY~YeQi7uz>m=Z=;DZdat!%%SB*W*}H#RUwt4Y)T~6K zgoa-2>|hBANzQjF(k@3WD#Pt*|8X95e)bu6uitPNyf&^*UlS+UyK=mSn-pi-*KY z5v7`iD`C0yF6*akzE||ywk8e1oZWb@@74gPRyfJq)?f$cQ@y* zMg7||!YyvEp?my}jJZ=3zw3yb;AbT+Xg3#KSStHx&?Vc?5QHQ?;RUQ2h)gmJx zp*mPLi9Z$V1c<5kBCRaGWY?SDgze&Oyv%f-_Dt0ALiz)%(PorBo|QgKlpvN0h;fBA zS34)w?i2}&0S~@4Fc|GLbZBuLuaZB4nEQ-Hf`(4gX-ux8O!`?lBmq81gh;Vc{-GIE z@pm`$FnSEMA_idQq3K=+l|i{BkX z3vtv3>}XtcjWHDt{UDjPOr}GQoEfhG2-Dh9w*t-H$8MO1M4;c5T9=~9$^F~sKsiq= zy!rN9cfA-?V}-(&-Vv03^!lfi)Bfo7uk`;u77hOB_1%ptx+|YIJ;2U+ui%nk?F$XH z3Y$IhTb|RJa<~#=(3aA{7qtX3bmEITxO7m4MS^1c=$NRt44l~l%pf*QwqXjwOKO3W zGpucQTI&Bh@cOhXK7FHruUJIqm*J?`u$-ok1OrkitVS5*unokf@s_J8qK9g8YyJTi z7_ZBzzZuNZt-6?W?e%hBlx{E~h-TKe;|qW+VdX_-%=MEgv4(nh9X=qwPJ$ae^+}{c zC-m$Nd_6fOPr0Fd%aDxgM7nL4+o#HDAd{{M#M{Me#>Qz~UO(-n*88mjmIturo&C@AFZ`MGT(xznR06q! zO`NcUP#n5Y*eIpD!o%ma!LXqaG3eEVrAiX$V9@c;9{~(R&NbI5$vLe)90~ag= ziT48@%yWrS>4Vjgaf5qzXFq>oKk>_aHyq4&9VfKoYf}=RT^*EDb@jfPqF9=%;ZPDC zs={QD;pzGsAC;^JpAvWo1Cei}k$jd0~Lm2>#`@CoQ(z-xg%*7SKE%jPNbszs!_<~?}j zX0gWR7rxiJY|MOc_6*_93CnzF=FwU)gDaKN8{BDjef<}AAN-RXC)_sHC-IgT)}AuY z5v~t1V2=*ZZIZscI8`<;I(lf_z^R+0l^~RD9?syTM4;^FS4c>Mbw4q=wACr0-nW9} z;&~X)<1A^`SIyWKH?!j1$3b+9^R5qvtcujFC44eq=jhsoqhfP(2_tS z@e7ueS2HlM{3uRLNN%}oD%jJE>!p}j6 z7K~XzuDDfaziq@_Pu3a>@9eiH(dbhm60;B;vGcEqW8{8Nx45i=a%_`Sd@_Y3{#@Uh zDfh69{+lTe{*9#J#A`|Bznad|~Sl0cP^a zJD9t(pZ_wh_KnMIUh<`kvq5i>Sv#At{T@tv;Y_Zwf}(@%>)(|&Kive~N~P(f*xHZF z7Mz660o0N?ky}^UKIj*(shlU+^R{=EA6-OuiQ1)RJl0l@yTw6zVB8yrbXQ0(^h^Wztl0IgC9v5TP< zm0eG|7j-jHR&pfR?|xO{3C$a%jXYAJ*8D9%cLL*4^8Q+;4l(C?y;o0hcGbK^@WQ^w z-rCiB?AnZ30Z(QIIQ?ty`Rvi@5!uV;(sDUiN%j!!6G1Mm(WZD=(ob1xtGh?UD$J~1 zFgJRDcT)4t!8Pi4GtDM4T&P9zP3;kGd{EUNU+VWU5Tl5B(}lk12|x?kR_dLMkbqmH zt41suTpe2sv-n7VU^|1AbYhc*{l9@O%1>Slo6sMk9vfyOie`;~~cIs6amt zDZxu;n8C6;3HFvcAYe1Is%g&7tL<=J`XG>c*~w{3T8Z>sd9f(!B8t^zlnJL1;3CG% z#_#&gIKk!e#&E8U0n3)(n7AS~8ixu6Cpbf0W{vHpOISlLQV~JNOarf{nsnlAs>C{X zpA3kEfzx2gM4%WdU8?19j=*j-^b2qs3yAiK1bI>PDB-1{0YMBJ(M~A}qeuQYDfWCb zTZcG7RW_;kK}0o(r6)@Ig9>Y*I@FnGRa{FP1~nh3$;t}d7+Ut*a%d!Ba8ekHh#u;f z#vp2C_K2D1Od=_#VJ=!;Z~4#wV&e$9?N9AtCU0NG(yeVT^VhKazV83A={E*Wvu!o9a-ec z>ANLGq+hFHdeH|VN%a92b)5ODxqQkSJdF4`Yx8)cl~kk^{4c`}1Ps$n-vmH{uS5b_ z#Em3S+ZWyZ=D=fTALbbT5^AlESDA!ntQkgV&pYqRfr5y@?Kj};>%i;+9(2M6E*_@l zWlZd^b@dM4^*nvidH<5qO!F)W{rMsO?6Snb=ANjj=1>H)bWo0*=i1~GzdM!9A`LA| zw=5kLF!-nK<7(q6{1VM`59gmCSW5)DQJRygRe%r6Y_%ZUhq5CknXV=(xl&>!sSsw9 zvk5x{nOw6>PiF4GvF78r97+iWH#f@aGA&F6ml^4tWsik~nE%RHATVto)i5~%5cz(H z1y-oJy_abA7&~&Bx;!(i6QfVSh;XbdP7&A4g8nBL4l;z01H!Ff10NDT!SiSV|?(FrLnH`CE@j#Z1S5_EM6}Rgx|ngdA?e2YpQ zY@B8WZD0S1WSDO^sL|Jfq8-=Mnp=X}*c+33n0hBN8n(|{tKD;rM18qF_jp_z=B}Y? zkiw#Ja&KQqT_ZZU4+jNt`?n&A8<>MWq&9$##Lf#L{PqF52;3c1j^!cyyK?U}{bc(D zTEg;hrpB-p5akl=-9k=&FMULW+tlUlb5YZ?^KZ9q?W_SrV~l546?6ALE(GVZgyQxo zgbjbV^~WWM2OchVZ+tJ;P5ub?%p8#C z2!KB2Qa*c4S>m$}RoNCc7jUl7H^XiaC$RK_?^*e`fKg7qee65xVR1le=>6t&Ovhl1 zhPMKB6R$-z%V#xToE|==Fo?7W+xE2{mt_lJ2d58@=$&*547#mw7LK&zqiEtd8DMhw z-Jr+qc_8mYI298!yF2l={0L7^PcB}rUOsyDb4r$X^v6&6<9*xvdT#gRh`J1S{Oqjb zx&Fa#r~GF`^@2FFKFgVR_7{FMq1@in=Q+wZS0I^pzIkWz*22J7m~BrR)U@M97l+DC zFFd=wHvr(Bz3H;P%x>lrAT0c7ApCj@6z)I%ABS%m7{`D4)76a|n);j7J9~Rvr8J(y z!fZSH3qN|Ccjs%+c)$MkXD_D6$jo}}cQP#p7n=u2-nxghF z(a!$xkMDcdZ#C^AyoW_LI1j%~;0wJEV*4X0e7`%rTdK!hgwxk?Jc*2jhi!!081 z=h%sv@R@%F+HQKk2@dp(k|FWMPxz$pd~x?%7un#k^{oknmN#_w;ltCzuU&~b1Jb4^ zvHZ4nk?J?A8}B?4$j#B3)&XOC=YtPAIk#>456dy}^1YIJ{5@)N==LaUP5{Es7Ajr% zS|ba8k~fLB%f9lNBQ7qzw1!GQt*9n`8JeF|t@1JEWeTJ8t{5kFW;*F2Y0&-vnH>~a zD?~MQCE`B}r$BlpUF2JfuC6kncvDX$J!{=b7%9vVk;8Ek{=m`T^%B+`gi^_yWr1m? zFJdhb)KW92NEY$;j>+hkdC+fst7|6H2DYgkHj8K7lxUCAN}wF9t_YAXUW*M4v%Q$U z+O9vXy@1x;53EJN(th=hOKoj~mW_hqS^Kiq9FAXkdj$*~7Uzv_od7ZL=JO9@g)e!P~4_S!EDbp+IS zEk-Ein@c{`d_a}qXGC3;W#!mXP`i(nG)!eb%!cUZ+&Dz}GQk8TYLQeBUV4Fbc&}cQ z;a7_c6a(K{;svW2T2M@MxIyZz%kHVI2^00^yYW@lV7JFOo8&=}`gN){LOLmBg0jd5 zGAs)+_G%RxCz1LCXS5-8yG^}ynFR#k1iOVX(~*qMY~@D41kEZ20Z##?;;gw4C&C84 z+Omd=u#f3x<-vl^ykxzayn*-DCtMt_L9f7PCJkSpSh^b`gui`{^-dz8~kotXky>n=*O8T^X<0L3mUxg zVj;DSr7c+G23q`~yo&%ci&I}jrgLZl>EuS!oA)V_{apImJ*TZs{+YIl;{2l1t5N`q zSP)>|jKz5Y&FYBGzGT*9^NKXW7h7)3K$%$fVfh2rA zg~GxSD_1YI`TQ#mw#)K~+9MZa6hl7UFTd+v_#x*`dIu8ddYAIHSX*g=)3P7!KGm0@ zP+Fm42zhI$RxrEf2?u4!A7Ac5>CGd{h~u4;+MHv+K4G*hfNVgrvJ!6w!d#yy^_jUR zS8Lfv^DhO0quji&2^kJXm54MBuUPmPZK|y{r7;MqD}^J0A%+O%hiP)#Dcr%n8+t<8`xcbC_@SdS(D!bh6ScaVcMh6Dcvmx&(g z=}FzR8c|c#n`L33eno0jYL`AlWBcfsAId`}d5|Bw(=rGBpt6hB@+LRb8f|KjJ@7_aK2wv5?wovyq{ z<&kX}@X|$=o@%fda)m*3V4}_Rot^)AczU9cfpqDe22C#dO<4LPr?ch5cCZ~b!z^2F z-t4KD%21R3)6JGf?}5x?v&%&2j>pn^6X}f>4dcs+q@icx{)@04ra`0-U9F#uSwf`< zn!eFo^QNY3ZAcdoi+BtfPO#L>r*0>@KMr_#p(|Wo7JNv%KNUc34igSnOD#0?AzE$h$)^ODn-i(MS4an=RR5L@s)JlGla6_-=eD_W<6X#)opS9! z5YoYnVSCTALpWK*kc0k>^N7x7Hz!?E=go=#zjo8;JmluUx21s$3Q{ zkv)2W+9_X2v`MTFIdMjHg`SN+}Us+z-W&51WlH%<>(Vi+(XOILd{+@oc>>pf z!nqgZ@w(U+9?>KFPka}O`40JEJz)T=YL zFkuS57DW_FE4gec+x8i#Q$*7gNE2B}<%Jd02LpNz5a~u_<~t2UmSrE-I#f+jH(9pq zOX(YR8-sH)!)+vym{vG8Hbko|Z*^yrq0_Z2F&6y`Eg>`ozh*?TF(T6ugQQMRZP4!P z?%r%$CsxqAqJ(N>6(zpY)q#gE_!1=N_x4z9KkVoaL6Y)@?sK{)wV^PL`22lavtW%1x} zdqI5)a}K1v#`KP~xhx=HbFs3?I~<6U8$V$=}m*h$COBILVJd`{*(v7p(5c z&_KogYJT`%^Z69TA6yXkz=si;S_SGUbmN^}hfed&Wz}&}T^hgcjBlnkZDD~4S27~K z)F#cYqBG5yqfY5+;7KG`Kh_t!5=6OgtfvX$O~99oOWv9RkW)zijX-g)@9et)bP%#j zXqIr)g$!V1p_$Y?6)4Mx-FVvst`AR6{#&S_#fgN_a!Cn|r=Ff`yJS zsn0KwyS=8U#nuP**LKsgI0<6K* z;w%{CNV^?5C3@ny)Y_9bhlS?ilyyW%X{>G4xXMc7;)!^Z>eD{6%r}XkL0O93Y@2;@ z;C-yzMVC339(OHi`N;3(f)@?2cJ`uiP92TmSUqdR;0?3Qwp;tOoRc4X&^NK?kFSqL zpV3g|2y-rvWs5%6Fls1#b37TvKrvmgm|>b7&VC+KHm0TnQqL`WcqL@&I#Wb@@XZnc zf_tw1>A`28(v%iu2xKr5ReP!UMFQ>#NeswzTi}n?+k-|{#7=l!6=z3Uhvg&M5oqH0 z>+CCRQErt%)ze4UIC^-F+P~zK;NOKr#0zrWr8T(;cX>BmsH{|F^$po z4S!bS91JIrDH!9+r*sH*qHtsru7PKWmx%w{X{37)i)>4TJ6KYT#PKjdFXlt|prJ=|kIgP%rllhLms7C|OH#hn?cWW1)pQ zNW36K72J<0=A3yz@x6`){!6B7mAqF<$-*8=IC7Zyd7vu%nYBPCB82wY#Lal9$#`dy z=Bo?Zu`rf3oLKpmdEq)jqAX>F=cqlg^z0+$xhT&jS0ju)J~??gG4|}xymStQ2y=6# zLzfX&G7#j~GA`L2Az__Yke4o{BWLZD{^1U0&lX;b9kF}BJId{j@maXz+HlR*qUM{o zO86U4|=ykDlG@An;xD1xO6zH?ZDk zs;}1Qs1qQtY@FG#YJ7MT20F@s#YtP2M8Sn^X0feUl;j0JYsi-%i_93L6J7UVIbu<6 z7>>t?`;{)#s&B=){CUkeIE(1H*Z^VTu8gAds9g9ce_4I03p*}c$RSR;*=bi7s!kZM zEV0%wLOU_%o%Pcz!HU-r#uQSqev6n&aBkx-04Q4>QpFNqYvJe$T{gm2Dlq*?ws3?2 zH`YIyA&<_9o8M<=M@S`8LPWchDb+Px`;}nGTeIq}QBVQtY8<6+JRbVBhZd z#i^YUq@v(P8FV77rE<7z6?u)VSHMze4q}Sjm!Y<03pIuc<}X`$M9SJ1VDkL7HF2-* zttyv`?dA<4b;3D605JzzV7+d~AU&Wi|U_2Yq(A{e{60V31z>}zBb(Z0i{ zPC0xR_Saz{J{U@qO1(Cuaa<_JnD)Z{FQSnmwd^JXY)@Y5?9^Z-tU7{9FK}O}L*(j4 zF!kNsnS2s%#6e0m8VVO8;(Xz6wPf$zIl-~h=XZ7p&+B)}o~mo7wjF&u@*7E&v4Q8l zSPt#3xFlgtSQ?YBdXK+_o&)#;g(w+suQXf%H?{B~n{qkdv64O0DM(8d7F3#*dUbJX z@H|?Rzz$$6(T%x|H<$CZQA z_IIFRUZK#kfFjq~oPX^7KSxs18lD+LKi`yPI@#~{jHg?+d!^l)ktA~pd|3Z%Oj1&W zs#-!jr&(S5;K(GOyJ)yxIIl zGKG8{?}{cnI|Y4h*emP`Uq5SsmPeN@2!Y#Igt~aig6#9}%gDvEkiL2?FT&B>eB^b) zpn~hK0{JM7jTD%(=~|ochaPVv08>kf?s9FVVI5hC8Dg4&F!? zpSN>%S~}9JHF=5`!f5dPF0Z(5zh@hQCGApk4BG+2tu=tQ`18-}{h!gHnqgw5vlOSd zJrI%hP(z0viNB*Y_34rO^F5HGP|=&~1@DI;9iKjWv8$r&rzg5%5L~^pGsH&A14LN* zvim<2OPjzZ2prsv%;GYA)m)n$NY|Ubu4oeI5$i+$>5!VX8AC-Rk*e8Cgo;`TIb5sc zaX0pJ!=;_3r%@Wr(TlH7s4Ew*%T-~?i322HZ3{JPdA;3Rx*S>p%&tDD=5HTG)8A&U zz5SNXiJs6hH8CHRbtJ;fT-neIZBjfV-nmYkv zVwr5nI1y)-PTu;YA`jPy0p|$ml0}W%cx}9%0kCQ6+sqw&7nh$Z0EPp8baFwjoxp+R zjY=KjgpQtoLt=V1(&V}l;ceaLqQhn_D3xIx0fKn zM;zO^v-8lgnGbM69UrN+=;L^0nn9Q>y7C9*Ill~Q55$m8p3|5cQE~X>i9CkA{Smhb zpB}t8gQB^%0l$EWb$alstC1CpLK4o%Vp;lV12Syi7Y#KI_ix@5A`2xP0GOQhlHX5c zHputS^v`LNQU9b|hm%+1YQrJd>A6|?=SL`6LoPCnVW!gGjykiBmQGwcfbFZS8&KotQbs4qmIPsB zXFa(%Zhl2$xRUO~I$uZE|G7?>dTkeJrgS8IAyEVs3J=B0;5|LnBMe$G9O*T!CuH)A zp7eC~?BrGW3cGi*e+-f(c7+uWM5PdTRklPsf?9qP@GkIz5cbjBX%&dxYQoO^vVJNl*fg${as zs_K#5-Rf7bfey|o7WVg0ieU*1#A~^L#67X)-8z3pE3>E1u7TCjNcNlG{A9pwM`LzT z9V6@W&dycBxKhFDbY_^N!%5XdY-aE%DC_F69ZTpSAh?jp$vo16d3SPX+tbhc1QVD0 zyWhxd#ppNQMPr>PU}l)x*4&+SfxE2Eo$|ayIVfX|axMmG(;2^nO{0ZI*wZif1lx6=kx*w74wXK7$sbE7jq=4+V3;?lh)uGnv7&s z&(|NM))Z+w=5;A87LU?@3*zQa{blEzKtERTHKF~v9yEzPQRL^99qDY-bt=t)loa`` z>uI#Et6%zCitfgW*D{P2>tAyH(&>hRmd!P$5ArmLaB>91dc*tXsFv2v`Q9DX-8%hb z=3O_i7FpV}zAO!~Gs_B8gY8}>YDCVy9(}Il#O0AhbbnQY1K3=ua!0UZD+t2iIN;t~ zU2y;65UhlGQ9&tHWTrW*Wy~53=f^R@)3q~+vlOff&>}j46a`-fj~>-+Pf(JLgk^}E zjm`>x+n_djWwXg~s~VVpp`H>KZU@;+ROd)GP8CBvd@r~muP7cO$;7G^0R1?}bI^OA@2sQK4WrrFP17A<>!vgQ#M%O_pR z=Xfe>K1`6Yn87_sme<*xex|cdL zhjy~$SbmuLEnLd3&1vz4E@;?g3Z2XZQ*NuxHGoexn9@ofaD~p)Ph`cM#4{&~m4hQ( z=(;lsr@ZtF)W_B(EHA-IRY#Ez#?Kr`RD{D0LzdGo;QFYRb1xic&^E|?x12f?smF&u zv&bY(Q4N82WBA=fE(yHYW9CBC_G;+q7VoRNz0Squ;5iOhV%=#t)!1s?0uzbwsG6}! zUumj?jL59kAAF~Ebwan!Th(fcNj1MLGtRrc)i$JXl(DvgA#h%}R)QMTs^J9n;ZCbS zCMM0=uJFA6*w=!7_@gNDVwoB5-cg-b|0i~6ept0$8RclUaLsCY)*%>ug%;MUf|}%a zlXywHcB}@}HX_&_Tf`Zg?o=~Kr5dMCP{`FqbReC9p4T=-Iwkx=FHQzow?n1DVk@JL z<2Yv!@S>q#1mS(Sg%&#QL1FHA9UOb(%13c2tIHSPYIzNHB55HB>xvs0d5`unV8P+* zM~BWwll7}GVVfv(5$vc_)#r<+alNXLjTObst;6di8&vw53g92yd~wFbxR$%`LOBC8 zR(iGFkSIwlzfNI)U%-{|pL21pZ6mLkz%lqtY?D-Dug4dtc7XGIH?< zQ9IYvi;EA(J)mvGbzeLWv6{md!-4m52c>-LhC#;{MdGyOt{-$JQZE-aYF`t1eAP)T z$y-zO;DJq}W!n$Hsg@DxvIWjmJ<&T>n{s=JE|r?3dI&0SzvT{KAWQ=!ws-6{(_q?3 zJUpR~r1;U~p1^urxU>=4eRS5lCQ;2+n@QXk(k2wu#5{t9{#8JIcu4lTQgeIo{t~dt z`t8Sc$;%Cd3Fel2*%x`k5dm&oAUq|#HOHP@5t=#bbMP#o@zmao!G4s}7DlDK%99&0 zxelW(H?Tir>Z4boNOheWIL8QyiHpl#D)D&nPzN2~sQomnDm zmmW;%Cbi1An5vW{yU60{C^oqjSCM~t$c*uIkGeVj2GI2j-UD&j4+o2Om zS}M^P>%J-}fD@!i9DIF3NS5&W;pvm(i<1kyaf=I<4(avGRYSUb3j+3ygd}Y|42IeW z&pVqjzefGaD<=?pc*=%4i(k{K+hCh76|YkI(r&tr{ef&yc^xu6Kw%5RSt@bP^y|q$rQkLY?ot6#$v~yueqNmT?Y9t$Bm)g7A}LvP+Hlv34!{RB zA?iscRpsLoh*=(GI4*rGBTsadIq13G@yuC4r4jXhm3snYJ7Jhz;ty2ggST&M4TB-gkcTa`%`7vE#vJu?ujaKTpF$*|X?R zS9dPv`;pS&l|X2=i9*r2vr{6matyE1&?ivkKWH(y5YOL4*l&Ia#RJW^5cM{BG1}rR zFj-X*ayJg73~wV77@IZTFt!gRU~Cw@EpLm zWjVEPM5-`1z0KdUeao@+cX^ z-$?Bvo2vrf$;$x{^HBZ1w$?H<#9JS8f$wtXQxn>PKp2wrG?f>2h!pUfj+LhL2}}7Qde1IwWX;9&686$^->oM`|^p& z8E=|=K)j}`yXI?@sn@VA3IbA1RVpksF!oZcvS2;nCfU2@%&b5OLvz5v2DVXLOB(BG zVH30J2MhP}iQsuibibl?!IR4L+a&mpA{Y8)yBGTb`# z(7Hl+SQRk1ONU51donnTmVJ!^0pnDw51`IaWr-|SF-Fk<+0#@c>Na&(e;!cb@>HE8 zfWxqiUQUYViw>2PX;dy}-?;%BerDnJRZ9h4?%UWXW|SzHsuL_96MUj1mg6zsz=7Ym zg55Ozv?b1W=!0>~bC3 z&G&co!)w{$ACS9oXQz3xh6hhh;`eCrI6OG$pWK!AYp8Fx#iDGGw*8RQgJe!#N}nd) zliDL4wf(5yqzS-mG)$Qsl{i#sL;CiEzCAwJ)U_w{q}zMzU(9lhHqPnBJ{dsvDDy@A zubcln9xL~jc2m&_=(gXT#Zvarx7!=m4P#+7BpX>;mUCZ*?ap#9%)g&foFKQA@Wo#E z{(ce_p}4%e1w~qByn`*MF zasl8FOApZtc-z?-*BG|U5mSeBKZK3U{)Th}_}HJ@mI4iJ)evdVt(DElX#?e}$DUgK z*mdA;ZcT(erlMLuJ8u5?=qXo*97n|z{qtB-Hqkv`ndwFDM=CiG$V#!j@3zV}Sl4j+WJKD4v2JI2ht=4Ji1y+@CO;$=Sz z8qB|D(Xx9Lp7PE^l8PYnBvY7<5vI9X@8uHZiGhCD?2N{jXC97Z8aAQkt<}d24qg~7 zsHIURFL>PB>o#A*zG4>RJ4>UQr5%1SUmM;JeDBURe{`5O-uJaeP<6M%i!$`IcQ*s+ zY@3y-xO^z~cXo=!`TZ@~EMV*=jA|}~Na7TyOAHEXB8MQG6=f<5`D#(}FW9FldAlNd z=n#NWGLDi>DvGZj3QRX|w>pBBYjPB>QR75+tOSkRnO2iTGN=vKv)R%tzrUeP>8R~| zVnb1^lm#lH_e-HC3$_J&%{cjNScnW#(!IP056tJ&|3n zZ5x{r!_IztlChk^rLN`PTr zBVu$sMj&aBYN9SY5AC&38yPhjH#U(m@gxg)g(HFM5}~M-8XAWv5k4|UG$!3h5T5-2 zJgo~ww4fMpGbM09n1-AXW`KJK@lnYANq>m#6)hg5qW>%jq85Xh_*cVr)RNj_rUr^? zZL?@90})q%#+XC-hVj;H-N6PL=?=T}iqI**gw-iTa=zMMBSsLB(yNhSpi{?!hwWVx&f0;wbX+RDC(REaVGZt$xSX+4@4`^cfJoj>PNNj$T?>HT~Q==R^nv z)Q`ERT=gY1T#5zCtfE4#1m7hDF_B`dim$ii7XqVk2McpwWBJXTeYtPdy>U>~YWfnu z^$s;#v7B(#)W5F7r3XWII5i6dxz%Hu`kZiI$B;?@y@~xqRapO z*{2__sIXBrxlu}u<~VdXSg8t-g0uI}PCCWU6m$E+JvvM2YTHos_)#U4-RJX_1AKCs6kB+X zjxe>Qn|{&a-Q5mW;y@}DEtk>;s=Cf>(j$VoeV>Ya+hLmJwHJ8WgbDs&syyoJL|;Sm z=(}&Ln;#aZ<7iZSL0S+g)f}9JNJikrsHTTztuCNA)!jyzw?XH#0+e<8p=ViHP!s^$ zAf1bi$=v^CFMMuWP@VIfLlZ%o&}) zZi6EXYR6}Bq+b$|lsa=A^<1*edD>FX2QBJ4>WH+}QXzg2{k-jVhVhgcp#44DXAd|o z^p{d=>YfrAFsZW-YTN%kDm%u+RTc9|;Azl&jT|JZT}d%F26wG_rk8gVasRSX3ERcxn!RE11RSmY*Xvv|3LM; zZ*{AJuBL@w-@(A_xm1K#NE3}jtl?Fauk>wsLyw7Y!C2Tw>?>Hbj{{V4?pdVj36dp3A< zj{IF{#(hZ2FyGEau|@yW7g-HF@D1;1DMfG7Dy`YUAtSA^a429+GVl+&re$PBP@hoQ=j0SaE{eTJfR7o1fOAMo z-m0Oe>K&Rg-2|+b3w1eMszzVwvdTdX!Xcn*0&p9Ihc`cm-l&>9#aRW?bZ^(&Z+$W( z0yJI>@lNad8^ki)cCfSm`t7&gM1;9LoLBOj>^sSe)M9Ci6CJ>Ee*B}?cRzfyGl}26 z+`qvZBMu|VZat2dc0~VY(dz((xHmw-*NwBXllhdc)Pr}@v`$f14@kBHpfk&csRrcx z8f=m_P-Lca1!p11=R{fmCp2w$u$d{Nb4qQ*h4Vp8g#|23C355m{>wfxYKOMomJoK7 zmK8dPqJM6Sam<{9dwc%HD!5V&9DUQ3x-=kMB*U2g#LGvf9G=!p{&FdUYpxO}RS<#1 zngee|zJb^o?1r&OgkHzhj^hJYOIvZAt?Wg70c`VtOqE~~KEh}}mNcRLL~)_HTfw>d zyTrTC4_T49pwQR*d3NARZJ;ksof~_k^t4Cx-A+bp3U?;RlCP~Dl5TUen++4qJ%wb& z4V9X!uXuii`1fSME#9}E2DAjPk^#MbpDaIIhsv+9AO#u&+T?3^E8^+PdN#U)6Ll}99 zs(!sdW1&8buhgNdw_+3Xxm@H9Kx|@HpPiwvk!LES7(Z;A8X#D@Zz0);8TIW|7SDMf40nq%!*b!G&Zn)zumdIU(HGX( zFI%JgIK}C&Wruzz01b+c%U?RYH%>QTi=x!RpRlCe;)KP-Z&yC(**Q;{ zXisMsf#!WMYR4GLS=$5hpkr*7F;9a?#WXRkb>gVJ7hY%@t0}7$#qla!Xp-+q}>ZS5h&o%ynx2 z4h9xV5By|WbAN>egilBb81{5LiMywY#keD}F_jIbcHgG{5KaBH8YT6$^<#=7@3la) zaFtEcn)O3nGOE-K#=^4f5cFu7f0r}Tg$DO9u%J@P$-G!|EQG4%JZkIs&` zE_Uf2iuyqIsUcw(`hG?0OfI#U@G1H`QhnP z_u#uba{efc3)h854@yIa%VIbPHBnwK`3k*g&!OU~S<5sft*uoG!sSS-(^VGr%(u5d zWCM9`UIsU!qI_|%-)$8(KahI12Acf=uL3fHj_x?_C(l8o@N^hPEKO2i4Xn@&1|w=+ zfMndGXv*T(?SYz?v4-lCy~nMkzVj*;qZN+Fkz97Nw(LOVV#Vv6&vHw9@PdZ{Jt7b_GqGl8=_g_D0x!aTDgVi;XynEJr*wZ=;az+^1)>}YN=wg)KFqlbBNOH?y^6HO2k8H19~ z>7HCa?H9`tNS{24W0en{9zLgMqc$FvP4A4=dkrWtf^umbgkyBX*JV`iO-F^HgqQ#t2TfYxjM;vKA3T5gA^piqL{G(X z85k{g(<~zpx-|ib2!)jRzOPtWHdP8bZ}Ao2>_lwE@N-_Rj~#fv1wbdY48W95%Pt!u z!Rl2H}4dy&%ro3DPeQO zOsk^x?YC|uZ_dCD*N_~%6cY381ZKRq8$;~Y5Yj(8m2RkH z6b5Yx;K;(r{8ol@;qQ8Z;HLTgzANxJtVySrH{5U&!!163aZ;0w1&lSlmJDgGKqeli z9=q!ttk=PNUe@iYD@P&QiA4^6`PE*XQsf+HJvIL{K~Y5x5BCW))i*jGCG#_`8Urgw z1$p_5b|4Yo8|Jf1vzX3;1HKr|y9&+@Igi-_msERmnc0X#<*gXVw9r8Gfiy7_?8ol_ z;q5Vnm4urQj@dx%RO&>~fS!0K`Dl;;ATSI{0ZcfIdJTrb?}8NAfL20wiS$)5VZ6rsa{`ZdoMrI_r?~UCz%ct-pxEK~c*6&O*k}GM~hD|-u?(#u>V{L&Hma^to$WaPCT;r= zeku9AQ-93&=7H9d;<8Emx8q?gOh1_XSzstdLOC6&HYL&b0XHn>`m;w-TctnN_d2{< zXFL8-UP?6#?L{wEsI0F}bkk(x2&*wFnhbS$s=rDRW8=QFI~3)@ZO_)cE`6KhsV03Q z9>TeE_U!NwH|2 z;2=OPA5{gN2L*mfDM8yE;t8pST*hALAaL+PBC&#`FttkjN;UgXJ$x&t>_M*t73+#u ziv<*?>?HyCE2V8m)3=H4mU?##(A4ZvL@wQK-5IdNqCh4aCK@0VQ@Wg`2G^Fcj!7`| zJLf#1xcK>!qwfbhKY3WPI%^;IHx`eccTOrB7S1?HM3ra(8$?WXnB3PnXo8^U`uNvBemYp9TM4GaWQ1De!(R<*FU}6R!k)9QM8F{g z>vmsst-$fZ-h%*sUASrW6cP!4JXmRqJ6;+$(D>rIA7-YXeH0gBjSAT&bu91Bx~~59 zofVs$w_s582^czN0E$78QAZKq^JOaFH6Pi?c_L@ElieJTHqW`Pz9c3!?IjrMxPlSe z$HZr5-6ni)S;{Ll#XCFef$SLZKdOJ#m^WIS7X|NYuHmu|^YGw|8v^lxA*?cq zH2L3Uxw;(FM4g<>oMw-Q!t|efk zq5$0?snyr4bJ^&K5W_}wy`#D^&O{dSpks!_#cvhl;*Fs`?8ebeN~LxALVYhQq}Fam zJouLDuPZiOYFWHqi+J824@zx!}(n5rNx$xXe z=LWIxb``BgSnzFTVUD2PUk#UqmgIi*4!ZDp9gtJz9+7D92J9fj7y8HqLp50r84IhM z;baJmAnfkHFpg>HP?zTh7W;)ixk3}5PSBz-Y~1rqA3?`>j3sh(PIliZd0illB2YLc zGiU47>Crj13P~KmvUu)%k&soWs6(aVM61vTwHmO2YX#MlY~^*J*qWuPQi_QrK_lL!((BR(&Kq zT|!d5x*Y}CpL^p8XlG&;^kX<_$Fa6Awf$Z4;h#kYcD~K&U$9bmw*%r0&aOJM*ZKjA+WB^o#zVtTV!^2sc8x+#DoCt zedXTjqQ4SXftYHdo;G82*lZyrVV1IdIaOk|gvK$uu(J70RjgW;-7Cr>3Pq>z*taKD zFQByVg}BWPe9+SVLPr=$cGGHof03-jU=-j-B`29WoFhdn47sDu*uMKO{~q!JOKC1| zw$&b%R@dT$^h($T<`>i&Nweg8Z64hiql6Ukp>nx_RY;D`F=O9WHI{RCDMKY!9Y@Ep z8>bt2`c`DLBOsxKhrk_8Wy_5_>hWvcy`U&!4P;6b(yR$GuxLkM((KPP5;<7M7mkQaOBM$Rtx!04 z^6*cnD|!n}oo*J2o2x=dV$Xh5L_a!)y?WyE;;b6s_Uk7XFMLt!e!Pf0X}`SvR{a@8 z=s8qTdngl$KZbO(rmR5*!waPoD!(5Y+Bt~!qq}Y1!^jB6XNXr8O)9v>&r09zM^i6G z?ou%^(uCHtpt@w3qlX3cCQk0O`X+%$M@EH|EkLF<04@N=5kxYM93FV=>j-o%NHQQ@ zr*M<%XJ9S>qA;bQk&A%r#)W>;r9=pzuBHSqY$X5F12TP-bp4d9k!P8DSnQuW*3M2| z%u#>v%ILz?7jDZ_FWE8;F81Vr3z*+bpO)%b<(YkEABT)Yyx>MGqJ9#!*bC+<%W?(!w)1XJ_91AI*tDlM>VhyZ3XXSp+Vqk&8TuLE_UTMT`h z$x7oz;qE>S)Af=6-ZsVn{lR-45wOV9(OZ4!2@eFvIZh6hj>+Y{)y#IGoF5%Ch}uGbZ!ftPX#BH0yi+)>t-d z<{C)$U}7_6Mm*eHsk1?cj?>c3rX@d=*fsB=nz{3hJZ)>gjLTKxG{M0|;Kb4bW*bjZ zgj%q(vkP2vl98Et5f)` zZC1Rv#xCVuYO<~i3XMSN^>k#{>0O6A);Gx@hhi2tz*uCI1+!gjW3<*MAAInWKm4Y4 z@W(&g`r}^vS3}Rqz~9CumT@@T;9>2)o%AfHYG@aHw+X6$MF~4Z=MN4%32&}n(79>k zx}$S8ES1VR?#}*wXY92B(%P#-Te~y>ELL&JCHDdBzZdw0Hi*U72M1dr%Vl#)wWQz1 zI7~;4&0+M;GbFoif`-(?yd9KV^M&n3Sy-L;ZSuDHQOgchps`Z0exlqnIXULEQY$Xg zXJ#dnpxUs=sgcMSqMg+C09PvGIi!nfQjG^(6jp{7UNf#K>i<7`Z`vH!b>92FKC04p z=s>;}Sw#S(-K*P=GCU}f9Bc*2kUYN{L^&m2 z9>YKYw>L}@d+XyQLqNug0;RBZIG-MW8tVcQ)|l)C5D{z(=b-?yamRF&^o3c=@$4b6ewV)N7%G7jqQMk=}>#T-}Kz{uyw|gi_!UM z<)a5R2avILb@@padtAIZhd0QqN`)Zt6s*-+-96{wF-L5d`K3=>9_E{MaH-KOETzVl z0q-~S#CnPf#_#UGKHj9|N6wLzxyF8YX*&1SCssaXzGoV9jGaqgy7$H1hRXNj{OCFX z)gxJ`LF??-t=HOaso(x?KS76PgYi}1I^LGc8XsfnwpO_M3#{(ypPvTUH+Ht|GYWt! zH(tZ-ZYVeLTtg@Wvhc~Ah@j!@>hkc(#-DysWBz&jPdb@DZ$J=$li#$X?SHbr{%ktX zYWweZI3U1HvBvY8%fsmgfsY#)%#3}kId|kCrwSCu(dE_FA1|O0``YSpw2EBPrw<>- zLb_o_c|bbDpskE(OlKj%cMcCGh|Uh|vUP6VewuV-;qht$44_FaeOmd4(e4 zi|_R>iR#XK*3&i-?GsXAEaSMln)C5~WO9G>r}@#nKmC(v@cy>d?{j_S2aATz3 zKe>3be{f8Nb**s{^@;sv^RIev0K_Sv3?5iU0k5&PRY!_zMea6yQ+M3b7{OH``WIUx z8pcmtUNAGy(gD`*g{l^kx=Vo3XHBKE)8aD=0Qp2T4xwze4)}$T_LmS{7&M)(q~tC{ z$R0v6o^bQphd|}<+kR2(tuCm#lSFfWF5ZVz2piiZx>)i`;IPYI15i98K{>*%f^j5SK4bE%Vd-B_YmCNCdmFAHO5cq0@tiIm`#O!pc zaYQnV?Zj-n4;Uaeka~(OQGlk5Bs9&_xL4PlyAaJVZAuRmqtFPN2Zh&@^|l&+iV&Sd zj2;VQvqO2Zk$uF-2n@Q^)$g-ruPuwvgJeuR&F3r8UiR%_kG$x9JGbPS$Wf3Z>h^!6sJ7DoB%hbM77Wh=YsW>qe ze{qNC0Cg!Vy(ZX3Vsc8~$n51izf?(M$gq){Kj5?_wLig1yCehuH ze{#=>Uq`(;zQ<~BV>lib7qMCY79nnQ;w6Si=G`?0ddgp=%nNRUf{)wNdy-#|r^1t5 z{N#Cc3E&fq#au((OmypG{*z;B#Vr-ZicVjjgT!YvsspJrf{~&r7biEGCb_882M@Fx^3fNPe+L_c?sJt| z3xC?mI#5q7%W|TbKp_xo%cby&R6-A98+iIfLzc?cvJ-aJ(vGq_o)LE`+-ic1?cue_ zc>i$2 z{~{7YOeO?_@}UXj+#Syi>#DtMuf@*3gTG@(rmh0^JdRawl#{|?2iZOTLaU4d&lf_ON`y>vt*Yg zdCA4s4Hs`u=fw(_4QMSYT-LRwyJ=?V!jPybUt6*X{^My_z; zA-RZZx+$R)EJ>>H75yBu*9M+bb(F?f>E-e88PXLQcaHj{hLTb`yytGbDjo>+)VFO4+i^YlRlte@yaJ^Y8zTW&d{r-^RpC-~h_Dd`FKu=h3xey!+0zsbNe!}0mBN+gLcI6r{$B?Y6MXA8zZx?wk*MlJq7erL zjYRg3HwgyeM#H8kBo}^Cyibkw*;W zYWCX7y8Keh4?lS5PmU&6j^+7z zB7Cht^4X>MqrqTlBCdGtiedpvmb7B6PIg9(vGFc|pA6%hl&Bg7b_=C6ybs+66 z^b7osFXxA?O*G0q`ht9?i*KBrpKUIAOYs|pl5;$3VTT~a^9rYn>u*i>@KS-_p3UXo zqKq~YIXLkbI9y-QiR8ukgV|R%{KMz3uE|aP>QsyRifgtl=r~wH?E9X>4D4jjSNFqL z?8(sn6X()ub@|HUU5-Ddjf4{4yhd(O*oX#2x!gtoupc|85h(0lMwF4zaxEdt{IKSm z5qhA3*^4+J+yJs}_%+dOvzQldavC(VbO7ZXZ0E(~s?8`IMuKCb315+CK8Rg@@a zYiv9oisgLsRX$P9%T#QB#Rb;p>S1b0sV{d)Tp{CL?z=Ho=Ni_=0-dAY=Tt!pn<2J6~)=s+? zVz#IJrE$v&D|XVMvZvO^X$49z^NH{CCd(>>s{(JA13Oyw`@GuQmJg4?lSReD+-b{M(HGKIeb-4`2NI?BCa# zT^$3#P6bgqF`+Bkq+E%ZoHi_olqe?dtI7+7hbL499J5MF!sI`e*T`wlTCu1q2Htt0 zbQX?l$qHc z2UuI$EyWzALd!p;$8M;h_!t02j--Tj6SSGL!Rq0S7L21^7?)tM4Nu_^19DQz9!iN5 zj}PVtq-S}G7PUlJcKECifR%L9tujmtCNBo=`^hD?Bn#zxAYI5~m*X#m_tiVv9i7tb z@`w55`I*jB>Pw!j45rA^NGGBEfAlOKz`x4YsU#Py%6!qJp`VSjs^tTj zF!#|k*wwji95D9AKF`$Tv`a;HyG;T=sMR47hhkKs924`NAD$C|*4--Ja7IL1Tdak4 zG?prE&sCG2Q<}MMWcD96z8uZ9$+9?5)Oswhh?; zhfEd@(@Aa)m_V{A6G59t$BI5Ej-jMJV;%5+{6q@&gqv6+0@fWW$l~ATnJVb5xlTh+Qapt@lJV?1Cm zHW#3GEP4Ys@AfTx8lXXUc@^^q7d_1G9dEtf;xwROjKB zm`DTkC5mnpru(JtBmC&?rw?~O`8*+)qk&cbif#V={#mbmi``9`f6K=G)tckFfCbF| zc_Zx({qYU5&;Rg9`<-Tf`0Q^VP0d{RZ3csa*!y!dEu0v3Gtb&Pd-%6sJp6nklo2mO z;QF1ebg9n0(L-#ld#X10n`5-}ADfzRr@cOUc=xsZ^YJ+smSV#cK<+fX|q^_kl|IY;BzJK3^{OZ#?pTjpNN3)kF`#+#IF5P`^|7E@DNXiN`6vpCvD?R-VC4z`NMA-+c#*G@I2(TZ3RF1z|yGtc9sf*v5Rh6lGd5BW$ zOeZA1V7eVY+4!&H%X6h+NyXTR&8a9Qql$WG=}jd{Vmbb-n?!%c45Rxd%_qmfqnep)h6r zmRn8+X{mx5bREkELq_a1pRKf&JiQbLl&Iw;o>H8wzzlJmW9A*Hk(qQ&9xu9As8a4` zRM4ts^=#EDg}OsW+T=gh5e8_bfc2qsxh0)Egz4OT$s?iv{^t$AQiV5==xW&g28yhI z6L4Fj$apw`fIcPT+iIhI_mxvz8@Dt&G#Q<9_KIn{kDR+rJ+-mZXW+GhFIQjOXH&^@ z7W)WIWH1*x(>jxFD(P!DqmA9BjlCMWNL`LLVpo2_RZEkK^vSr9mT&=-Vh%L7Qe39X zK*0xog+0RUwO3usO!w@>OHkN298rpuDT>(4!@F?PZ^rmVDv|`o!v@(#hqSrJ;<&sr z!hf89CL2(~=Z?8B?USblJ+{sH*Zy(U_uyj8@z2YD!$0YzBYH0@; z)U7jvzKx>@*5a=_>x z5$6R;avj2PK5HFC@{fb_qj&LHFKY4p`9B_C#}EI5Z+q?*42evi9KFs=%i?hjbD1CY z#~c0eKWvmyx=0Nsj4|P0@AJ~Ozva1T(+e7pkaQNn& z%>F>!UQP_(>XwvdH~tdq{6S3e)h4uM$(Xq+JSyHq$q8g(rhbo%<9-d;l?h~2D;bn> zRk1vhMU|IgE{g!t)iDl({Szy|io1^Kf!Vw^oP@5`T?K#Vk!^Jap#EUSN=oQG^H)Kx z@2$?^NVs0VcR4q25|TVyQ!%HN2lUqscNPFs)qtD$tO?ZLV@THO7 zX*bqKCvX7r+XM zT)?D{i^B8<1)n&0ZL8ea@%8xjie?Ocl8Zi1{;9tvq@5nLrL6)|+OlcH-jxXsguxn& z(4W66Gwkd@gL83N)Lu6>;rwG;Qxc%JHABU!SJFP^)ih5ntRTvY_tvyQYdK+=#$nn5 z9Ot`cKJnxKQv7_$1{yeW7e8524Dh9-g$r?c%28>E9!iCFpA6h7Wi}@I{t5CyEz|II zTi|BZhE$|q2c6+s7Q5}p#A_9PuQOt>o9qw4;G9X2MIrxbl$&%yj zS|KxuLdj5+Ec~r?#CI!qjHZe@Eg+LYrp==d zjAcH6M{lTHjS3h*`mi`7Xkoh|ujtkt)CM%=P(bHz;QznIir z%KRP6?$BUJDN!PTB`z#0!Nav@SV9i?fxC94_!GT~i~;3-Vv!Tv7mKmq&^E2cOd6o; zPJE%p9c;>#RKe9u%}{|L3zVfZ!_#I&+^RIh5T+67|2idtiamX^sjt9%g`77nTk6Tj zpIpp8PFdeS#!YGztHdv%K8?DvTJgzLffi!anXWoE(Gl=o7S4Y5S+~ghrn!7tQZy!U zlCO#5TT!#CJ|F+Wi(Ve(vQ6O`6b-qai5k_7Erlgf`}V;Z0m#HZeBpCPG^Az+oZQJqaN|EWtuAfIe#0nE8=Kss| zSgc*%Wp#?7NC#5jz_VxwKhkNfN7IQ%9IHnU)dJ!i{!=#FCMgWK)%oYpOt_!cMX;b9 zQdvtE#OHn;zKwH8ha$?|3mud%9ahOkAt>kPci=P6}$@h-K@Mh6Fm5ASeT*S!O&8`$U})i>oGR`^3K){31iC~kBS8`p}(ts-SG;NtO@VNLMR^j~(u%;YG5{bI&^x*@DA z?{J^FMc;&4Y&|YdgocKjeRnM&khPSfwKmg823_q9)^n+BMV|IRcegL)haP3O-*REY zFTz37JSpOa9clI$&gocoXEx&0VVWl?fyl&gP)CC9>JIlKsZiUt2?UBF>!@gQ4DzS4$TSe)`S#xDtp5^D<+Yv>q4oNRe2o;3M| zxePlq4L-58PF*E5E=V>1dzH{ATtsLpYG@P&PX{>Y{BJ9v zSq+R*-lrj=pJG?{}_^_3%k(V5#2P90I8+F~M0aZ(n?)Vy8_)9D4Z=fcR7aV1{tc-jyRKp<2R&*!XFJ6buj82%fGwQ1^7S6W&%6a<}vj8aX=F*14+6^?Y2F2=wU1BFBG%3Wmu13pq!Pe!1HZ1J2=TnCX0yK_M6If zXXNH{%o=3d%-Z-Zlp={?;p0seoDd5^83fA|fjW6-3U14+gGyr;NG2}RIm=Y~u*LkxU38vb6Me|N zH$|jRFln%Sve5x7M-;C~cO2VN~seF}JGBDQhfw`?Qk_cHhxm%A7*>xNtHJN8XC%^-LZFXyr`F z6IerW*X3I31J4P3OYOx8sT?3Sq;fg?&74ciDYihp`tXCSyfD{`>u~PwfHVEeJ{`Zw z1qC=OYNE{YS*s{Y;%8J7rRx*27>sFF6-A8;=YswYDJ+$&xY(zwaIZ}-)!mZ?E%2Qo zZs{&9y8)45jSW^!mPnheST18>_||J?6dRC2!T3DfTt zVLm_GKcQ|1DiePI85Y*#EZw+=IUlfr@WYZi`B~sm*cX*-Y#tPGCsj2cgw=0r7oW6& zu=&1SGRI|F4fhJ~kQ+`i-lK0;a(0LvTd zJpf;}px|UJQrPs{Tf=uXxchk@-(>J@Wm;>#mCoUaEZKRxaZa)mOANSlj$a#g@i_x- zJ?8P7k!R74?9DnJaIKO$XcI8VpYz1W&GHd`b*K}P$UJSmVNaZv|0&AxwcYx z2OfGutf5jH4Kxl>%YdWJrrkT3(oU!gf!ajo5_QdLhbN2f4 zN-R4xtW2dy z+Cv6cbkRFYa0hN!qH#Y~mk58c^J@|s!jN&M?YMJ0xX~!M*u-okFA9rMgtzr}dL7c+ zobcDd;QJMdr;92NdVtokeuvcL$WAQ7D_d84EOISiNBRlY^)8rJ*r{Hk40+CdnS97i zYzA5s+M7|x@fc}Sr-;T0V)^MbB=IYp77!%|2o?qc+?WP@apqMU> zHCp1;s^T@!9C%zE(jLzAbg8iIMqTOm`1H+R?-xzfYewI~``nfKP6)RyXc&oE>h8r3 z!WdpWttYw(oYdOs4(da`^Fq@-3YKSIVy&se(Lk@b*!j28KA1Fm53q&2Mjmy($zOAd z4d-vpO`v3K8iF-V2M3wQqc(Lb;^wQEegh_RdHQ6dTN zfe&tIr+e0Yb0HmAODB^Sg;4h{&7k~MdFC<%4VolnaHcnxGwcvj^#1+ZeK=0R<7#%# zoztWUTH{E#T;yXaY0#^bxywOUHv2&c5;dXnyZK&6f?UjHK#-0F-78zmMgS%ZX?3`m zUUS$xqp2DgNsB!jOK|#;(MnBtB@*KB-MY0@r9N*6DH+Z*jBN^o@ zqNPkm$evVL`TtdQArhosaB5pZX@*zVdd0Gs-O@%!ujb+e9%VJh6e8Ktlieikpq-F$ zWEmB;*R%i6co09RnY4*+L)wM#IerzQNMO+?Ct4B#v_4%oo(u4Lvrt539J;RUAd{dv zb7cv)oUv_Byj&4W7Y^GYb24{eyL}-~y+8E(Kcr;x&8k8~jJShN;gI}Ezq9=3Zr%nX z2eGW_ILGNxa``Mho=tBToR}U&1?uF+4FwDP8(+HEi4{#PSf>sH)qSYNWBTV*L5yZ4 z5nOI_o_~j(hn;di+?Mi0=rn~|p*{&0Gts2Pb)kIn*olRk#|%DXD!=P?) z{T81U;}^4%p9W%dHPf{?mjk+3uN$Gx%@RG>AZ-5ZP;R}HHMhJ53W_!IhCGtgmszLq zjN1n&Vi`+_mLXo_ZPui4j_8X=?gocX11U?g@ExQYU|y>MKWl(!wK{4y@yWpO^>FjR z{5`qhxL_1b6bbFWoiv1dV`D+;3*@e*c|!zOcgNa<#I-fj@2fOsG>qz$%ZUi$bJuwC zNERmdDhemoK`KM%V+}+Vk`C6^EX8!tv6Bqhx9$N=>`&*hVD8ArI2t@x97Y9#?ck3D z26&LLr=wxq8rEtTY?L+~`~=P>eU0oNTynM#`7W)NG1rXoaQ)7(U?CPBisx-GtUj8q zpa<()22IZbiqaU|GEa$YJ4RCwFIb{0$bd3d z(PvBwSj8xTma}1ai27Ms5VGwKGz4;i+AS8)=+;u?&-h z#rW78lcF7`m*JwXUtYrhP$swz!0z$N`4_hn#jD1tmv6_IBnnd6khT#S)$SYdi^c>W zG)D9=tX=h92t`biLn?#Qz&fK~1Hxon(4i~RG2`V|dzrq*tkP@Lo)TqVz_RUHhkc}Y zk(@u^Xk|~B{5RTG2m>`pabzt_>Uo^s!VbGAify{VdmN2jlO>40hqNswSF09~wD$?V z+rYn!?Ix1=aX}GwL8dEtBBC2?rKxa3?`;UD>VZ60Sy|c7Fdf#5q{#NKZ;BgUDWQ6ojX!%_dU@jqDI;k~h z5C6^;ImJ+{+RKb5)9$6wn^E`xh|G?0TpVAneU4`vb&h^}&U?Y+{4Nu|$3Y|-P`W-o zlRlXOys#{fZ?KHbDmt5EYf5bry7XF?Q&CA1=vSvW^y7)a>Xz(?bwGeC(9IqzIxCoN zZ*2`Qs9)`r2AT!~$mwN3zm*A$)_I%$X%4I2u^S@=YON8c>88d@G8IOUEK%n#tZX)A z!IGt+5tKG}TRz!%cWjz;RHc+>B7oHtxE!-DBgt9pyfmlcDB-~!buQcE0G-2%^$oY5 zbZ7<-oFv%|JM!MJPG$&}f%9X^&O_ya!qoQK$=vYFi)E2ipqWe)zhMciY6~(Bfi0M{ zZqi}Ubt>ZO`5@qsUq;(XSgw{Jmh>Jy(VAv1q+YN2huh;WT!ZaE0Za956G)RLqKPuC z21h;4^kzCDZ8(=@7IZoUM2RuKba~W~@DbsF9azuleZ+-hHgpaR8;t7`JtJMmIM|j> z`ExDhohgVYSKcV!X7VLELl~+KS7<_#CQUW+aOlW*AC3kR{75?txxm>N^1GkPUBGJE z+%&5OB;;SNkq661fF5cRbO0yHXN~bodrOxZeNQ#=g#Rl}sG!U|=0idS76zxUDNHpb za)?eAJs(amoJ-t7uoT@p6)T*-kaF4GEk2o&v9rDR&M!8t0(MQ?`~4#>AQ9*mH_3>2 zBRjH4sMfi$Qy*Ktd7it1bGyJXmz{eWtN9E|-%ic|fWJODKS1-%M?n=i+Chc%yIZ4` zALU$!O#0Cq$V~(#dkHj?U0#zkQN|$)dz#<=b63YpWrb-dq0un*^qBi$&gNIA(ycwb zRwJ=?yjK|{8;>fJJnR<`zZo!XULl^dEUUs|+j2-%*w009fwCF4p514AP>{^Ry<%yU>1*;sG_^LQkrKS6}}V2Mr08JM3vA1bY{O`)V<>p zZ+>~Gj;xKlhIx`59y;P5&+mLgguM9*EbQBjBj%jUSw#nK7RiE|NOS<@@wPQAWPTMv zxG6TAb7SYxCl?ev^!6%x9^ygYj&T<57i^DId??nK-J%i{=#s@!@6Q zN@p+_I{T}lA$QcGj)$MN1-O|aWnPj_E2vR$9zG<>Femc7s;(&6m)w1U1?ba@UNUIT zm~t8Pe{ zxH6AyaLLo{q~C=Y%PQzd0$Q<{fk$^qt`D9by9Rqs4K`p?t_CU>q%gsxGPZv8X8-8? z$Eysdzl=Z0#ftqfhhEu<2O}~LsEixtQ}TND#~GPB!ZFW75WmW_gVCFXP0HTzz(XZB znDbHtOBTCo4;PC)?3C)#7(J~EJsoLAuNu-JxjM+5#!Bm2icRvVwjU=!87V0Uw4}pD z$ssQz~FGWJAnAKf? z1W}H#>TW>l4|s5fsco)lB@|Qvp1ikC-;IbnvERF!Fc430W114ZvGe$Ee*g4~-M!u4 z4~3-5*H`D~l6&@(t#r{X<(+rziA0CRhuj~yDtVZjG`b{=juSOG(eGij?AzCtcou8x zH}+GO4IWf6*clg(fe5`992EPP{U5+GEa9GK=aj*->ma4QbvUWL;~P>S!}Nn@)VVfh zSt@S7h;0zV((Uq8M_hstUqX$cygew!dYeM^w)t>BHl=LAA%C%BMGG!-+1KpEgd#DY_yHfak{;Bo$O-A!Zu|mVyJjI44Ud7R42A-Q1rXPl++ao zG(%tUvT2&X9C&&bhuuJjO3HPobcfOLc%VzCRP`zjT{oY@iZTN`jitDa%nVICw*%$t z(^i7p?eG&nz$R9147QHTaG3`w#3evv49(AO+}5Lb@U>k_r`=M=Z94D@7q6(&^5wr3 zRG8!2rWvxGz0|?h@YYIVY#If*jyuswN8I3l!yNdFoepXv??VC*L)uRWvED>)gW00m zwZ{pL+^m9M(Rwp`h9Pu4l#y6F4rLEp3Rk0hVTVzYS@52xl8OY_&B^1ECecnQ#YgG$ zb?YkacHcy|qRtL8hV$3T(Cw{*MR;l{uH=_lAVa0oK9mErnS4Eu0mACfX=JIzQzkDv zKY^gyATBRUo=n}4Y(5r6f6&ckOf*y&@p4$mU=>Es^=8!+iY409I-6_NZOrL@jzMm z{N#aYN6`C8On5du&6Zw5Ba#OC<@W$jKdK)YC$$s4A1^*{6WW=J^OF&|c*)nEGPj!} zME*=kX3-FKKyV71gIxrDSv-|u`2!?3wRLXe2{4i|`ht%9!3X$o-q`}Q)Bicqw_0b&oO17dJ%zi|kK##O0wDsCw{H=F=Q0hYNuFRX>ZZcFdisvSSI^tV2;h=|rGPU6sF)&q^2bc6egU?xcwzGcKo3=&dvtqND zWTp|p5e)bG+S?(okwuH0cvk~OMFY0l=B|(a0l8mWMB6kwQ%8XC4|JJ|73fJVOc3U% z48~vUx&Q)&?AkoYUqtY32ddokGDu1FUtRq>K1eFG=FHBj;{dXSn77EnkH)o(TZeR(T zc5zmqLygsjaa`p*EaR5HM3qegD~KgTCV^;tR-66 z+&y!5W2gTMjt~^U$;L{x^yU$YZiT}cB`Y$LT1(S6XcL%iDwk2_v?t?jc`p0i&Wt@# zxtFs-({Wg%{oi_I4jVr+6KL;qWJ;Q?Gg}w&3NE(f4)5H3ZQw}YPc*0LPig6yrFLyc zCNIuW7yR=!lL+^m!g|(1$tJ`*njD=@aP*Y}c1)hbG-p}iR``GBmnwKk zvgIi>3{`bVR~hA$;0;BC#MjJu=|_ihZODNN($_cDz6b}o%PY@MVV11SG3T|CS}We; z$`%yuT)CS3C`r~4PLDGLO!#t6c$!=5_8?y6_XH!*S=2T5@fcna?p~I#l=g-eqv}e{ zKDt6p$Ba`&={ODywngd@55tAL7nGW88!Gs6Y+|pd9;DT1OC-RkE0O&gc{LuLL<}>$ z{4gG#j#VZW<@P&9xjb#PMp=@!sy!}GT6I~biESLq@=X4NN}!as3NO;Sh~lPK33^}} z!Syf_gb`Y&wJVWCf{mB>gvbT+4)?964E?K}e`ib_HhUQ8Kv?YhkM4`AiQ+&qOYH>ckRE2^OpLeW*6BTWI}mQn~~_9 zsKEh$n^AuwLY=_j$<5Jm^L$Sf>Oc@vsMk(KEo^A9VfRs)^&AQJ2b^KmWWo(434)4p zNmP8^9AdEwS1xIuWhX!i&n1jk?R+G(RbM2ZHMu8x%`doIsEF~7Fk>2B;3Rn@ZS~9o zeLJyYZqL0TT*CyD7_cqJiNgludS;JwBZPEQuEyaS`!!k2;C zM3XUs$|dOE0_g4u>6++;?DLG2SdLo)P>;Ny;b5CP$L-cYQ}qn8YrO5Zr`%6%K9&C3 zHR;k~nS-G{J!<91>9mm(gKn47DY_Y&>Xy--msZ2V-d*O?XtaSK+;O$O4kjekqdK@d z6N2it*BNyUXx$QUzO7!P)w}@8R}iLLz}Sp|J+|g3J`t-Jux5MBJjP_7QDx<5tY&>pD@I)1f}@4(D$ z%{|!A^*adT%`vIL&OPzhA;6^{!9ZIYn*SOc+kPFY)ult{wmZwUY4t4pMP*j(&+INV zs@z}WHyoZEXfEYAGT1_SACPo_2xjNJNE=%i!_6EA@9~i|^&5`JkuVo}Z)4+rdN6Iq zg42TH()o9sWt;L)QpTBRNb&ZpXf}(jwaRo;&%+PedqBpjN;isHX6W^ZgN9qA!K4u~ z%oB4t_Xi{)y%Bkr081p{fGRu8 zjwwNe_A-@&(9wsLB<-KTiPbEhrQCFi4N>820Jk|mx|gHg+aOZE1-IY0fB(sC5A?BR zox@5)y7XZyF9pWjE?89L=F$9wyKkMWJ5^cAV5-{=6a&a0>A1r=N+IO*cRk9WxJjc3 zS2OFNbGz_)#iKe_zbPJzun#Y4QwC05_DHD-tk2^Iy;I!Pb$9jVhR*n>G^V{#e~v4hk^Ia zvIFq62`bXsgLAZT;viC^mlMo@=4*tiMcKLojR!IN;oIhHn`Hy~;hYWP9R)}{*?=LjC~#oiZe2lJM5dj17asfueOC@asf zm+bd8vip}~5cFqV)_F2xH1uw3T+=l3EF4u4^y@QqS`D9CztR}br2~i3J9cQ$bL7ma zpC}sSu=;|aA0|Sj^8>+n90}ajWM`}ajdJ?&%HxjP9ENOlQGT(ThC}zTwE3%Z?zl7$ z5corA*B6|)?_9`%mO2kPJ7-7xCxkw{+Az>Hl<8)Q*(Z$J-C`DKdvPJOTFFG7@+0t1o!7Q@wO~rPWS=Ss99Fk>ZUagj7p#^ZN1@c%Q z-xu!CTpbm-TZvUusI)G8cw!EgbmicS!R@|A_a5tNqYBn72bFMEFfU%r6r2-lV>vo^ z1?9kyFBK}v_RGeIyhA`TDJz$>c!p;u#X(&%f9< zHthfZvK{uov$>lW9LPOxfztpn^0fJ!&Xtkkk<7$;$t=m)uq?#<^c6B`8G70qGVw!% z^XQDbEmvwTYz|_)^}+#clWCIBoPF+-?e<=lBrd#0)NhoecVd#Gg71xt*yQ1xwWmO{Cflk9fd@oOPKx3 zgE=OEE*P|+B@WE2IOyzyCX-woEn_ZB1-ixPrboGn4ea(ecUyf1lYI*^5(1P8Jf^OyebJnTxac5^mf(!JzeIgApTdYpU%-xk(qh zbC)_21L?5;f+@%3vK_uzfBXxUhtiDoyLMx5@O=I88;Pu%f=t8fcUr=_tt&WPgGH1Y9-_>iYMNXC+}8rKO>IfceX+8sL$;mvxfQJ^YYwkU_M+|% z+cP@ayXL?Z%d+0A=gOuK_io8U?mU^;C%f=yt(=^2QH|7`ZDE!U#u8`^Qi5OerLBZ! zr{lQ(*@M>*dDQYzC*e}U)4+1?HjB^SV)}gu%2w94f;8+>Z(_)@T@3{!@}_ZMVbz`B z_C?FH7#bh3O-y>^s*UDrHxzNj%og4&klo;SQ~RuOoRUVAvgQ^wqDaR^Ab2HZm?uNa zTdB$o7CC`29T4&$RuXm?9IIAFRnnRkgpV@K9uu=x&18gy|)M>;i()jT-W zi0N>y@GVccjYZs~EyEJ9?^zvwavL|SkpX12&LBdm&_RV7+FpslL@j9zN}mUr-Jj92 z;h4R71)ZoSJ%i>98H_E0Xe@&LP(&}AYT0tf+(w=XMSU%<5XUDXHubWfX+at8wX}^?Gs5TLp zB?MYYPgXUvTB88Ff)c>8p`wFEL8>u2#=k{(hwswKzEK_O>njir#iO&fy99Qr+hF=_ zx$BNxsFq#kdgZeNo#naera@Ndx&_TFb4PO7G@2hJ!R){~a#`X;=a4~Sy(&kjYmLK1 z*e0BpXqtPRPMw&Js-pYZn-|g(#4ES^eeRi%CK%UPiX&wn1mH$L%zJ*iZ0&KTfyrXI zQVM*_fC;Y<{V2mtBwk95^L%Gys67r1OU_PuD<(Iy0-Vs;y2$5n%9dx~nnEf$E8TDg z>5aUOCUdauN*Yt}#>~m^piU$=#7T$Ci%Ca084B<$g^~kDPKpS|0VWTh)*Q+znbfp? zB3CyL!D6>wpN?ZcTPuixA7Q6BsgbWv#{7DaN%*}epnUK(a{7@_|DJv8I1s~dvCYM1OR#r4cS z_$0cFNS7;s=1w0qjY1u0TIK)*f3v%{_to>4dpKNvAlnWuH2diJ*WZ5i{Mi?aTUOvH z*5Ny5lZ%E{P(ZI46^_VHFE{zM_xkJD_|8iu8xtAmoJGNL1OO6>N99~P9MdX%DaN(M zG;%>{;=_16lwTtd9G+_ITO)MNn`P1j3WeUH7OLdQ;M=zFE8T>p6NYPY4np*LaI z%FH1LN0|~)hX;jw%Z+6S7aCQ-SW7LEs{=^F(;m$O1PJwB79S>!=0+{_f8frj{g84W z{VBJh3>uN<$9nwZ2V3{=XZtD`J)*2x}WHDMQ8{NsmUQ&K!7MX;Vp9u`=f?gVh0*ym-_MIehnk z&a4axBrS9aBh`~BNt9u+vBP=dnbg-~qF+vW9GOXo*0x4^6UpXR1|!x0W*Gn+BDter zcIy!xuNKYO`qrg4pHtKprNx+Vc6((|v(YR1%n4KfH3nO8o3@C!|MKPNF7%~@_CpQ#KY!FT_9({om|5-*~C;W;c{+OG;# z4HAnw+hh3tP;P{P=?2;~s0 zCZA$lUy$~x(Lms_Lw9OJPobHuuK6pJ%D4PcZ*qzn6V06b&lCIv7xBPZt2eW_q z;$LTvwjRwM)2P5D$%h|oyt%%<_-uRok2EjZy1F^rB7;>DovSUbueSeq^!+w(Y=h8O z+xr)De@vWm>kHqtBhS{GYZ5m<{9tGBEjP)YAMVW1;I1(Lbdmw0$(6&kt@V*5BH0X@!O&t}h`qmvmR^l^;|8;)wIL2 z`XaM*SizXWEDivheejZQTcA`?ggp3O8Q!fl3CCRhj5?$GDz(3|74;pVa$zs4K!F4z ztTZjc6&$KTl*1h2+=KCOJ3*I@BWSg{rxrq37APFZN%o813$h#OykRwpNUJ*A+72CQa&2Xwdz$?b^z5 zj^?lDKj%V!?(ssL{VWK2=Zu3SJr{r@Wj(mLJ28e&ra%qeSsMgSc~n)n~(fy@Es@ z?q?7awhFU&UV}{Qc~(I)a%{WhGr1@QyvA{b_i=Nxn(%U4WPO>Ndn~?@BLRahW+Es| zsjz6cF3WqBIO2v8Niwa(8>TCeY&it?EI4G(wd|;$8V%yIyPGpBb@lYG*u{||QP{KB zB;tA^q-iHCEAk(Nc^2v_wBSPA>wOX(6|j^MafmaCn1fu^2?q0>UwKV*XS=v~?gk=B z#E)8*;}DY)8)@`}xBmiT17pDwCedTbEhO1K3g!QDn*7Umrw8XJSJMi8{5aI>J&Ak& zusVx#-Dd1$+jmOZYi`Z=fgVm72T%_O2SMO0Z!o;FQXw6J4(2v!g?pf>ab7RoybLuq z-WrZ0m7-9oM|`K#{L7viG9q&ex&}(lVVVhIW0)d@Tj4Ml7c_`XYpexsd>rc&s*Sms z+*~db$*#Wl#FElO5}K(8I}fe3SmGrCpa5XJZWrZwI?n)S#a*-4v+`%0nYrXSoV9VW`F%Yk*X$PFq;P^#59WWvqsRYu7Zd=`N&*p-T z-tb8V>-^~LCl5b;c*~hLEot2af@zSr^eU@(UVf+qZStXl+QVf~8zw^Rfz6ZTA1cQ* z<+t4vV+>Eqkn-0dRV{igScp0?diDI0=~lqTS%)2PTXuozqh(+k66}KZxX!vwB9%Lu zpcY1_qMP`_&S))Re~|2-o)+Vwx)0CtDs4wibMU}^uphd{%r_6usmE$NN;)QMYnqPN zJogE14fNe~`AahHH1eha>+>74_DBZ$?-lGq=-|fT3Jma<(_#KRUWkblWSOd$az>3o zcee$JUbrP=|AmLJZ(UW0>^9Hp^)#z>dRdLFn*{&|b|w<~r@ej~uOr$=NDwxMf!y$` zC}uNeelUaWeX+pwdNiBzubLE5 zWe0`ZUeY!*H2|b4owKNGLEnziHue0*PW)wWcPYf22flXc@m*s>Qwo)BMwgCDtHdRDjO(MPCekS8Eb z+;|Z;EmkZhc6(1jm{ELEPG#4bECXbJNvL{qSOtzo!&RBA%pU~L6^XH?H{kQ8MJkz{8%XFiMjO=?uUV#3C>?MSTg z!8_9#6+G<;H}G82bcr@24qZnbV)^4ybpx zBVBTklIyz0<^1O4DEn-gY5f1)foe&aRWfx#LzCV`R&N}R@bvnePlJhx~1#z#L)dtGR;kV zMO}hC@P&UZR>Y+gT2Tkbs!tJ(rCmFf{pL}xjnRm5MgIp!MQ@6;ouT zr*n5rn_#W*a<0W0Bn&6JX~Fa}i^bZdMDQ;<2h`A)9{NKXsBwu-dH3bh=g;AdCl@4w z)0O^me%N#&1Sy1N;Y+*C#E&0IZI6>8)jw!Uk7;L5fyW_Sm zynv5Fy;>o`1SpGWz%dufsAg5&-8dLEJBf@v*-xPZP69J;AD}=#2&psnKp8sHH9-X zo+VwoWoEaFF_vA{xrnkX)(F@euSNIEh9^$x+J$@OTTTTpX3w_1TU+@-2f9tkT%**; z)46&_m!4B@Z9T`YL40kglou7<4EiTsxG8jFl*(mBUrKNR7zGIQ{5l&j=sLhiJ;_ZBA7o0cthm*m*t* z!P{X56S1{6XVGvj#8X78U@^uEPLfSq-`E0o^9)_J&Jwsrj!$=7Q8HVAKB_Q)GBvTo zJOPWiNI}h?5x4eMM#WQLcML` z36Y_{k3K_(qF{B?^Nt$2;>>t=5dy2*J4UgZ$Xx8x2>0F_n(EreYJ*876Fd3%YWBP1 z16&W+$DmsWpzRI|I)8yaj;{}aUay!$k?XkG!oR?#w2Lkb;7gJ;x_2$oKR?@~rRnkO z^Geq2?ovgiu{?q<@gX>-EOc(3yY9p1(i-TG#Pz3?`bL8!AHZfB|9p0y!*6%v=Y-y0 zUBvQa1Cp@&ukHDDi`!$PX^eHd!cZkj0LsN9BF}yGKvS9>l2Eu^J(ePPHpIKoqS zZz5+S+ZB^Dy80pej(ngrVMXxcHz6#C>fDN_A?NmYGJsfz3A}bwozi}@@1xDb%>8#s zvEyCD3K}!aOd_GiYBiQx9%HOdv~11slC-JS02Khz(e%U?#`e2nWhok zwGaWXW#557y;I1O5OoJn%&aO0#U^lGqF2HIedAV%`jr>w))^rbuda#Z`wa`WM3j0> z>o4KaHm&h$yMMjUSMMS%O^OcObGa;MCkY;BC&;}6@-#MR;uIe(by69ky7z%Iisz?Z zw$|&snkAx2x}1EoM!8~`k@s#-xF?ZUy@S2bB%KOPx41^{uJLFsISw?tcV`zHWtaZoW(%qE|)GPAW`gNM|yzO!)5YboCp6Ql5n2Ya6iymOu zl+u8M~kk2NtW{p|=b z+-v1gN7OgqV!0s{-zZBZ>hV>YSclEwtM#>_GBpw~Kv++l0(!_HkH5M(k(}wSmXpkJ znL_yPY<_i$C1ty77t#nD_)79=pNn+&kA6Vml)UM&GfIA%A~@5npB3Ie^IX3X?U;`je8pU)W>yL1L3W!WbhESbmiji-i~5OHdB8D)7WFc! zYJkKshdlUv1%$FDnP8c27}3Q=x;D(#LI0kZy*=+f<{gN|3a-5*&+UIxdou^Nt3)WZ?%$lcX8t{j4(D-j_|NZn5D3Vc6ZLyFXX{Z@x=*J&$cr#l7_J-ZGg zR^Ny^Ri6YH8x^Tk0F$)#(~CDdf)7f3*ajFn`tkvfo^2Z5X8gdiAq=~!W7@_}eQZI5 zJRqY^o3ReFaE#xiz`$R{Gw=>N17La9fY3B8mP;Lz_o~tz81OUIw!#!pY2ea$i|XsytLi zuCWsGTWWjX`L!&U5V*lR%TzJf3FqFbz%w_S6S3AeU{(B(6dg!h$(X%6n5j~cGj3i`dv4%h8V+MJK} zQOujf9!5%>V85C?lv=M-h%7lVk!~9CxsC<*gK$L0uU?5PiGoKKfS^nwmEEclJgqSy zyXryG0m$gV#M{5>5vh??Em1c~*n+glHTb52SnwXP&i- zuK-8qVV2fAq27KO-55sttyKD9QRuhmW!9}K-*^GWZB#c0{e`Vck17b#(slaIOA9YQ z1%=wn`YT>;;U~jRF#&Oney~0k_Ts58!~Y#)+?#Jrv42tFas8XJc61%niN&N+-?%1^WX0b>Nfd&5?;Bq~%caGyfnXs-q!UQ#~R3TjR!?p@|JTkUH0aCj-885 z6|%87(=n7}$_aNy*E)kZeyWzgpoP+kfV?8%RGp+yl&uW#M=l;SU#4HYY>E)CId4%m z%pQnlY~R;0G;YUx$Zd=1wIF0CEi_t-J|xO|=K!X)WO{S=fZzQ-{c@cImHb2P)aR8E z+ntWP5a7{v3ll1w_CVVKyL)R88mUAL)3FiWP56|4$st}A8M8M+Or{`|KelbLOQ#HI zG^sR7{5d#3r)dTbbM>Err0M?kWFxWxCzJh>bNK20FPHiwbx9sDJO~pv!KD&nj>-+S z{^l5?69)QIytvaelinH~s3t<=byJ5H&S28Man>sLpt92BXhWcPEv6;x+iE2b#eYpT zkqL%U2bl%!yB1;iJABI&`A*$Zp6ej}+f;(4j`6$as+rCRP<7{VXo*z^?v6EF4tklX zzhUA_Cu*cmy@b^1gm1E^%5|Gu=nYEg#A>?MN9zH%>&A%Gm-X1B69SbTX_hA&$qoJw zy?p-B`mJJqW_npXguevA6fc!X5-H`h^jo6y9HkwrQ4^2#*&=#sB+-gor#-PODCOh! z2;Jk|O44gB;*z~38DLaa%zjFtqPmDQhn73_js3lbGSyyOqDqwqK0TWmZtl{Xf|R_Z zy=y&!^c&<(=}yy%8y|l*{@Ge*t@vF(g`sl) znwi{9Fa|6cLOtBZKd;VN8_%s1_1gmfS~HTXAuVO|;)dMu!*15`@!TDdIkacH{;WeE z+Ug?2l*DtjVhTgwJgBvBOP6O2`Uj0q_GdkKySv@!ZN(;TKLX-^w_WJ7yI>nBjNz=x z79k_{FwUH@^c=*<}0JE>IBV zIeUnWW(4?>(m+T%1oMp~6Y2(9Ce;27YjH&?OR(Q?w=1GU)kRknIWTU&D^{f>ZlA!! z5~|e5d*eU4R@l93=>=h5*v?SHdPg;qMYG+qi_c}Vs69+%E0`ra0HBTCDOHN{-!`h0 zSkrI~MhcVY^xiiFJUzLKp*)nvzA%uN>=_GtScY`YzF8s8Ej60~TrFE9?M1n+`~b3z zNyScumMFynhv)vMeS7E12pmOrRQQ9NZPPo1|K|(IxE=dfxPwR0n}}hCeJ?eo+pDsL zZHI#}hGl>L$8 zv1a7wF&shQb(%@J*KKI=bO)hRc5G7esgA^7I5@^XFb9;Tu|hnDYIn)M%@teKh(^*1 zZ{?|tDI=n|8N!N_Sw<h2UlKPl{g5cYPA-hsgwa)yWX`c=3_g$EJZf_o7}-aKqDzEAdncKrM3pmi zF@M{Yzq^Ev#zbhY&pt(FxkTdZJ!urTARHbyl@X%%0ULlYz}4KFCGgy17=pGXRZmc^ z?;^~f~t1CCRq=8)iQkcaa~JY9Ck$-s5cKgLOnP-vMUEji0d$yH%Tfn6E}r# zuRJ;Hw3CV;JLuwTX|)ovtNpvGV_3b7G~^aHBKSx+;mk&(sU|J>al`flS5P4I*hqs5 zY_<&fbE?f1Qz%^)WCc@eIF3^pYwk$bZ^UfSbGyyv;S>gBhO6Dk%a!sQV&dqGx|Cm&mZbpyo}9i9z~ zR6|un9QRIfI4DLIPSIptkG#v{*A#3+-bY@8ts$PPKjzsnddx22uGdQNdtudi^p-It zTLt@%y@x$WL81Ng-50wk@8~H0tvPC*Y<&0S(@#G6`+wjUp#oMdl{*#$@TL5FcEVYW z70oD`#tZx1uQ|H+_4BWvMPC5I8Zv216vkoOyam=ipR$e&`Gcr&&cza zvH+BmqAj64LloTigOlcXp8&Cum0(;xA6Wd+oQ1Jel#Y~&okE9|HGOmoYtSi28*h3> zUb(uv{dW|d_CX@ZPvEl6yL9_%B_usz zt5`~|R`eWRs;2+_#TP&A@>$%7vb?@-T=pMJ5sw~KVJI55dgLzQ}{&h0_rB#l$l0q-27i(9z2NssD|(3<6DzdJ&k(t$ia zY=StoL4EBhXr6MU<~vjsd-;glvz%6+qz-DeIWx#vHdO?{bHA=?xejKT1C8TLa60fh z749BUhOvXJ@PDhzt%-M(=v;#_$Xvjg2E<~aCy*CWg+D%~8w4**ya zMC&Azwx4Yxif}t7(P(*)<*738Z=4r>Z7!5P{N=5XgQ7Yo)qr>T(iR-5=A@*Qfi8x2 zwGF_~TuQsUJi{{a4l^h69H>Su`Y~lkm)<23`m>i~ShybRhdDddw%Z^- zUaE5L5@ANsnw*A`JKm<@^-f*!1-ksr7vDVH`;Tv*>D}4y^~JLnd(Zd&V|F}y`5!O$ zo_#&rP^G7?3ifqul7$fU)qnGQlWo6O%;6787h{Z_Umnv4RUZZgq8B^pGx!1gl&6RD zigkI=)FR_~ImCS4%zO|9#qOF<&3)B~X1Hi`L(pSkgck`Qi4~+F9VOHNz>i)4l*`si z)2?RUsb-z~RbZ^*iUYHSM_=rI{p=+M3Y_CPwpSX%&R`u`yOkL0F^Y$KkZ^lUd_hg6 zRLsp%JXJ(p`%$Nbs_9tDW0WgCw7l%9C5yA1j9@}0xQ7X4^1#yS%)Sz?0=Y9#=oIfo z_^k>YAv_}uDbqQ^Pz7~8aO@3fow~YiUt_Zi1hGWp zsCh|h2g~&2rrUJWI*H4^{R5<>5ZG=X(&a|rJxZ!Is?dgm6&A*{WfdxGD!DW(`0gk_ zY__DcbyhU;tQG1MaXG?vJ~r_Q@eI(~j8SRT%fT=H;E|${f2mSX(Mz~@746hwJHv_k zc%eK|I61pENm>Y6tm)dOkHkpKsSVbnE=YqEwEP}A_r#Hmw+x{M8}?n%-QFF$?ZyY! z#Ud(WE$}2Ji@r&wS~|RcyZ7bhKh(;U^4PwA^mp+C*wqXHR|BQ<+A$1g zE+rX#xNy8T3TkI4)Sa>oI!@}^45Nzkt{RK?IzMe&Y;B^Ic4}|62bi-t)`C(-%QjD| zELS5rqH?wu7HLefvZ2;284_z$oTs=2yyV<41yp@Xd;qUCmKngtX-QD=Zp!^pUs`wcL>nRuo&cIMTm=&$!>dn zM#~sQTM6^frUwsTso&hRB1AFxwBcQOaJM zmbJunye)?V+jtEzF1wDk?M8|qAmaxVy;KOdx4%Rg#sO@)jg9R zEUrMuHFG(~Sc;~p5&6&t#&@^_alVENm5FXmuUzj$V{?Gm-eY<$FR>_&qhG>eEx-l6 zRRpY41r#Ht2E@g>0@DeOK`71>qL#_28aG(rEl1ID2^D6!(-RV0 zawTJsEqf4_k!Rc}(6o0hHg~QY{q^n1tNm9j!4(8wsAJia4J8`ZVmUJamNukxi-Tx`Z0vaTpFZCn~QQZ+Qq`A|2 zowJ)4l6MhTY(Ur2jyq5oyD-rx#s=roEOq_t>O3`b2jd+li%>}Dtp=bs^#KWXVKB2& zSLTT7I3_xd~iXZ&4Ed&`u|Pqo*VP2+$oiFK7h zVd>w9%k|K&+gq2gi#iE_2eO&AekQMXN z9YcaNV;l6`U+cd6WVB83OUXFR`ieNpghk9`xo(=W&}p|F*MS{U!^QQ>-YW`RYHP%p ziOdTr6}oxEZfO-#xG-9k$I*11TdX11(#KA@SWPNr*2OEFVJOXB_h7CoJ8|44=_2Yp zdnpnV<}K}F5zdfH!k1*D=2giBpr^Ih6)NIl4&-S!Tc=^mJK`pBZwQ(nTEHu7xk0jx zy|6U&jX?!#oxYH+mWP-F68kh&kz;0YbxMd~k70+%0!O`Jn|x=m-S_ZW=QO@oRS=#X zctF0}*;(i_fTLz{GK{ohS1x+6tQtqD3b%wR2UqnDtZx>u%B!xebw>)U6A#uUfdvHnapJX9&5&9;52BH)!V>pWnTW&x^QeS;%RltLzkMLl0W=kjs>o>c6x|$QjPCdYlH%Bg#sDj7kSD0?6m(7A5S)j zP*}U|+2B9y?d<9^-g&a|OSuKN&aOVE4@D|%HgL0NBctreh8B;ZQa@)eQTK{J+1U1a z#K(G$r!w%%gZL%)bvm5LhRX21eqT@GIRjV*XEMt^7>TGuTirqGp&ldwXd-6{Q ziZ${`OFA7%o<*}*ZS%Rj<+TJ4tPBT=Hz!N`J8qU6iZFM@TZe;%eSh1k(8?)Nw*zS< zN42GK&kr{-B2<9t?KmaLWg&1jXr~&Jn*ESl&JD!s(^{AAq>C981F%w@z&hFGW;(4MK1o8 zT<{ZLy~4kkUng5XX!rH_yY$V=XJYGIIHGocPLTQDoAG<|(+s#~Ii%e^F@A4O;?2?V z+x$AF0Pys_3X=DAKm@rhN%)-CGV9~fv6WuPULG3jnr*|1T_0Sk!;k&h>R(M(cMG8Sb)IQbj*5W&GOaNI8gd4*)musVvY3MCIL#pk6-EUAkD(w9>Q3TA~vBR&|-WBcy1<=Yvz2G%TX$ z=AA~3&~1|F*D-$D8ABbs)Ty?+_ar?(CcJ%Tn{jn$cRU=5GD*h=6A1%aWT-RZZD!OqZ2o%dldVS^J1@QNDalM=L^+A<>#IoPtMAQTW#S5dPopsy0#jp6iLtML)hSsfNst^e-1u#oD}gM%Hqq z`^ddrdWZesf#TZqlgwJ^T^u!s{c!fV*>2^2P{U`KaV+kFaHm4K#mI|Ja( z7=H6?_lq1iG^ajj;}0ztnz3`q&BT$fpY83AMjRqmmELqT8SH)Ono!&=*R$^aEQ!?| z`PCv?QI#P|Oe;g$JFb$)TJcOGYg1X&KRnrZ>e}b#9?2dXnZ}_zF7apy7N1)T3J>9v zXTN>A`7htfV}rY8r1L#DXZb>&UmqVM1h68_4>`1d7LlR~%bBz_Wg*c*oZRj2QF2;J z7&l6tL8Dtdmz?4^%kx{62#07CATrG_#97%jU7}x;k94X+y0|24XH*1|J)q25?;?2~ z5@^H7o+BnOr(dyd(;Z%E-UaZw1ix9I-zV$8u|rpy6Db+m@yu+&{5Fjp$#Pz0>r!s6 z??=#a1lripx_N)oKKEKk{ycC44@4+Vfg*Nqe(oT?g+43vQ_2ogc_2y=ia|80=lxyu z)dC4+RfVI^#BfC@0-VzKmj) z0E3S_2uz?gDEXiZ5@e+i$b;nkPi3KBya^iP{#9Jx`s??uJln7nQ_G+JV&=VeBPCtrfs!%rN8$HEZyzo3CP%j zO!63RnbmR4&Wh`)=VuFt)$bq$YJ$G$Sw(^l`FzkMuRE;TMEQ1*bR1mJ0IG>h8RROb z0qt}Ra5}4#S-9M>`^}rE9-BxurD2tR(JPU+ zC>TalzF+rlOIr42;kSv)5y|+)jPom{AZ_}%o9QEzuI1M}eqj%Y7k=Y3@+{nl!%FFP z>w&ndpVQJ9jk`v!W~TLp6+E;7GfT+H(Q3>!%lIr3=Y0Q?KryoO0p)1LN)D)+o5w z5YK4FVOUi3RlE>SN#=~8t475*dPOwnh=88-6GhL#LMxY2ayKT^m27d-SXu07PUBga zY6`Wh4V2);C5ps=_kYxIk*p5?~g%|uBTxzV5n~8&B;h?D*dN8#y;ZzRgH}c%xG-VWd<$|4o4TV`g)pE zuZ1$I;J{x_NyMr8JMDLbvHQoI+t(9q)KJU&T;5g*D(NB55%;u`p_1CgQb4s8j^I>k zV(MEh^>TMn0r zRoQj?u7sjkHrow+@C5P>+L=Hlrym3%G&QQW8$7*32pn04hlG`}jtL1(M1SwiOtXJgIO#S~-ezi!K7n(D_DfGQBTvao$mi42nQI8hk)Bbe(}N^oAyns>G&(E7u&ts{ z8>2H_xZhO@N}LOFc;bddLs6Lu&HY`(Z7R=nlbGgfnQI$6z2M7lTk{jGUQuG}v8J8N zSL7{coPefYH#0Z{XgeK_&`L3gDi^Trrg@iJcpzkHTdtfeZMAUreklnSW{01^O1(l; zOn0(|bD^P%r5jqsrEGB&31g*}Wi+JSb(vF->3bz4*L~ZH&N7)*blz)3OVw+603;FY z->R8m|H%xOiRVy&oG~p`8{uV^aOJw1KfLZiZ{TKglN6ia}hX4l3LWC%hX990{nW_Vytb8 z_(Hd!Kxbek47Gar06w@Z$Eq&67foy>u612uz6k2V!Q#c&TQ24$bIKj0>W zVoz#cWV7p?{X=ztWd21X{`$@3`ORxG*c3G(asTpYw)^e#xM|b*_ZQ!M&4?d<@N)Ox z4KTx+(^V&7%0#g)pmTXCAoi6h>fsnltl46D#(Dm_myrYtyDVZt)6p?&TRsVfSO>Y$ zNx42jl2PGDsyUWv2?k(i_yV0E@uQduHOL;hZ31=31Nlz~8q+ezt5EZdISRu*!+pXW z1k-Vvx0H2n)Tsjl&3f`__&7zzz`$f@b78!QZ-xZCCt2(cP||Lr-HJ7*6iVam@w9_9 zO3jlGPQuRB|4+*lBD3UEE`ib#wZp4-IDy`#O(Ej(#rr*y{GIB)(T+>aeumJ!PPuF^ zHXbA9Jh?mbI=}?np}x8Y6)ICR1lU3;2$-{C!Nx@StPNPqfoJ5TJUBl&3KYkkaq8^S zK2hmy;B%$k@MGB70XHE0Oy}0mZ8j0u8E$8%r;)YJJ20NMU@7?eN^1hXl((JV$p69e zXu^4ODTY8F3~Bzn=kc1h+p^SQt=7sKmcWu0&76A0UAdJ4zOhrkWPe2RPEQ!CnL7lp zQmqLyr)6JBQ-`7U3r(zxOz+tLhacEl120O86kaX(43;ek!*#cbK4%kPY7A`Atz)Hv zs;4*5bEW;i!;4~KXXBT^%NSP-B&rbRY^Ugd!H)lQwP?=xvt4dDk zB8FG(;P<<`E4@`T8?|*VML5HPmS_wzB<48R6W@=V@f>{vy(LmlpgC(0y+5d zTo_6K)862k?i8Y;vWKsL>U32t_>W6N9opftu>!M(X*>x(tFNp{cyBv)K7;h9sSJTW z8u%m&$arUC(dQSNCqy|;>Uyf%m#+ns*DBhSi^#*o9FI7$jF9-zEUj-6*Yl$UJj+&7 z0JwtTc?6lo)U3opoYmMJD5HEr*eGg4ThJMLBm*y?ZSLW(Vzjv5b^f;L)R;vfU$0f(+&XbLV)VZMD(~%w1*+?N?{!jk z`{4ZOoeKipyeu_2G-;0(e3uz~4EzSeH6BnQl0RT^0x9ORiAQ;e+uE+KV*(m2nqm&g zY|Oqte)nWUpLTwCe#vbfzeAIp{qFc+_7uH=E_L%8($ReX&zugdP2<=5m*3x9%)TN) z^aee2c8?+M&nPuw$4+&>`}8Sm>3eE!&0f+1^VUquNB{Vb|6u8sI8rwC$HPw^J>1xt z%*vd6*3+&YZ*#?fYLK-1Zp+@tv-a&RK*ziP@Pozezo76SdWAZ_xoxXVyiR7BwkbBw zeRPw1ROV?tvubbO?(Y324^-DxjyE-Nq@3DJkAC<;p5$$t8l2xR?OIIg#wHg|_008~ z#RYFCE|%dqtfc$KD0O`_e){s|C$Vk&xC)U2lb`MrY*ULp%nezOvzKpl)mlc1s7?4B*U1NCq*xmM6}f}E9) z6Gv|!R~@}CT#wDH>MnIG15BOt)IqPswOW4o(z9)ktT<1*A;r0qeT>tjo_Bo53QB2S zvXUJ7X?Ar0H?Km-nZLatR3rig_t(U z7`)Qn-v4di_ViKa6)?W_{A=e0sgQ(7NRIlKt=YE(gO5+Zv_L{GlYvqDJvGQ0T~Ooq zug<=ek$a>bWOcH$=g;=OoH6iBSIw-HMf2n9SC;pG`RZ_+f9!>8F1#I5?Xt&i+vcZ# z|9643+NS>6d8YQ?s9cim4`yF)&A!IHbN&CecePDzBuRR{+7bQ_ee!YxUQ0r}+88v4 z4fe1LHUrGqa~m5Q0))^SBu+vGJf8o4&y$(;(yeYGu;*?*$T1$bs;jfAva&L>va&K8 zeyHHnj~?zw0vNh=hEat<0nXh9#Sg|9lvTTRP|s6?!mm2Ak-ubqpkBYc&1fonn9Fwa zjC@x@mw*dkpbeZ^fc^NP{@iKH{(i4%*1Ii#(H+kCxZN?clDt9CoL(CjHul9X;O98P z207A*g*!BN^Z0<+h}m*crBS%}`R&%LcN?4KgFPrMO3YT2NKqZ*qnCRuj2%~XuMRxx z-D^su;K#MuO5)zWg0G)(^K}=N&tt&X%EADCN1$V%of4wA=PFF>(Es|?%U*333>lhHg?!lU$|A^WY#2*`6~>L!m-tNsSO>Oda9eDwE!e#reSqh%Hcl&XD!^*)c$G97Z4}jhIpB2# zxeFm7)sGlP%mp!}1;SflXzxReaUxnLI4Aa@=~JKg&PgD z0MVkN-4M$$hUAau*tQ(j=pjJ3;-Eky%a>k@m+SRRxalr3(um>O@Fqm-kZDqzi1T0= zzJa3AQwTjdg+#0;dRCyFNY4WM;sJ)40PWB?MpNLRN9m~n&72KQ&Mbs#WqveaP!=Ef zSg2m2=i=l5R=KD)2}mH=j2dOq%rmOFP$Lz1vz@mv5@?z*GrLon>=2&bHB*j+DF z^guq6GI4gQC(4pW3Y~_r9ti`m^wZ!lMx5dlK;+C~Hn@j&#zpQ*Al@@a5YnhD=x8Oo ze%JybDzqJXmx4&jJ{_c+YCki6X3+t&TZRD5{xfgk)TEYTcVfB`bPkQl)Pi^TDe&*J z9VMb@K=&Oa1^Ye;If9Acy|Gp2h{DWlXE@;tbQ1W39&-CrG>n6-D6I)Z%+@+j(52nT z@SP?LYmw8b-mWgXAkDx$HRYgq4h`iNK5!%$#WC~&ghX~ zCBM(<*vyk0D8&Hc!OEbf(j2<~A_iD4CqFyqTrNvFSjSSqa`uC8N3HY=0YPfoEi5gK z4p1qVd7cok%}@#@8IYTZdo=L=@L(qfe?MyA&UAy5&}$4>ZNk}x`06W>ONAAxp71P8EUB2T zt<0IpMJMSwhUS!SzOr3e6Udy+TnEU=mmQgVbO3f)LL7J>+?D%O& z^Ty!p@(WRh!~;k*{dj9Y;H(4M<3wPhl!94&XlKCNSE(T)H+Hu zeHtsmd2DglNQF#w?_~YpV0&x-z|@@Rcu9qnS2Xj;FfGSqr=h4MZQTi-`oXv;Wo-;0 zv&~ATgt_)#q>#16y}5&xBYIWPWRoesh+c?Db;u&n+-iaY)|KTN0rigS zBVp#%Dg>#kIoaIVJ@|E~clPoWT0t@e1pr8KRF=~_ce$V#m_y_RBu}PsZSG5_8)OWb zgL>Ucxa{2bFso8uEZd#@;ig;#y|e3Bj9(5(k{{tce`#qrwEUiiE73!Z*l1$fU}1I*^uD1${|ZBK&l>A~d?Hrq_Yx%0A*n zQmP2gZoy5H!3Le;7gcyF6<``gEGgIkbq-5g_FG=UseaHe)8E*GEz66yjkoRtJW3G` zP>vu3e8nkOT83X96fxJY|9pHsH#vy6mySf(XUFAZI89 zmV@k!;4Z*+rk0Ywvq74v%+Xd%5GIsVA`e%{=})9uq;`I{Z`{}^-HeO2!hM{%udq(Jr!?Q8dCZn*`bA1L$Qija7!*u_do8#JCxtC+cr_*Ns!#Tkd|(Wv zPB)(t%sf%hynCyG&W4)K8IN75%xB@j^kM!%)4CV=;?f*_fm`WJQqoc(ZCDHkTswig zMAN0Fw}N+qI^X5gS8@`OOkwZ(BKf{1s%LTr2Rti^bw_lRYs=zi=qIW7d3;0w7u4eJ zBMT-6j~NSqu7d8$} zSJW}4;1(f9JO7X~7he<9NHUoGztI-8Mg? z*h;ECFRFhiao3$smFC0r8 zUQg~6!7c>fLrcgv|Ep%|k4uI(=mnpJajT$=$vIl|UXzPW113U7A=gzI)r?ndhh(vA zS;n$!+}2gUh;PMHsCnN$-_{p9pSiUszTQY~MzwI%47h`;RIkQGY|bjSgRN0bMrU7{ zR-uAmw+74p7}~HqFdNlUs1xBR74s85e$XY(#j8d~lP=JUCY$O~k#eGeA5OT5;=RDF zX6)fZcNJI^%`HQZpdxb^EsWlh1*IT(d*J|uwqTqKu!}T`%LOr{7!ZdRoUV5haR*Fz z*Mqhz{KdT%@hU|66!n6@={oL;MVC4=hp=XLgRKKmpWBzrUw}h}3rximWlwHNTdA)# zP0k3T%Vl%s34(B7zXNCa%r{n=`%N0UG@Urc_=Rq6CTv~%Y;I%)c4cG*0v)Q5A_sRB z)$N=yTPZ&p#v00)GtG>3r<6lv9qB(}$Yxd|PF_)3qC86=;_6`L+}s$xC0UW=APSZ^ zpUntO>2^&mQW}m)dgQEbYo&I-TYmOz@p+l9o+C0Jp=}{z+3TCGb4en`L;SLn_*Lk*4%$R989y zIBSEN8U&fzx(o~BqQ@WG8=29GmBwwsrYhYEImxZ92)h~18YMl;j>8Bc-OBtpt+3L# zW8soCL|u#ln{~zo&Bg3MBf}_eXRKi_?8-=-?(YA32Ry|s^2y-QYmj- zx8(79PcIxEl6yxr95Y?uup6sNBL4cYF=>{)@$S{Zue+P9mS{}YgiN5~976F-MC&5j zaQBlAxX5tNod58((P60hO1XZLt!ShaD()+*AF4=dP+2EO%G}M-^@pWskCPX@qRYdTE@%BmcCREne~-MNKh%+v%m{wZ!Rl}*nh1JUW&Kq;(UyWuq%F8{3d5F4N~Xx3}UM(Cb%s==;%b$MEUzFM`XoH~bHoC4LC zsT%Mh)DXXZ>CA{O?n93n-Nl7*dN_Z34*}<1J$w&tI}(RW<8-iGt1RB$$GKbk2!1Y; zo?0?WF+tdcO^{162?Lq>W+M&;qJN*XLF;<8f-xkS6W~y(f0Y+m{f;N&SrOnur-VPg zTTgI$Azzl&B%3Gd+C%ESvRSBxxr)7aIEgcedQx{(Mf>ntso8N8;`s6dW6b+R4lZvp zSXMuId6R6|dI<7a=-=^a`C-fmFeLojq1cQeIM~Posc}gG^#B!4>k+_HU%02s35|rZ zH?nq8MK8HRT#5EWACpo@Y*52C(XNw3zd&2bdT@a5%7%)T zcr_NTm8=4`9uFR1WSp=v&FatqB?_o}VHhA~m_eNz03tvUp-5re2*9C?$lreaSNj-* zLRCR(+KBbDGzj)|X!qiJt8I_?nujl^?bhiT{xVcn3;sj!vsa!)52GH4Tioy1UA6^l zKQ))NlGULakRU9CAqH(pwP2H?tqi%2c>Ba^JW%Ubk>p;m6XrXTq6iFOBYw&jGxy;I_Sa3tvfnXvMQ2UhVuHt#tm3SGnQZ0Xwd9 zv4~ifRUz}aiaDhm#y@<7u(c=1R6Jp1I$>!b6iaCskVE>RUXofXd8#bcRs0xSNSR`| zB9PO7Avx_!h*^8mvd}kAHI!fye8wyUV+yKBn23%aWC@g{AyGo24ZpM|F^3^}% zi56Ged>@&dI-9ufO|Nl*oX!0z`ozBs!0~+cS#8ek5P=VN&0M(DMQe?|yCrZTUOZQu z5G+Q=ZO3`xp4w_w`=45}Rt-Yd>L7`*@eb9HC+Qg5+qI-L-|m!>(e9&odq}%oVt4^}^tCB?4;#Jj`^hzPA+#klp^6x(!{*)Ra!9yVLN&@) zzE1yf(s|kenx1*GFgsJ8pEZ)mYgXsoq#EIw2-f>B)}&S%-Kt61CqO=DGCN2FQbd3k z`pQS6V_|}6`|U62^6!{iJYrtTf6)eiM~mqp@|V!*jMAsH;FQ>6GvD*-NN1ny);- zmRV_@fj9t+tT2b7YqPqHBDmH#*_BCtoA;qQ3#cNRfYL>1Q}`|hz_xQ5UC6fS>a$G~ z=tFI~Ig7GITuv~#yG@X6+fgU!9SyL;~r-cb%F{~)9Ub_E3e+kd^)xtyPaWXvhN z$t(PvF5K6+rs7*8ApR-;k2Nx)w8CV{7QzBL_~4>nJwCibkdi8Tw9Bznu6j_tY>ayO zUT$CG0yXD??7l^vE)nJD(Zj8ds64-EcNCKdUYp#h<#i}hzF~!t{4N8~AI41W4gZnJ za(v&*t6^n@i2f~ne(hv_M$pBQ#PA~kSM_s2)v%#-HD!WI`G8g03i7S` z@1f@ofC@7=atJf`B0Oc}old3}j+S{s=(h;H$!cIMfU-s7CK-z2(qdP7@0O&P_5Oetb@%H`;>Qtnb{yQUH3Ba6=w`e@^DY4{b!u&T$(l z!5AHqjOS>xe)O@^y*g_iws*z6#{@xf{L{6eywA!?aek8vXfvS|~(zK4Te3t>Ss-+>5_A|iQ@Buaq4 z-h)k-@^q-){v|B8$$#?-3gUIHA9oLNlIvkr-Pq7`JAX^Xc|cHYTxm8D6E zmLfgrHG>)jN$aLrYjfj4Lap!Po-~e|){oDch~*_kc!fyGZQO?CuDHJSa%Z&~7vMdz z`ZVjWx@8=Jw(*(#L~%6$4^IMf9p)2^2N3{UXNW96eoMt5cM z0kqzr-^~a25|qL?{O5N8%7DYkD@1hq7=RYzXjh^zDW~W+(fd5IdC7!W+l|8vvPKSW z1C+SYxXA~YM~B6I1ZTKlrh*N|HpvHQ>>{4Z(Y-@Hl@44C+eaTUsJt=ng9rXJXn+>S z28xBzYhER9aQfUSAdvga=q^GG0B9Ex^zyqJ@zk;Vx_LanLhB}PAXlJv6s(Xt((?d( z23Y;E(SzE+2+>7|l}wuJZ|;Cw$R%D0{d@cT=(L>=c(c=PokPNnje}RGJxI`PN}K{Z zMFhe-!`o@Vx?Ir1YYAIIG1^27=I&Xy)n;t3v3s1B*7=P@5`1j9>u1eth<*gm?|*dI zB`b^pY&WkVK=06573e$2p<1+R-yz@!S!vwN$YrY>eYb(wvy=pR>afaCIfkawd3@X}-t;yoYMorM^F8dxwAVds7qBZb z!F#M#$ev{X2>bocHu?lq`Ss1Ip`CY#c3*T>a z{@wo6Ku(7y4vye~ScL35OIxpp7j#y2^9)vR*CVSzb>%8gUqV1hxk% z3?6D5J0g<1zf^aF}D!D zGJh&v7+Q{qp=kGyPLYK{uurcckPvsgJ(SANvEYPE&u0uoCPbC=26Wr9P(TQm)dCNf zOq9rRH8`|%5ryDEF<}{iPQ-Z8J(DG9!Lo#g;#e=wr6EvA6j;;JZIl*l#v~aQs*wmPeB9qYz=JKu=Vxfpv2(_MZ8L z&|e6gglU` z%VJEExM6T1%@lS|E}NrGhSIelujg>khl^6D_@fT3RA-hrg&oAk8{KD0O)NPEighra z3fVeeX1wI>HN3oRA3}LxjAocHB#>QkYmMY&(2=`j#(T9jP;b9`_Nh5d#PF&$vZrX= zUsv7Yt4eY{^&-`<5@s)I^jdPMLS;&?$#en`<_z~p{s2Y@?`I~F7S~WN)&NZ$C=#MQI);_PQ5 zJIe8TQcH$8a4((mlWw;<4O79F!^USv>LZw# zz@q?+S_cEwfbfujGtlzh z>K@))HeWFaRy}b5Ly^@{ExcD3Z}OFD_3G-XqNA~3z|iHjk5~0+v@!M z(wroTG9Ye(4SiuY;;EbemUFjQS;W*iiW_6@4zY3V!Mt9DhD+`LUvrhk%5|91xyk}^ z*>o1p`Tw$2nyY}>Ra&3FW}>glWSD=*RM(bQ78eb4!J2%wyij|-aF@A`7N!t=0piA< z`T5E$@@#&n)v#P&ARSQW-F|}EtJI*f0^MJ;f}exU&M#GJ<1OL;7dA^-t#dy%oAr!2 zo>I5MzLtg3y>QxXP|un#lAZmn&4hFG0x5wz$2jsyo-f{iQkE*qih%8*zm?N%B732@Vsdr5cXuiUrDy=6oBw-hkB-uqM5mU|@oQ2?iz@m|$Rnfe8jC7?@ySf`JJJCK#AtV1j`O X1|}GoU|@oQ2?iz@m|)=l9s~ai?9Bt# literal 0 HcmV?d00001 diff --git a/common/pom.xml b/common/pom.xml new file mode 100644 index 0000000..bed69ff --- /dev/null +++ b/common/pom.xml @@ -0,0 +1,128 @@ + + + + 4.0.0 + + + com.bytedance.inf + css-root + 1.0.0 + ../pom.xml + + + css-common_${scala.binary.version} + jar + Cloud Shuffle Service Common + + + + com.bytedance.inf + css-network-common + ${project.version} + + + + com.google.guava + guava + + + + org.scala-lang + scala-library + ${scala.version} + + + + log4j + log4j + + + + org.slf4j + slf4j-api + + + + org.slf4j + slf4j-log4j12 + + + + org.slf4j + jul-to-slf4j + + + + org.slf4j + jcl-over-slf4j + + + + junit + junit + test + + + + org.scalatest + scalatest_${scala.binary.version} + test + + + + org.mockito + mockito-core + test + + + + io.dropwizard.metrics + metrics-core + + + + io.dropwizard.metrics + metrics-jvm + + + + io.dropwizard.metrics + metrics-jmx + + + + io.dropwizard.metrics + metrics-graphite + + + + javax.servlet + javax.servlet-api + + + + + + + net.alchim31.maven + scala-maven-plugin + + + + diff --git a/common/src/main/java/com/bytedance/css/common/ChildFirstURLClassLoader.java b/common/src/main/java/com/bytedance/css/common/ChildFirstURLClassLoader.java new file mode 100644 index 0000000..1f2355a --- /dev/null +++ b/common/src/main/java/com/bytedance/css/common/ChildFirstURLClassLoader.java @@ -0,0 +1,76 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common; + +import org.apache.commons.lang3.StringUtils; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Enumeration; + +public class ChildFirstURLClassLoader extends URLClassLoader { + + private static final Logger logger = Logger.getLogger(ChildFirstURLClassLoader.class); + + static { + ClassLoader.registerAsParallelCapable(); + } + + private ParentClassLoader parent; + private URL[] urls = null; + + public ChildFirstURLClassLoader(URL[] urls, ClassLoader parent) { + super(urls, null); + this.urls = urls; + this.parent = new ParentClassLoader(parent); + } + + @Override + public Class loadClass(String name, boolean resolve) throws ClassNotFoundException { + try { + Class clazz = super.loadClass(name, resolve); + logger.debug(String.format("loaded class %s with %s", name, StringUtils.join(urls))); + return clazz; + } catch (ClassNotFoundException cnf) { + return parent.loadClass(name, resolve); + } + } + + @Override + public Enumeration getResources(String name) throws IOException { + ArrayList urls = Collections.list(super.getResources(name)); + urls.addAll(Collections.list(parent.getResources(name))); + return Collections.enumeration(urls); + } + + @Override + public URL getResource(String name) { + URL url = super.getResource(name); + if (url != null) { + return url; + } else { + return parent.getResource(name); + } + } +} diff --git a/common/src/main/java/com/bytedance/css/common/ParentClassLoader.java b/common/src/main/java/com/bytedance/css/common/ParentClassLoader.java new file mode 100644 index 0000000..e118dff --- /dev/null +++ b/common/src/main/java/com/bytedance/css/common/ParentClassLoader.java @@ -0,0 +1,41 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common; + +public class ParentClassLoader extends ClassLoader { + + static { + ClassLoader.registerAsParallelCapable(); + } + + public ParentClassLoader(ClassLoader parent) { + super(parent); + } + + @Override + public Class findClass(String name) throws ClassNotFoundException { + return super.findClass(name); + } + + @Override + public Class loadClass(String name, boolean resolve) throws ClassNotFoundException { + return super.loadClass(name, resolve); + } +} diff --git a/common/src/main/java/com/bytedance/css/common/metrics/ByteDanceMetricsEmitter.java b/common/src/main/java/com/bytedance/css/common/metrics/ByteDanceMetricsEmitter.java new file mode 100644 index 0000000..6aaac5e --- /dev/null +++ b/common/src/main/java/com/bytedance/css/common/metrics/ByteDanceMetricsEmitter.java @@ -0,0 +1,192 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.metrics; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.net.DatagramPacket; +import java.net.DatagramSocket; +import java.net.InetSocketAddress; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Map; + +/** + * Emit metrics to bytedance metrics system. + * + * Metrics name format: ${tags}#${metricsName} + */ +public class ByteDanceMetricsEmitter { + + private static final Logger logger = LoggerFactory.getLogger(ByteDanceMetricsEmitter.class); + + private static final String METRICS_HOST = "localhost"; + private static final int METRICS_PORT = 9123; + + private static Map emitters = null; + private static DatagramSocket socket = null; + private static InetSocketAddress address = null; + + private final String namespace; + + private ByteDanceMetricsEmitter(String namespace) { + this.namespace = namespace; + } + + public static synchronized ByteDanceMetricsEmitter getEmitter(String namespace) { + if (emitters == null) { + emitters = new HashMap<>(); + } + if (socket == null || address == null) { + address = new InetSocketAddress(METRICS_HOST, METRICS_PORT); + try { + socket = new DatagramSocket(); + } catch (Exception e) { + logger.debug("Failed to create DatagramSocket.", e); + } + } + ByteDanceMetricsEmitter emitter = emitters.get(namespace); + if (emitter == null) { + emitter = new ByteDanceMetricsEmitter(namespace); + emitters.put(namespace, emitter); + } + return emitter; + } + + public static String makeTags(Map tags) { + StringBuilder sb = new StringBuilder(); + for (Map.Entry tag : tags.entrySet()) { + if (sb.length() > 0) { + sb.append('|'); + } + sb.append(tag.getKey()); + sb.append('='); + sb.append(tag.getValue()); + } + return sb.toString(); + } + + private void emit(byte[] buf) { + try { + DatagramPacket packet = new DatagramPacket(buf, buf.length, address); + socket.send(packet); + } catch (Exception e) { + logger.debug("Failed to emit metrics.", e); + } + } + + private void writeMsgpackString(DataOutputStream out, String s) { + try { + if (s.length() < 32) { + out.writeByte(160 + s.length()); // 4 byte string + } else if (s.length() < (1 << 8)) { + out.writeByte(0xd9); + out.writeByte(s.length()); + } else if (s.length() < (1 << 16)) { + out.writeByte(0xda); + out.writeShort(s.length()); + } else { + out.writeByte(0xdb); + out.writeInt(s.length()); + } + out.write(s.getBytes(StandardCharsets.UTF_8)); + } catch (Exception e) { + logger.debug("Failed to write message package.", e); + } + } + + private void emit(String type, String fullName, String value, String tags) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputStream dataOut = new DataOutputStream(out); + try { + dataOut.writeByte(144 + 6); // 6 element array + writeMsgpackString(dataOut, "emit"); + writeMsgpackString(dataOut, type); + writeMsgpackString(dataOut, fullName); + writeMsgpackString(dataOut, value); + writeMsgpackString(dataOut, tags); + writeMsgpackString(dataOut, ""); + dataOut.flush(); + byte[] buf = out.toByteArray(); + emit(buf); + } finally { + dataOut.close(); + out.close(); + } + } + + public void emitCounter(String name, String value) { + String[] splits = name.split("#"); + if (splits.length == 2) { + emitStore(splits[1], value, splits[0]); + } else { + emitCounter(name, value, ""); + } + } + + public void emitCounter(String name, String value, String tags) { + try { + emit("counter", namespace + "." + name, value, tags); + } catch (Exception e) { + // preserve conventions + logger.debug("Failed to emit counter.", e); + } + } + + public void emitTimer(String name, String value) { + String[] splits = name.split("#"); + if (splits.length == 2) { + emitStore(splits[1], value, splits[0]); + } else { + emitTimer(name, value, ""); + } + } + + public void emitTimer(String name, String value, String tags) { + try { + emit("timer", namespace + "." + name, value, tags); + } catch (Exception e) { + // preserve conventions + logger.debug("Failed to emit timer.", e); + } + } + + public void emitStore(String name, String value) { + String[] splits = name.split("#"); + if (splits.length == 2) { + emitStore(splits[1], value, splits[0]); + } else { + emitStore(name, value, ""); + } + } + + public void emitStore(String name, String value, String tags) { + try { + emit("store", namespace + "." + name, value, tags); + } catch (Exception e) { + // preserve conventions + logger.debug("Failed to emit store.", e); + } + } +} diff --git a/common/src/main/java/com/bytedance/css/common/metrics/ByteDanceMetricsReporter.java b/common/src/main/java/com/bytedance/css/common/metrics/ByteDanceMetricsReporter.java new file mode 100644 index 0000000..b28d1d3 --- /dev/null +++ b/common/src/main/java/com/bytedance/css/common/metrics/ByteDanceMetricsReporter.java @@ -0,0 +1,137 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.metrics; + +import com.codahale.metrics.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Locale; +import java.util.Map; +import java.util.SortedMap; +import java.util.concurrent.TimeUnit; + +public class ByteDanceMetricsReporter extends ScheduledReporter { + + private static final Logger logger = LoggerFactory.getLogger(ByteDanceMetricsEmitter.class); + + private final ByteDanceMetricsEmitter emitter; + + public ByteDanceMetricsReporter( + ByteDanceMetricsEmitter emitter, + MetricRegistry registry, MetricFilter filter, + TimeUnit rateUnit, TimeUnit durationUnit) { + super(registry, "bytedance-reporter", filter, rateUnit, durationUnit); + this.emitter = emitter; + } + + private String format(Object o) { + if (o instanceof Float) { + return format(((Float) o).doubleValue()); + } else if (o instanceof Double) { + return format(((Double) o).doubleValue()); + } else if (o instanceof Byte) { + return format(((Byte) o).longValue()); + } else if (o instanceof Short) { + return format(((Short) o).longValue()); + } else if (o instanceof Integer) { + return format(((Integer) o).longValue()); + } else if (o instanceof Long) { + return format(((Long) o).longValue()); + } + return null; + } + + private String format(long n) { + return Long.toString(n); + } + + private String format(double v) { + // the Carbon plaintext format is pretty underspecified, but it seems like it just wants + // US-formatted digits + return String.format(Locale.US, "%2.2f", v); + } + + @Override + public void report(SortedMap gauges, + SortedMap counters, + SortedMap histograms, + SortedMap meters, + SortedMap timers) { + + for (Map.Entry entry : gauges.entrySet()) { + emitter.emitStore(entry.getKey(), format(entry.getValue().getValue())); + } + + for (Map.Entry entry : counters.entrySet()) { + emitter.emitStore(entry.getKey(), format(entry.getValue().getCount())); + } + + for (Map.Entry entry : histograms.entrySet()) { + Histogram histogram = entry.getValue(); + emitter.emitStore(entry.getKey() + ".counter", format(histogram.getCount())); + Snapshot snapshot = histogram.getSnapshot(); + if (snapshot.size() > 0) { + emitter.emitStore(entry.getKey() + ".max", format(snapshot.getMax())); + emitter.emitStore(entry.getKey() + ".mean", format(snapshot.getMean())); + emitter.emitStore(entry.getKey() + ".min", format(snapshot.getMin())); + emitter.emitStore(entry.getKey() + ".pct50", format(snapshot.getMedian())); + emitter.emitStore(entry.getKey() + ".pct75", format(snapshot.get75thPercentile())); + emitter.emitStore(entry.getKey() + ".pct95", format(snapshot.get95thPercentile())); + emitter.emitStore(entry.getKey() + ".pct99", format(snapshot.get99thPercentile())); + } + } + + for (Map.Entry entry : meters.entrySet()) { + Meter meter = entry.getValue(); + emitter.emitStore(entry.getKey() + ".counter", + format(meter.getCount())); + emitter.emitStore(entry.getKey() + ".m1_rate", + format(convertRate(meter.getOneMinuteRate()))); + emitter.emitStore(entry.getKey() + ".m5_rate", + format(convertRate(meter.getFiveMinuteRate()))); + emitter.emitStore(entry.getKey() + ".m15_rate", + format(convertRate(meter.getFifteenMinuteRate()))); + emitter.emitStore(entry.getKey() + ".mean_rate", + format(convertRate(meter.getMeanRate()))); + } + + for (Map.Entry entry : timers.entrySet()) { + Timer timer = entry.getValue(); + Snapshot snapshot = timer.getSnapshot(); + if (snapshot.size() > 0) { + emitter.emitStore(entry.getKey() + ".max", + format(convertDuration(snapshot.getMax()))); + emitter.emitStore(entry.getKey() + ".mean", + format(convertDuration(snapshot.getMean()))); + emitter.emitStore(entry.getKey() + ".min", + format(convertDuration(snapshot.getMin()))); + emitter.emitStore(entry.getKey() + ".p50", + format(convertDuration(snapshot.getMedian()))); + emitter.emitStore(entry.getKey() + ".p75", + format(convertDuration(snapshot.get75thPercentile()))); + emitter.emitStore(entry.getKey() + ".p95", + format(convertDuration(snapshot.get95thPercentile()))); + emitter.emitStore(entry.getKey() + ".p99", + format(convertDuration(snapshot.get99thPercentile()))); + } + } + } +} diff --git a/common/src/main/java/com/bytedance/css/common/protocol/CommittedPartitionInfo.java b/common/src/main/java/com/bytedance/css/common/protocol/CommittedPartitionInfo.java new file mode 100644 index 0000000..a66cfe1 --- /dev/null +++ b/common/src/main/java/com/bytedance/css/common/protocol/CommittedPartitionInfo.java @@ -0,0 +1,98 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.protocol; + +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +import java.io.Serializable; + +public class CommittedPartitionInfo extends PartitionInfo implements Serializable { + private String host; + private int port; + private ShuffleMode shuffleMode; + private String filePath; + // normal length; 0 length; -1L means commit failed + private long fileLength = -1L; + + public CommittedPartitionInfo( + int reducerId, + int epochId, + String host, + int port, + ShuffleMode shuffleMode, + String filePath, + long fileLength) { + super(reducerId, epochId); + this.host = host; + this.port = port; + this.shuffleMode = shuffleMode; + this.filePath = filePath; + this.fileLength = fileLength; + } + + public String getHost() { + return host; + } + + public int getPort() { + return port; + } + + public ShuffleMode getShuffleMode() { + return shuffleMode; + } + + public String getFilePath() { + return filePath; + } + + public long getFileLength() { + return fileLength; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof CommittedPartitionInfo)) { + return false; + } + CommittedPartitionInfo o = (CommittedPartitionInfo) other; + return super.equals(o) && host.equals(o.host) && port == o.port && shuffleMode == o.shuffleMode + && filePath.equals(o.filePath) && fileLength == o.fileLength; + } + + @Override + public int hashCode() { + return (super.hashCode() + host + port + shuffleMode + filePath + fileLength).hashCode(); + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("reducerId", reducerId) + .append("epochId", epochId) + .append("host", host) + .append("port", port) + .append("shuffleMode", shuffleMode) + .append("filePath", filePath) + .append("fileLength", fileLength) + .toString(); + } +} diff --git a/common/src/main/java/com/bytedance/css/common/protocol/CssStatusCode.java b/common/src/main/java/com/bytedance/css/common/protocol/CssStatusCode.java new file mode 100644 index 0000000..eaf3b23 --- /dev/null +++ b/common/src/main/java/com/bytedance/css/common/protocol/CssStatusCode.java @@ -0,0 +1,51 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.protocol; + +public enum CssStatusCode { + // 1/0 Status + Success(0), + PartialSuccess(1), + Failed(2), + Waiting(3), + Timeout(4), + + // Register shuffle Status + ShuffleNotRegistered(15), + MapEnded(16), + + // get reducer file group + StageEnded(18), + StageEndDataLost(19), + EpochShouldRotate(20), + + Dummy(100); + + private byte value; + + CssStatusCode(int value) { + assert(value >= 0 && value < 256); + this.value = (byte) value; + } + + public byte getValue() { + return value; + } +} diff --git a/common/src/main/java/com/bytedance/css/common/protocol/FailedPartitionInfoBatch.java b/common/src/main/java/com/bytedance/css/common/protocol/FailedPartitionInfoBatch.java new file mode 100644 index 0000000..dc2bfa5 --- /dev/null +++ b/common/src/main/java/com/bytedance/css/common/protocol/FailedPartitionInfoBatch.java @@ -0,0 +1,93 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.protocol; + +import com.google.common.base.Objects; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +import java.io.Serializable; + +public class FailedPartitionInfoBatch extends PartitionInfo implements Serializable { + private int mapId; + private int attemptId; + private int batchId; + + public String getFailedPartitionBatchStr() { + return getEpochKey() + "-" + mapId + "-" + attemptId + "-" + batchId; + } + + public int getMapId() { + return mapId; + } + + public void setMapId(int mapId) { + this.mapId = mapId; + } + + public int getAttemptId() { + return attemptId; + } + + public void setAttemptId(int attemptId) { + this.attemptId = attemptId; + } + + public int getBatchId() { + return batchId; + } + + public void setBatchId(int batchId) { + this.batchId = batchId; + } + + public FailedPartitionInfoBatch(int reducerId, int epochId, int mapId, int attemptId, int batchId) { + super(reducerId, epochId); + this.mapId = mapId; + this.attemptId = attemptId; + this.batchId = batchId; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof FailedPartitionInfoBatch)) { + return false; + } + FailedPartitionInfoBatch o = (FailedPartitionInfoBatch) other; + return super.equals(o) && mapId == o.mapId && + attemptId == o.attemptId && batchId == o.batchId; + } + + @Override + public int hashCode() { + return Objects.hashCode(super.hashCode(), mapId, attemptId, batchId); + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("reducerId", reducerId) + .append("epochId", epochId) + .append("mapId", mapId) + .append("attemptId", attemptId) + .append("batchId", batchId) + .toString(); + } +} diff --git a/common/src/main/java/com/bytedance/css/common/protocol/PartitionGroup.java b/common/src/main/java/com/bytedance/css/common/protocol/PartitionGroup.java new file mode 100644 index 0000000..5bedbbd --- /dev/null +++ b/common/src/main/java/com/bytedance/css/common/protocol/PartitionGroup.java @@ -0,0 +1,67 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.protocol; + +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +import java.util.List; + +public class PartitionGroup extends ReplicaBase { + + public int partitionGroupId; + public int epochId; + public int startPartition; + public int endPartition; + + public PartitionGroup( + int partitionGroupId, + int epochId, + int startPartition, + int endPartition, + List workerLists) { + this.partitionGroupId = partitionGroupId; + this.epochId = epochId; + this.startPartition = startPartition; + this.endPartition = endPartition; + this.replicaWorkers = workerLists; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof PartitionGroup)) { + return false; + } + PartitionGroup o = (PartitionGroup) other; + return partitionGroupId == o.partitionGroupId && epochId == o.epochId && + startPartition == o.startPartition && endPartition == o.endPartition; + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("partitionGroupId", partitionGroupId) + .append("epochId", epochId) + .append("startPartition", startPartition) + .append("endPartition", endPartition) + .append("replicaWorkers", replicaWorkers) + .toString(); + } +} diff --git a/common/src/main/java/com/bytedance/css/common/protocol/PartitionGroupManager.java b/common/src/main/java/com/bytedance/css/common/protocol/PartitionGroupManager.java new file mode 100644 index 0000000..87ce9de --- /dev/null +++ b/common/src/main/java/com/bytedance/css/common/protocol/PartitionGroupManager.java @@ -0,0 +1,61 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.protocol; + +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; + +public class PartitionGroupManager { + + private int groupLength = -1; + private final ConcurrentHashMap partitionGroupMap = new ConcurrentHashMap<>(); + + public PartitionGroupManager(List partitionGroups) { + partitionGroups.stream().forEach(p -> { + partitionGroupMap.put(p.partitionGroupId, p); + if (p.startPartition == 0) { + this.groupLength = p.endPartition - p.startPartition; + } + }); + if (groupLength == -1) { + throw new RuntimeException("partitionGroups could be wrong."); + } + } + + public Integer groupId(int reducerId) { + return reducerId / groupLength; + } + + public PartitionGroup getGroup(int groupId) { + return partitionGroupMap.get(groupId); + } + + public void updateGroup(int groupId, PartitionGroup p) { + partitionGroupMap.put(groupId, p); + } + + public int getGroupLength() { + return groupLength; + } + + public int getPartitionGroupSize() { + return partitionGroupMap.size(); + } +} diff --git a/common/src/main/java/com/bytedance/css/common/protocol/PartitionInfo.java b/common/src/main/java/com/bytedance/css/common/protocol/PartitionInfo.java new file mode 100644 index 0000000..60486e7 --- /dev/null +++ b/common/src/main/java/com/bytedance/css/common/protocol/PartitionInfo.java @@ -0,0 +1,83 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.protocol; + +import com.google.common.base.Objects; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +import java.io.Serializable; + +public class PartitionInfo implements Serializable { + protected int reducerId; // reducer partition id start from 0 + protected int epochId; // a reducer partition could be mapped into multi epoch + + public String getEpochKey() { + return reducerId + "-" + epochId; + } + + public int getReducerId() { + return reducerId; + } + + public void setReducerId(int reducerId) { + this.reducerId = reducerId; + } + + public int getEpochId() { + return epochId; + } + + public void setEpochId(int epochId) { + this.epochId = epochId; + } + + public PartitionInfo(int reducerId, int epochId) { + this.reducerId = reducerId; + this.epochId = epochId; + } + + public PartitionInfo(PartitionInfo partitionInfo) { + this.reducerId = partitionInfo.reducerId; + this.epochId = partitionInfo.epochId; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof PartitionInfo)) { + return false; + } + PartitionInfo o = (PartitionInfo) other; + return reducerId == o.reducerId && epochId == o.epochId; + } + + @Override + public int hashCode() { + return Objects.hashCode(reducerId, epochId); + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("reducerId", reducerId) + .append("epochId", epochId) + .toString(); + } +} diff --git a/common/src/main/java/com/bytedance/css/common/protocol/ReplicaBase.java b/common/src/main/java/com/bytedance/css/common/protocol/ReplicaBase.java new file mode 100644 index 0000000..9dc7f2c --- /dev/null +++ b/common/src/main/java/com/bytedance/css/common/protocol/ReplicaBase.java @@ -0,0 +1,68 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.protocol; + +import com.google.common.base.Objects; +import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +import java.io.Serializable; +import java.util.List; +import java.util.stream.Collectors; + +public class ReplicaBase implements Serializable { + + public List replicaWorkers; + + public List getReplicaWorkers() { + return replicaWorkers; + } + + public void setReplicaWorkers(List replicaWorkers) { + this.replicaWorkers = replicaWorkers; + } + + public String makeReplicaAddressStr() { + List hostPortList = replicaWorkers.stream().map(x -> x.host + ":" + x.port) + .collect(Collectors.toList()); + return StringUtils.join(hostPortList, "-"); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ReplicaBase that = (ReplicaBase) o; + return Objects.equal(getReplicaWorkers(), that.getReplicaWorkers()); + } + + @Override + public int hashCode() { + return Objects.hashCode(getReplicaWorkers()); + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("replicaWorkers", replicaWorkers) + .toString(); + } +} diff --git a/common/src/main/java/com/bytedance/css/common/protocol/RpcNameConstants.java b/common/src/main/java/com/bytedance/css/common/protocol/RpcNameConstants.java new file mode 100644 index 0000000..4442618 --- /dev/null +++ b/common/src/main/java/com/bytedance/css/common/protocol/RpcNameConstants.java @@ -0,0 +1,41 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.protocol; + +public class RpcNameConstants { + + // For Master + public static String MASTER_SYS = "MasterSys"; + + // Master Endpoint Name + public static String MASTER_EP = "MasterEndpoint"; + + // For Worker + public static String WORKER_SYS = "WorkerSys"; + + // Worker Endpoint Name + public static String WORKER_EP = "WorkerEndpoint"; + + // For Shuffle Client + public static String SHUFFLE_CLIENT_SYS = "ShuffleClientSys"; + + // For Heart Beat + public static String HEARTBEAT = "HeartBeat"; +} diff --git a/common/src/main/java/com/bytedance/css/common/protocol/ShuffleMode.java b/common/src/main/java/com/bytedance/css/common/protocol/ShuffleMode.java new file mode 100644 index 0000000..0395d3c --- /dev/null +++ b/common/src/main/java/com/bytedance/css/common/protocol/ShuffleMode.java @@ -0,0 +1,37 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.protocol; + +public enum ShuffleMode { + + DISK(1), + HDFS(2); + + private final byte value; + + ShuffleMode(int value) { + assert(value >= 0 && value < 256); + this.value = (byte) value; + } + + public byte getValue() { + return value; + } +} diff --git a/common/src/main/java/com/bytedance/css/common/protocol/TransportModuleConstants.java b/common/src/main/java/com/bytedance/css/common/protocol/TransportModuleConstants.java new file mode 100644 index 0000000..89802dc --- /dev/null +++ b/common/src/main/java/com/bytedance/css/common/protocol/TransportModuleConstants.java @@ -0,0 +1,34 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.protocol; + +public class TransportModuleConstants { + + public static final String PUSH_MODULE = "push"; + + public static final String FETCH_MODULE = "fetch"; + + public static final String RPC_MODULE = "rpc"; + + public static final String FILE_MODULE = "files"; + + public static final String DATA_MODULE = "data"; + +} diff --git a/common/src/main/java/com/bytedance/css/common/protocol/WorkerAddress.java b/common/src/main/java/com/bytedance/css/common/protocol/WorkerAddress.java new file mode 100644 index 0000000..14bfda0 --- /dev/null +++ b/common/src/main/java/com/bytedance/css/common/protocol/WorkerAddress.java @@ -0,0 +1,59 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.protocol; + +import com.google.common.base.Objects; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +import java.io.Serializable; + +public class WorkerAddress implements Serializable { + + public String host; + public int port; + + public WorkerAddress(String host, int port) { + this.host = host; + this.port = port; + } + + @Override + public boolean equals(Object other) { + if (other instanceof WorkerAddress) { + WorkerAddress o = (WorkerAddress) other; + return host == o.host && port == o.port && super.equals(o); + } + return false; + } + + @Override + public int hashCode() { + return Objects.hashCode(host, port); + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE) + .append("host", host) + .append("port", port) + .toString(); + } +} diff --git a/common/src/main/java/com/bytedance/css/common/protocol/WorkerStatus.java b/common/src/main/java/com/bytedance/css/common/protocol/WorkerStatus.java new file mode 100644 index 0000000..ba021a6 --- /dev/null +++ b/common/src/main/java/com/bytedance/css/common/protocol/WorkerStatus.java @@ -0,0 +1,107 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.protocol; + +public class WorkerStatus { + + public String name; + public String host; + public int rpcPort; + public int pushPort; + public int fetchPort; + public long rttAvgStat; + public long lastHeartbeat; + + public WorkerStatus() { + } + + public WorkerStatus( + String name, + String host, + int rpcPort, + int pushPort, + int fetchPort, + long rttAvgStat, + long lastHeartbeat) { + this.name = name; + this.host = host; + this.rpcPort = rpcPort; + this.pushPort = pushPort; + this.fetchPort = fetchPort; + this.rttAvgStat = rttAvgStat; + this.lastHeartbeat = lastHeartbeat; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public String getHost() { + return host; + } + + public void setHost(String host) { + this.host = host; + } + + public int getRpcPort() { + return rpcPort; + } + + public void setRpcPort(int rpcPort) { + this.rpcPort = rpcPort; + } + + public int getPushPort() { + return pushPort; + } + + public void setPushPort(int pushPort) { + this.pushPort = pushPort; + } + + public int getFetchPort() { + return fetchPort; + } + + public void setFetchPort(int fetchPort) { + this.fetchPort = fetchPort; + } + + public long getRttAvgStat() { + return rttAvgStat; + } + + public void setRttAvgStat(long rttAvgStat) { + this.rttAvgStat = rttAvgStat; + } + + public long getLastHeartbeat() { + return lastHeartbeat; + } + + public void setLastHeartbeat(long lastHeartbeat) { + this.lastHeartbeat = lastHeartbeat; + } +} diff --git a/common/src/main/java/com/bytedance/css/common/unsafe/Platform.java b/common/src/main/java/com/bytedance/css/common/unsafe/Platform.java new file mode 100644 index 0000000..50b4ada --- /dev/null +++ b/common/src/main/java/com/bytedance/css/common/unsafe/Platform.java @@ -0,0 +1,259 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.unsafe; + +import java.lang.reflect.Constructor; +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.nio.ByteBuffer; + +import sun.misc.Cleaner; +import sun.misc.Unsafe; + +public final class Platform { + + private static final Unsafe _UNSAFE; + + public static final int BOOLEAN_ARRAY_OFFSET; + + public static final int BYTE_ARRAY_OFFSET; + + public static final int SHORT_ARRAY_OFFSET; + + public static final int INT_ARRAY_OFFSET; + + public static final int LONG_ARRAY_OFFSET; + + public static final int FLOAT_ARRAY_OFFSET; + + public static final int DOUBLE_ARRAY_OFFSET; + + private static final boolean unaligned; + static { + boolean _unaligned; + String arch = System.getProperty("os.arch", ""); + if (arch.equals("ppc64le") || arch.equals("ppc64") || arch.equals("s390x")) { + // Since java.nio.Bits.unaligned() doesn't return true on ppc (See JDK-8165231), but + // ppc64 and ppc64le support it + _unaligned = true; + } else { + try { + Class bitsClass = + Class.forName("java.nio.Bits", false, ClassLoader.getSystemClassLoader()); + Method unalignedMethod = bitsClass.getDeclaredMethod("unaligned"); + unalignedMethod.setAccessible(true); + _unaligned = Boolean.TRUE.equals(unalignedMethod.invoke(null)); + } catch (Throwable t) { + // We at least know x86 and x64 support unaligned access. + //noinspection DynamicRegexReplaceableByCompiledPattern + _unaligned = arch.matches("^(i[3-6]86|x86(_64)?|x64|amd64|aarch64)$"); + } + } + unaligned = _unaligned; + } + + /** + * @return true when running JVM is having sun's Unsafe package available in it and underlying + * system having unaligned-access capability. + */ + public static boolean unaligned() { + return unaligned; + } + + public static int getInt(Object object, long offset) { + return _UNSAFE.getInt(object, offset); + } + + public static void putInt(Object object, long offset, int value) { + _UNSAFE.putInt(object, offset, value); + } + + public static boolean getBoolean(Object object, long offset) { + return _UNSAFE.getBoolean(object, offset); + } + + public static void putBoolean(Object object, long offset, boolean value) { + _UNSAFE.putBoolean(object, offset, value); + } + + public static byte getByte(Object object, long offset) { + return _UNSAFE.getByte(object, offset); + } + + public static void putByte(Object object, long offset, byte value) { + _UNSAFE.putByte(object, offset, value); + } + + public static short getShort(Object object, long offset) { + return _UNSAFE.getShort(object, offset); + } + + public static void putShort(Object object, long offset, short value) { + _UNSAFE.putShort(object, offset, value); + } + + public static long getLong(Object object, long offset) { + return _UNSAFE.getLong(object, offset); + } + + public static void putLong(Object object, long offset, long value) { + _UNSAFE.putLong(object, offset, value); + } + + public static float getFloat(Object object, long offset) { + return _UNSAFE.getFloat(object, offset); + } + + public static void putFloat(Object object, long offset, float value) { + _UNSAFE.putFloat(object, offset, value); + } + + public static double getDouble(Object object, long offset) { + return _UNSAFE.getDouble(object, offset); + } + + public static void putDouble(Object object, long offset, double value) { + _UNSAFE.putDouble(object, offset, value); + } + + public static Object getObjectVolatile(Object object, long offset) { + return _UNSAFE.getObjectVolatile(object, offset); + } + + public static void putObjectVolatile(Object object, long offset, Object value) { + _UNSAFE.putObjectVolatile(object, offset, value); + } + + public static long allocateMemory(long size) { + return _UNSAFE.allocateMemory(size); + } + + public static void freeMemory(long address) { + _UNSAFE.freeMemory(address); + } + + public static long reallocateMemory(long address, long oldSize, long newSize) { + long newMemory = _UNSAFE.allocateMemory(newSize); + copyMemory(null, address, null, newMemory, oldSize); + freeMemory(address); + return newMemory; + } + + /** + * Uses internal JDK APIs to allocate a DirectByteBuffer while ignoring the JVM's + * MaxDirectMemorySize limit (the default limit is too low and we do not want to require users + * to increase it). + */ + @SuppressWarnings("unchecked") + public static ByteBuffer allocateDirectBuffer(int size) { + try { + Class cls = Class.forName("java.nio.DirectByteBuffer"); + Constructor constructor = cls.getDeclaredConstructor(Long.TYPE, Integer.TYPE); + constructor.setAccessible(true); + Field cleanerField = cls.getDeclaredField("cleaner"); + cleanerField.setAccessible(true); + long memory = allocateMemory(size); + ByteBuffer buffer = (ByteBuffer) constructor.newInstance(memory, size); + Cleaner cleaner = Cleaner.create(buffer, () -> freeMemory(memory)); + cleanerField.set(buffer, cleaner); + return buffer; + } catch (Exception e) { + throwException(e); + } + throw new IllegalStateException("unreachable"); + } + + public static void setMemory(Object object, long offset, long size, byte value) { + _UNSAFE.setMemory(object, offset, size, value); + } + + public static void setMemory(long address, byte value, long size) { + _UNSAFE.setMemory(address, size, value); + } + + public static void copyMemory( + Object src, long srcOffset, Object dst, long dstOffset, long length) { + // Check if dstOffset is before or after srcOffset to determine if we should copy + // forward or backwards. This is necessary in case src and dst overlap. + if (dstOffset < srcOffset) { + while (length > 0) { + long size = Math.min(length, UNSAFE_COPY_THRESHOLD); + _UNSAFE.copyMemory(src, srcOffset, dst, dstOffset, size); + length -= size; + srcOffset += size; + dstOffset += size; + } + } else { + srcOffset += length; + dstOffset += length; + while (length > 0) { + long size = Math.min(length, UNSAFE_COPY_THRESHOLD); + srcOffset -= size; + dstOffset -= size; + _UNSAFE.copyMemory(src, srcOffset, dst, dstOffset, size); + length -= size; + } + + } + } + + /** + * Raises an exception bypassing compiler checks for checked exceptions. + */ + public static void throwException(Throwable t) { + _UNSAFE.throwException(t); + } + + /** + * Limits the number of bytes to copy per {@link Unsafe#copyMemory(long, long, long)} to + * allow safepoint polling during a large copy. + */ + private static final long UNSAFE_COPY_THRESHOLD = 1024L * 1024L; + + static { + sun.misc.Unsafe unsafe; + try { + Field unsafeField = Unsafe.class.getDeclaredField("theUnsafe"); + unsafeField.setAccessible(true); + unsafe = (sun.misc.Unsafe) unsafeField.get(null); + } catch (Throwable cause) { + unsafe = null; + } + _UNSAFE = unsafe; + + if (_UNSAFE != null) { + BOOLEAN_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(boolean[].class); + BYTE_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(byte[].class); + SHORT_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(short[].class); + INT_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(int[].class); + LONG_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(long[].class); + FLOAT_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(float[].class); + DOUBLE_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(double[].class); + } else { + BOOLEAN_ARRAY_OFFSET = 0; + BYTE_ARRAY_OFFSET = 0; + SHORT_ARRAY_OFFSET = 0; + INT_ARRAY_OFFSET = 0; + LONG_ARRAY_OFFSET = 0; + FLOAT_ARRAY_OFFSET = 0; + DOUBLE_ARRAY_OFFSET = 0; + } + } +} diff --git a/common/src/main/java/com/bytedance/css/common/util/JsonUtils.java b/common/src/main/java/com/bytedance/css/common/util/JsonUtils.java new file mode 100644 index 0000000..b3a176c --- /dev/null +++ b/common/src/main/java/com/bytedance/css/common/util/JsonUtils.java @@ -0,0 +1,62 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.util; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.io.IOException; + +public class JsonUtils { + + protected static ObjectMapper mapper = new ObjectMapper(); + + static { + mapper.configure(JsonParser.Feature.ALLOW_UNQUOTED_FIELD_NAMES, true); + mapper.configure(JsonParser.Feature.ALLOW_SINGLE_QUOTES, true); + mapper.configure(JsonParser.Feature.ALLOW_COMMENTS, true); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + } + + public static ObjectMapper getMapper() { + return mapper; + } + + public static String serialize(Object obj) { + if (obj == null) { + return ""; + } + try { + return mapper.writeValueAsString(obj); + } catch (JsonProcessingException e) { + throw new RuntimeException(String.format("Failed to serialize %s (%s)", obj, obj.getClass()), e); + } + } + + public static T deserialize(String content, Class valueType) { + try { + return mapper.readValue(content, valueType); + } catch (IOException e) { + throw new RuntimeException(String.format("Failed to deserialize %s from json %s", valueType, content), e); + } + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/CssConf.scala b/common/src/main/scala/com/bytedance/css/common/CssConf.scala new file mode 100644 index 0000000..6354953 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/CssConf.scala @@ -0,0 +1,829 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common + +import java.util.{Map => JMap} +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ + +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.internal.config._ +import com.bytedance.css.common.protocol.ShuffleMode +import com.bytedance.css.common.util.Utils + +class CssConf(loadDefaults: Boolean) extends Cloneable with Logging with Serializable { + + import CssConf._ + + /** Create a CssConf that loads defaults from system properties and the classpath */ + def this() = this(true) + + private val settings = new ConcurrentHashMap[String, String]() + + @transient private lazy val reader: ConfigReader = { + val _reader = new ConfigReader(new CssConfigProvider(settings)) + _reader.bindEnv(new ConfigProvider { + override def get(key: String): Option[String] = Option(getenv(key)) + }) + _reader + } + + if (loadDefaults) { + loadFromSystemProperties(false) + } + + def loadFromSystemProperties(silent: Boolean): CssConf = { + // Load any spark.* system properties + for ((key, value) <- Utils.getSystemProperties if key.startsWith("css.")) { + set(key, value, silent) + } + this + } + + /** Set a configuration variable. */ + def set(key: String, value: String): CssConf = { + set(key, value, false) + } + + def set(key: String, value: String, silent: Boolean): CssConf = { + if (key == null) { + throw new NullPointerException("null key") + } + if (value == null) { + throw new NullPointerException("null value for " + key) + } + if (!silent) { + logDeprecationWarning(key) + } + settings.put(key, value) + this + } + + def set[T](entry: ConfigEntry[T], value: T): CssConf = { + set(entry.key, entry.stringConverter(value)) + this + } + + def set[T](entry: OptionalConfigEntry[T], value: T): CssConf = { + set(entry.key, entry.rawStringConverter(value)) + this + } + + /** Set multiple parameters together */ + def setAll(settings: Traversable[(String, String)]): CssConf = { + settings.foreach { case (k, v) => set(k, v) } + this + } + + /** Set a parameter if it isn't already configured */ + def setIfMissing(key: String, value: String): CssConf = { + if (settings.putIfAbsent(key, value) == null) { + logDeprecationWarning(key) + } + this + } + + def setIfMissing[T](entry: ConfigEntry[T], value: T): CssConf = { + if (settings.putIfAbsent(entry.key, entry.stringConverter(value)) == null) { + logDeprecationWarning(entry.key) + } + this + } + + def setIfMissing[T](entry: OptionalConfigEntry[T], value: T): CssConf = { + if (settings.putIfAbsent(entry.key, entry.rawStringConverter(value)) == null) { + logDeprecationWarning(entry.key) + } + this + } + + /** Remove a parameter from the configuration */ + def remove(key: String): CssConf = { + settings.remove(key) + this + } + + def remove(entry: ConfigEntry[_]): CssConf = { + remove(entry.key) + } + + /** Get a parameter; throws a NoSuchElementException if it's not set */ + def get(key: String): String = { + getOption(key).getOrElse(throw new NoSuchElementException(key)) + } + + /** Get a parameter, falling back to a default if not set */ + def get(key: String, defaultValue: String): String = { + getOption(key).getOrElse(defaultValue) + } + + /** + * Retrieves the value of a pre-defined configuration entry. + * + * - This is an internal Spark API. + * - The return type if defined by the configuration entry. + * - This will throw an exception is the config is not optional and the value is not set. + */ + def get[T](entry: ConfigEntry[T]): T = { + entry.readFrom(reader) + } + + /** + * Get a time parameter as seconds; throws a NoSuchElementException if it's not set. If no + * suffix is provided then seconds are assumed. + * @throws java.util.NoSuchElementException If the time parameter is not set + * @throws NumberFormatException If the value cannot be interpreted as seconds + */ + def getTimeAsSeconds(key: String): Long = catchIllegalValue(key) { + Utils.timeStringAsSeconds(get(key)) + } + + /** + * Get a time parameter as seconds, falling back to a default if not set. If no + * suffix is provided then seconds are assumed. + * @throws NumberFormatException If the value cannot be interpreted as seconds + */ + def getTimeAsSeconds(key: String, defaultValue: String): Long = catchIllegalValue(key) { + Utils.timeStringAsSeconds(get(key, defaultValue)) + } + + /** + * Get a time parameter as milliseconds; throws a NoSuchElementException if it's not set. If no + * suffix is provided then milliseconds are assumed. + * @throws java.util.NoSuchElementException If the time parameter is not set + * @throws NumberFormatException If the value cannot be interpreted as milliseconds + */ + def getTimeAsMs(key: String): Long = catchIllegalValue(key) { + Utils.timeStringAsMs(get(key)) + } + + /** + * Get a time parameter as milliseconds, falling back to a default if not set. If no + * suffix is provided then milliseconds are assumed. + * @throws NumberFormatException If the value cannot be interpreted as milliseconds + */ + def getTimeAsMs(key: String, defaultValue: String): Long = catchIllegalValue(key) { + Utils.timeStringAsMs(get(key, defaultValue)) + } + + /** + * Get a size parameter as bytes; throws a NoSuchElementException if it's not set. If no + * suffix is provided then bytes are assumed. + * @throws java.util.NoSuchElementException If the size parameter is not set + * @throws NumberFormatException If the value cannot be interpreted as bytes + */ + def getSizeAsBytes(key: String): Long = catchIllegalValue(key) { + Utils.byteStringAsBytes(get(key)) + } + + /** + * Get a size parameter as bytes, falling back to a default if not set. If no + * suffix is provided then bytes are assumed. + * @throws NumberFormatException If the value cannot be interpreted as bytes + */ + def getSizeAsBytes(key: String, defaultValue: String): Long = catchIllegalValue(key) { + Utils.byteStringAsBytes(get(key, defaultValue)) + } + + /** + * Get a size parameter as bytes, falling back to a default if not set. + * @throws NumberFormatException If the value cannot be interpreted as bytes + */ + def getSizeAsBytes(key: String, defaultValue: Long): Long = catchIllegalValue(key) { + Utils.byteStringAsBytes(get(key, defaultValue + "B")) + } + + /** + * Get a size parameter as Kibibytes; throws a NoSuchElementException if it's not set. If no + * suffix is provided then Kibibytes are assumed. + * @throws java.util.NoSuchElementException If the size parameter is not set + * @throws NumberFormatException If the value cannot be interpreted as Kibibytes + */ + def getSizeAsKb(key: String): Long = catchIllegalValue(key) { + Utils.byteStringAsKb(get(key)) + } + + /** + * Get a size parameter as Kibibytes, falling back to a default if not set. If no + * suffix is provided then Kibibytes are assumed. + * @throws NumberFormatException If the value cannot be interpreted as Kibibytes + */ + def getSizeAsKb(key: String, defaultValue: String): Long = catchIllegalValue(key) { + Utils.byteStringAsKb(get(key, defaultValue)) + } + + /** + * Get a size parameter as Mebibytes; throws a NoSuchElementException if it's not set. If no + * suffix is provided then Mebibytes are assumed. + * @throws java.util.NoSuchElementException If the size parameter is not set + * @throws NumberFormatException If the value cannot be interpreted as Mebibytes + */ + def getSizeAsMb(key: String): Long = catchIllegalValue(key) { + Utils.byteStringAsMb(get(key)) + } + + /** + * Get a size parameter as Mebibytes, falling back to a default if not set. If no + * suffix is provided then Mebibytes are assumed. + * @throws NumberFormatException If the value cannot be interpreted as Mebibytes + */ + def getSizeAsMb(key: String, defaultValue: String): Long = catchIllegalValue(key) { + Utils.byteStringAsMb(get(key, defaultValue)) + } + + /** + * Get a size parameter as Gibibytes; throws a NoSuchElementException if it's not set. If no + * suffix is provided then Gibibytes are assumed. + * @throws java.util.NoSuchElementException If the size parameter is not set + * @throws NumberFormatException If the value cannot be interpreted as Gibibytes + */ + def getSizeAsGb(key: String): Long = catchIllegalValue(key) { + Utils.byteStringAsGb(get(key)) + } + + /** + * Get a size parameter as Gibibytes, falling back to a default if not set. If no + * suffix is provided then Gibibytes are assumed. + * @throws NumberFormatException If the value cannot be interpreted as Gibibytes + */ + def getSizeAsGb(key: String, defaultValue: String): Long = catchIllegalValue(key) { + Utils.byteStringAsGb(get(key, defaultValue)) + } + + /** Get a parameter as an Option */ + def getOption(key: String): Option[String] = { + Option(settings.get(key)).orElse(getDeprecatedConfig(key, settings)) + } + + /** Get an optional value, applying variable substitution. */ + def getWithSubstitution(key: String): Option[String] = { + getOption(key).map(reader.substitute(_)) + } + + /** Get all parameters as a list of pairs */ + def getAll: Array[(String, String)] = { + settings.entrySet().asScala.map(x => (x.getKey, x.getValue)).toArray + } + + /** + * Get all parameters that start with `prefix` + */ + def getAllWithPrefix(prefix: String): Array[(String, String)] = { + getAll.filter { case (k, v) => k.startsWith(prefix) } + .map { case (k, v) => (k.substring(prefix.length), v) } + } + + + /** + * Get a parameter as an integer, falling back to a default if not set + * @throws NumberFormatException If the value cannot be interpreted as an integer + */ + def getInt(key: String, defaultValue: Int): Int = catchIllegalValue(key) { + getOption(key).map(_.toInt).getOrElse(defaultValue) + } + + /** + * Get a parameter as a long, falling back to a default if not set + * @throws NumberFormatException If the value cannot be interpreted as a long + */ + def getLong(key: String, defaultValue: Long): Long = catchIllegalValue(key) { + getOption(key).map(_.toLong).getOrElse(defaultValue) + } + + /** + * Get a parameter as a double, falling back to a default if not ste + * @throws NumberFormatException If the value cannot be interpreted as a double + */ + def getDouble(key: String, defaultValue: Double): Double = catchIllegalValue(key) { + getOption(key).map(_.toDouble).getOrElse(defaultValue) + } + + /** + * Get a parameter as a boolean, falling back to a default if not set + * @throws IllegalArgumentException If the value cannot be interpreted as a boolean + */ + def getBoolean(key: String, defaultValue: Boolean): Boolean = catchIllegalValue(key) { + getOption(key).map(_.toBoolean).getOrElse(defaultValue) + } + + /** Get all executor environment variables set on this CssConf */ + def getExecutorEnv: Seq[(String, String)] = { + getAllWithPrefix("spark.executorEnv.") + } + + /** + * Returns the Spark application id, valid in the Driver after TaskScheduler registration and + * from the start in the Executor. + */ + def getAppId: String = get("spark.app.id") + + /** Does the configuration contain a given parameter? */ + def contains(key: String): Boolean = { + settings.containsKey(key) || + configsWithAlternatives.get(key).toSeq.flatten.exists { alt => contains(alt.key) } + } + + def contains(entry: ConfigEntry[_]): Boolean = contains(entry.key) + + /** Copy this object */ + override def clone: CssConf = { + val cloned = new CssConf(false) + settings.entrySet().asScala.foreach { e => + cloned.set(e.getKey(), e.getValue(), true) + } + cloned + } + + /** + * By using this instead of System.getenv(), environment variables can be mocked + * in unit tests. + */ + def getenv(name: String): String = System.getenv(name) + + /** + * Wrapper method for get() methods which require some specific value format. This catches + * any [[NumberFormatException]] or [[IllegalArgumentException]] and re-raises it with the + * incorrectly configured key in the exception message. + */ + private def catchIllegalValue[T](key: String)(getValue: => T): T = { + try { + getValue + } catch { + case e: NumberFormatException => + // NumberFormatException doesn't have a constructor that takes a cause for some reason. + throw new NumberFormatException(s"Illegal value for config key $key: ${e.getMessage}") + .initCause(e) + case e: IllegalArgumentException => + throw new IllegalArgumentException(s"Illegal value for config key $key: ${e.getMessage}", e) + } + } +} + +object CssConf extends Logging { + + /** + * Maps deprecated config keys to information about the deprecation. + * + * The extra information is logged as a warning when the config is present in the user's + * configuration. + */ + private val deprecatedConfigs: Map[String, DeprecatedConfig] = { + val configs = Seq( + DeprecatedConfig("css.dummy.deprecatedKey", "", "") + ) + + Map(configs.map { cfg => (cfg.key -> cfg) } : _*) + } + + /** + * Maps a current config key to alternate keys that were used in previous version of Spark. + * + * The alternates are used in the order defined in this map. If deprecated configs are + * present in the user's configuration, a warning is logged. + * + * TODO: consolidate it with `ConfigBuilder.withAlternative`. + */ + private val configsWithAlternatives = Map[String, Seq[AlternateConfig]]( + "css.dummy.alternateKey" -> Seq( + AlternateConfig("css.dummy.alternateKey_1.0", "1.0")) + ) + + /** + * A view of `configsWithAlternatives` that makes it more efficient to look up deprecated + * config keys. + * + * Maps the deprecated config name to a 2-tuple (new config name, alternate config info). + */ + private val allAlternatives: Map[String, (String, AlternateConfig)] = { + configsWithAlternatives.keys.flatMap { key => + configsWithAlternatives(key).map { cfg => (cfg.key -> (key -> cfg)) } + }.toMap + } + + /** + * Looks for available deprecated keys for the given config option, and return the first + * value available. + */ + def getDeprecatedConfig(key: String, conf: JMap[String, String]): Option[String] = { + configsWithAlternatives.get(key).flatMap { alts => + alts.collectFirst { case alt if conf.containsKey(alt.key) => + val value = conf.get(alt.key) + if (alt.translation != null) alt.translation(value) else value + } + } + } + + /** + * Logs a warning message if the given config key is deprecated. + */ + def logDeprecationWarning(key: String): Unit = { + deprecatedConfigs.get(key).foreach { cfg => + logWarning( + s"The configuration key '$key' has been deprecated as of Spark ${cfg.version} and " + + s"may be removed in the future. ${cfg.deprecationMessage}") + return + } + + allAlternatives.get(key).foreach { case (newKey, cfg) => + logWarning( + s"The configuration key '$key' has been deprecated as of Spark ${cfg.version} and " + + s"may be removed in the future. Please use the new key '$newKey' instead.") + return + } + if (key.startsWith("spark.akka") || key.startsWith("spark.ssl.akka")) { + logWarning( + s"The configuration key $key is not supported anymore " + + s"because Spark doesn't use Akka since 2.0") + } + } + + /** + * Holds information about keys that have been deprecated and do not have a replacement. + * + * @param key The deprecated key. + * @param version Version of Spark where key was deprecated. + * @param deprecationMessage Message to include in the deprecation warning. + */ + private case class DeprecatedConfig( + key: String, + version: String, + deprecationMessage: String) + + /** + * Information about an alternate configuration key that has been deprecated. + * + * @param key The deprecated config key. + * @param version The Spark version in which the key was deprecated. + * @param translation A translation function for converting old config values into new ones. + */ + private case class AlternateConfig( + key: String, + version: String, + translation: String => String = null) + + def workerRegisterTimeoutMs(conf: CssConf): Long = { + conf.getTimeAsMs("css.worker.register.timeout", "180s") + } + + def flushBufferSize(conf: CssConf): Long = { + conf.getSizeAsBytes("css.flush.buffer.size", "256k") + } + + def flushQueueCapacity(conf: CssConf): Int = { + conf.getInt("css.flush.queue.capacity", 512) + } + + def flushTimeoutMs(conf: CssConf): Long = { + conf.getTimeAsMs("css.flush.timeout", "360s") + } + + def pushThreads(conf: CssConf): Int = { + conf.getInt("css.push.io.threads", 32) + } + + def pushServerPort(conf: CssConf, port: Int = -1): Int = { + if (port > 0) { + port + } else { + conf.getInt("css.push.server.port", 0) + } + } + + def fetchThreads(conf: CssConf): Int = { + conf.getInt("css.fetch.io.threads", 32) + } + + def dataThreads(conf: CssConf): Int = { + conf.getInt("css.data.io.threads", 8) + } + + def fetchServerPort(conf: CssConf, port: Int = -1): Int = { + if (port > 0) { + port + } else { + conf.getInt("css.fetch.server.port", 0) + } + } + + def commitThreads(conf: CssConf): Int = { + conf.getInt("css.commit.threads", 128) + } + + // css.diskFlusher.base.dirs = /mnt/disk1/css,/mnt/disk2/css,/mnt/disk3/css,/mnt/disk4/css, + // numbers of dirs will determine how many disk flusher will be. + def diskFlusherBaseDirs(conf: CssConf): Array[String] = { + val baseDirs = conf.get("css.diskFlusher.base.dirs", + Option(conf.getenv("LOCAL_DIRS")).getOrElse("/tmp/css")) + if (baseDirs.nonEmpty) { + baseDirs.split(",").filter(!_.isEmpty) + } else { + Array[String]() + } + } + + def diskFlusherNum(conf: CssConf): Int = { + conf.getInt("css.diskFlusher.num", -1) + } + + def hdfsFlusherBaseDir(conf: CssConf): String = { + conf.get("css.hdfsFlusher.base.dir", "") + } + + def hdfsFlusherNum(conf: CssConf): Int = { + conf.getInt("css.hdfsFlusher.num", 0) + } + + def hdfsFlusherReplica(conf: CssConf): String = { + conf.get("css.hdfsFlusher.replica", "2") + } + + def workerTimeoutMs(conf: CssConf): Long = { + conf.getTimeAsMs("css.worker.timeout", "120s") + } + + def workerLostExpireTimeMs(conf: CssConf): Long = { + conf.getTimeAsMs("css.worker.lost.expireTime", "120s") + } + + def appTimeoutMs(conf: CssConf): Long = { + conf.getTimeAsMs("css.app.timeout", "120s") + } + + def removeShuffleDelayMs(conf: CssConf): Long = { + conf.getTimeAsMs("css.remove.shuffle.delay", "60s") + } + + def fetchChunkSize(conf: CssConf): Long = { + conf.getSizeAsBytes("css.fetch.chunk.size", "8m") + } + + def pushBufferSize(conf: CssConf): Long = { + conf.getSizeAsBytes("css.push.buffer.size", "64k") + } + + def correctnessModeEnable(conf: CssConf): Boolean = { + conf.getBoolean("css.correctnessMode.enabled", false) + } + + def partitionGroupPushBufferSize(conf: CssConf): Long = { + conf.getSizeAsBytes("css.partitionGroup.push.buffer.size", "4m") + } + + def partitionGroupPushRetries(conf: CssConf): Int = { + conf.getInt("css.partitionGroup.push.maxRetries", 3) + } + + def maxPartitionsPerGroup(conf: CssConf): Int = { + conf.getInt("css.maxPartitionsPerGroup", 100) + } + + def pushIoMaxRetries(conf: CssConf): Int = { + conf.getInt("css.push.io.maxRetries", 3) + } + + def pushIoRetryWaitMs(conf: CssConf): Long = { + conf.getTimeAsMs("css.push.io.retryWait", "3s") + } + + def pushQueueCapacity(conf: CssConf): Int = { + conf.getInt("css.push.queue.capacity", 512) + } + + def clientMapperEndTimeoutMs(conf: CssConf): Long = { + conf.getTimeAsMs("css.client.mapper.end.timeout", "600s") + } + + def clientMapperEndSleepDeltaMs(conf: CssConf): Long = { + conf.getTimeAsMs("css.client.mapper.end.sleep.delta", "50ms") + } + + def pushDataRetryThreads(conf: CssConf): Int = { + conf.getInt("css.pushData.retry.threads", 64) + } + + def fetchChunkTimeoutMs(conf: CssConf): Long = { + conf.getTimeAsMs("css.fetch.chunk.timeout", "120s") + } + + def fetchChunkMaxReqsInFlight(conf: CssConf): Int = { + conf.getInt("css.fetch.chunk.maxReqsInFlight", 3) + } + + def commitFilesParallelism(conf: CssConf): Int = { + // parallelism from master to send commitFiles message to Workers + conf.getInt("css.commitFiles.parallelism", 128) + } + + def stageEndTimeoutMs(conf: CssConf): Long = { + conf.getTimeAsMs("css.stage.end.timeout", "600s") + } + + def stageEndRetryIntervalMs(conf: CssConf): Long = { + conf.getTimeAsMs("css.stageEnd.retry.interval", "200ms") + } + + def clientReallocateFailedMaxTimes(conf: CssConf): Int = { + conf.getInt("css.client.reallocate.failed.max.times", 3) + } + + def clientReallocateRetryIntervalMs(conf: CssConf): Long = { + conf.getTimeAsMs("css.client.reallocate.retry.interval", "3s") + } + + def localMode(conf: CssConf): Boolean = { + conf.getBoolean("css.local.mode", false) + } + + def masterAddress(conf: CssConf): String = { + conf.get("css.master.address", "css://localhost:9099") + } + + def clusterName(conf: CssConf): String = { + conf.get("css.cluster.name", "default") + } + + def haClusterName(conf: CssConf): String = { + conf.get("css.ha.cluster.name", "") + } + + def shuffleMode(conf: CssConf): ShuffleMode = { + // DISK HDFS + // scalastyle:off caselocale + ShuffleMode.valueOf(conf.get("css.shuffle.mode", "DISK").toUpperCase) + // scalastyle:on caselocale + } + + def zkAddress(conf: CssConf): String = { + conf.get("css.zookeeper.address", "") + } + + def zkRetries(conf: CssConf): Int = { + conf.getInt("css.zookeeper.retries", 5) + } + + def workerRegistryType(conf: CssConf): String = { + conf.get("css.worker.registry.type", "standalone") + } + + def workerUpdateIntervalMs(conf: CssConf): Long = { + conf.getTimeAsMs("css.worker.update.interval", "5s") + } + + def extMetaKeepaliveIntervalMs(conf: CssConf): Long = { + conf.getTimeAsMs("css.extMeta.keepalive.interval", "60s") + } + + def extMetaExpireIntervalMs(conf: CssConf): Long = { + conf.getTimeAsMs("css.extMeta.expire.interval", "480s") + } + + def cleanerPeriodicGCInterval(conf: CssConf): Long = { + conf.getTimeAsMs("css.cleaner.periodicGC.interval", "300s") + } + + def backpressureEnable(conf: CssConf): Boolean = { + conf.getBoolean("css.backpressure.enabled", true) + } + + def backpressureLogEnable(conf: CssConf): Boolean = { + conf.getBoolean("css.backpressure.log.enabled", false) + } + + def backpressureMaxConcurrency(conf: CssConf): Int = { + conf.getInt("css.backpressure.max.concurrency", 800) + } + + def backpressureMinLimit(conf: CssConf): Int = { + conf.getInt("css.backpressure.min.limit", 5) + } + + def backpressureSmoothing(conf: CssConf): Double = { + conf.getDouble("css.backpressure.smoothing", 0.2) + } + + def backpressureRttTolerance(conf: CssConf): Double = { + conf.getDouble("css.backpressure.rttTolerance", 1.5) + } + + def backpressureLongWindow(conf: CssConf): Int = { + conf.getInt("css.backpressure.longWindow", 10000) + } + + def backpressureQueueSize(conf: CssConf): Int = { + conf.getInt("css.backpressure.queueSize", 4) + } + + def fixRateLimitThreshold(conf: CssConf): Int = { + conf.getInt("css.fixRateLimit.threshold", 64) + } + + def epochRotateThreshold(conf: CssConf): Long = { + conf.getSizeAsBytes("css.epoch.rotate.threshold", "1g") + } + + def zkSessionTimeoutMs(conf: CssConf): Int = { + conf.getTimeAsMs("css.zk.timeout", "5min").toInt + } + + def zkMaxParallelism(conf: CssConf): Int = { + conf.getInt("css.zk.max.parallelism", 8) + } + + def zkMetaExpiredLogEnable(conf: CssConf): Boolean = { + conf.getBoolean("css.zk.metaExpired.log.enabled", true) + } + + def localChunkFetchEnable(conf: CssConf): Boolean = { + conf.getBoolean("css.local.chunk.fetch.enabled", true) + } + + def minDiskBaseDirNum(conf: CssConf): Int = { + conf.getInt("css.disk.dir.num.min", -1) + } + + def failedBatchBlacklistEnable(conf: CssConf): Boolean = { + /** + * When MapTask encounters onFailure, if the feature is turned on. + * the current reduceId-epochId-mapId-mapAttemptId-batchId will be recorded. + * and ShuffleReader will explicitly ignore the data when reading. + * because our deduplication is in Partition level deduplication. + * but in the case of AE skewjoin. + * the task read is likely to be an Epoch file. and the same Batch may appear in two Epochs. + * In AE skewjoin mode, this switch must be turned on, otherwise there will be correctness problems. + */ + conf.getBoolean("css.client.failed.batch.blacklist.enabled", true) + } + + def testMode(conf: CssConf): Boolean = { + conf.getBoolean("css.test.mode", false) + } + + def compressionTestMode(conf: CssConf): Boolean = { + conf.getBoolean("css.compression.test.mode", false) + } + + def compressionCodecType(conf: CssConf): String = { + // lz4, zstd + conf.get("css.compression.codec", "lz4") + } + + def zstdCompressionLevel(conf: CssConf): Int = { + conf.getInt("css.zstd.compression.level", 3) + } + + def zstdDictCompressionEnable(conf: CssConf): Boolean = { + conf.getBoolean("css.zstd.compression.dict.enabled", false) + } + + def sortPushSpillSizeThreshold(conf: CssConf): Long = { + // hit 256m to spill + conf.getSizeAsBytes("css.sortPush.spill.size.threshold", "256m") + } + + def sortPushSpillRecordThreshold(conf: CssConf): Long = { + // hit 1000000 record to spill + conf.getLong("css.sortPush.spill.record.threshold", 1000000) + } + + def chunkFetchRetryEnable(conf: CssConf): Boolean = { + conf.getBoolean("css.chunk.fetch.retry.enabled", true) + } + + def chunkFetchFailedRetryMaxTimes(conf: CssConf): Int = { + conf.getInt("css.chunk.fetch.failed.retry.max.times", 3) + } + + def chunkFetchRetryWaitTimes(conf: CssConf): Long = { + conf.getTimeAsMs("css.chunk.fetch.retry.wait.times", "5s") + } + + def maxAllocateWorker(conf: CssConf): Int = { + conf.getInt("css.max.allocate.worker", 1000) + } + + def workerAllocateExtraRatio(conf: CssConf): Double = { + conf.getDouble("css.worker.allocate.extraRatio", 1.5) + } + + def partitionAssignStrategy(conf: CssConf): String = { + conf.get("css.partition.assign.strategy", "random") + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/exception/AlreadyClosedException.scala b/common/src/main/scala/com/bytedance/css/common/exception/AlreadyClosedException.scala new file mode 100644 index 0000000..b182ac0 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/exception/AlreadyClosedException.scala @@ -0,0 +1,29 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.exception + +import java.io.IOException + +class AlreadyClosedException(message: String, cause: Throwable) + extends IOException(message, cause) { + + def this(message: String) = this(message, null) + +} diff --git a/common/src/main/scala/com/bytedance/css/common/exception/CssException.scala b/common/src/main/scala/com/bytedance/css/common/exception/CssException.scala new file mode 100644 index 0000000..c82b1ed --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/exception/CssException.scala @@ -0,0 +1,44 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.exception + +class CssException(message: String, cause: Throwable) + extends Exception(message, cause) { + + def this(message: String) = this(message, null) +} + +class PartitionInfoNotFoundException(message: String, cause: Throwable) + extends CssException(message, cause) { + + def this(message: String) = this(message, null) +} + +class StageEndedException(message: String, cause: Throwable) + extends CssException(message, cause) { + + def this(message: String) = this(message, null) +} + +class CssRuntimeException(message: String, cause: Throwable) + extends RuntimeException(message, cause) { + + def this(message: String) = this(message, null) +} diff --git a/common/src/main/scala/com/bytedance/css/common/exception/EpochShouldRotateException.scala b/common/src/main/scala/com/bytedance/css/common/exception/EpochShouldRotateException.scala new file mode 100644 index 0000000..c68a117 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/exception/EpochShouldRotateException.scala @@ -0,0 +1,28 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.exception + +import java.io.IOException + +class EpochShouldRotateException(message: String, cause: Throwable) + extends IOException(message, cause) { + + def this(message: String) = this(message, null) +} diff --git a/common/src/main/scala/com/bytedance/css/common/internal/Logging.scala b/common/src/main/scala/com/bytedance/css/common/internal/Logging.scala new file mode 100644 index 0000000..cd0a1e4 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/internal/Logging.scala @@ -0,0 +1,220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.internal + +import java.io.File + +import com.bytedance.css.common.util.Utils +import org.apache.log4j.{Level, LogManager, PropertyConfigurator} +import org.slf4j.{Logger, LoggerFactory} +import org.slf4j.impl.StaticLoggerBinder + +/** + * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows + * logging messages at different levels using methods that only evaluate parameters lazily if the + * log level is enabled. + */ +trait Logging { + + // Make the log field transient so that objects with Logging can + // be serialized and used on another machine + @transient private var log_ : Logger = null + + // Method to get the logger name for this object + protected def logName = { + // Ignore trailing $'s in the class names for Scala objects + this.getClass.getName.stripSuffix("$") + } + + // Method to get or create the logger for this object + protected def log: Logger = { + if (log_ == null) { + initializeLogIfNecessary(false) + log_ = LoggerFactory.getLogger(logName) + } + log_ + } + + // Log methods that take only a String + protected def logInfo(msg: => String) { + if (log.isInfoEnabled) log.info(msg) + } + + protected def logDebug(msg: => String) { + if (log.isDebugEnabled) log.debug(msg) + } + + protected def logTrace(msg: => String) { + if (log.isTraceEnabled) log.trace(msg) + } + + protected def logWarning(msg: => String) { + if (log.isWarnEnabled) log.warn(msg) + } + + protected def logError(msg: => String) { + if (log.isErrorEnabled) log.error(msg) + } + + // Log methods that take Throwables (Exceptions/Errors) too + protected def logInfo(msg: => String, throwable: Throwable) { + if (log.isInfoEnabled) log.info(msg, throwable) + } + + protected def logDebug(msg: => String, throwable: Throwable) { + if (log.isDebugEnabled) log.debug(msg, throwable) + } + + protected def logTrace(msg: => String, throwable: Throwable) { + if (log.isTraceEnabled) log.trace(msg, throwable) + } + + protected def logWarning(msg: => String, throwable: Throwable) { + if (log.isWarnEnabled) log.warn(msg, throwable) + } + + protected def logError(msg: => String, throwable: Throwable) { + if (log.isErrorEnabled) log.error(msg, throwable) + } + + protected def isTraceEnabled(): Boolean = { + log.isTraceEnabled + } + + protected def initializeLogIfNecessary(isInterpreter: Boolean): Unit = { + initializeLogIfNecessary(isInterpreter, silent = false) + } + + protected def initializeLogIfNecessary( + isInterpreter: Boolean, + silent: Boolean = false): Boolean = { + if (!Logging.initialized) { + Logging.initLock.synchronized { + if (!Logging.initialized) { + initializeLogging(isInterpreter, silent) + return true + } + } + } + false + } + + private def initializeLogging(isInterpreter: Boolean, silent: Boolean): Unit = { + // Don't use a logger in here, as this is itself occurring during initialization of a logger + // If Log4j 1.2 is being used, but is not initialized, load a default properties file + if (Logging.isLog4j12()) { + val log4j12Initialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements + // scalastyle:off println + if (!log4j12Initialized) { + Logging.defaultSparkLog4jConfig = true + var defaultLogProps = Utils.getClassLoader.getResource("log4j-defaults.properties") + val loggProps = sys.env.get("CSS_CONF_DIR") + .orElse(sys.env.get("CSS_HOME").map { t => s"$t${File.separator}conf" }) + .map { t => new File(s"$t${File.separator}log4j.properties") } + .filter(_.isFile) + .map(_.getAbsolutePath) + .getOrElse(null) + if (loggProps != null && new java.io.File(loggProps).exists() && new java.io.File(loggProps).canRead) { + defaultLogProps = new java.io.File(loggProps).toURL + } + Option(defaultLogProps) match { + case Some(url) => + PropertyConfigurator.configure(url) + if (!silent) { + System.err.println(s"Using Css's default log4j profile: $defaultLogProps") + } + case None => + System.err.println(s"Css was unable to load $defaultLogProps") + } + } + + val rootLogger = LogManager.getRootLogger() + if (Logging.defaultRootLevel == null) { + Logging.defaultRootLevel = rootLogger.getLevel() + } + + if (isInterpreter) { + // Use the repl's main class to define the default log level when running the shell, + // overriding the root logger's config if they're different. + val replLogger = LogManager.getLogger(logName) + val replLevel = Option(replLogger.getLevel()).getOrElse(Level.WARN) + if (replLevel != rootLogger.getEffectiveLevel()) { + if (!silent) { + System.err.printf("Setting default log level to \"%s\".\n", replLevel) + System.err.println("To adjust logging level use sc.setLogLevel(newLevel). " + + "For SparkR, use setLogLevel(newLevel).") + } + rootLogger.setLevel(replLevel) + } + } + // scalastyle:on println + } + Logging.initialized = true + + // Force a call into slf4j to initialize it. Avoids this happening from multiple threads + // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html + log + } +} + +object Logging { + @volatile private var initialized = false + @volatile private var defaultRootLevel: Level = null + @volatile private var defaultSparkLog4jConfig = false + + val initLock = new Object() + try { + // We use reflection here to handle the case where users remove the + // slf4j-to-jul bridge order to route their logs to JUL. + val bridgeClass = Utils.classForName("org.slf4j.bridge.SLF4JBridgeHandler") + bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null) + val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean] + if (!installed) { + bridgeClass.getMethod("install").invoke(null) + } + } catch { + case e: ClassNotFoundException => // can't log anything yet so just fail silently + } + + /** + * Marks the logging system as not initialized. This does a best effort at resetting the + * logging system to its initial state so that the next class to use logging triggers + * initialization again. + */ + def uninitialize(): Unit = initLock.synchronized { + if (isLog4j12()) { + if (defaultSparkLog4jConfig) { + defaultSparkLog4jConfig = false + LogManager.resetConfiguration() + } else { + LogManager.getRootLogger().setLevel(defaultRootLevel) + } + } + this.initialized = false + } + + private def isLog4j12(): Boolean = { + // This distinguishes the log4j 1.2 binding, currently + // org.slf4j.impl.Log4jLoggerFactory, from the log4j 2.0 binding, currently + // org.apache.logging.slf4j.Log4jLoggerFactory + val binderClass = StaticLoggerBinder.getSingleton.getLoggerFactoryClassStr + "org.slf4j.impl.Log4jLoggerFactory".equals(binderClass) + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/internal/config/ConfigBuilder.scala b/common/src/main/scala/com/bytedance/css/common/internal/config/ConfigBuilder.scala new file mode 100644 index 0000000..69775b2 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/internal/config/ConfigBuilder.scala @@ -0,0 +1,242 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.internal.config + +import java.util.concurrent.TimeUnit +import java.util.regex.PatternSyntaxException + +import scala.util.matching.Regex + +import com.bytedance.css.common.util.Utils +import com.bytedance.css.network.util.{ByteUnit, JavaUtils} + +private object ConfigHelpers { + + def toNumber[T](s: String, converter: String => T, key: String, configType: String): T = { + try { + converter(s.trim) + } catch { + case _: NumberFormatException => + throw new IllegalArgumentException(s"$key should be $configType, but was $s") + } + } + + def toBoolean(s: String, key: String): Boolean = { + try { + s.trim.toBoolean + } catch { + case _: IllegalArgumentException => + throw new IllegalArgumentException(s"$key should be boolean, but was $s") + } + } + + def stringToSeq[T](str: String, converter: String => T): Seq[T] = { + Utils.stringToSeq(str).map(converter) + } + + def seqToString[T](v: Seq[T], stringConverter: T => String): String = { + v.map(stringConverter).mkString(",") + } + + def timeFromString(str: String, unit: TimeUnit): Long = JavaUtils.timeStringAs(str, unit) + + def timeToString(v: Long, unit: TimeUnit): String = TimeUnit.MILLISECONDS.convert(v, unit) + "ms" + + def byteFromString(str: String, unit: ByteUnit): Long = { + val (input, multiplier) = + if (str.length() > 0 && str.charAt(0) == '-') { + (str.substring(1), -1) + } else { + (str, 1) + } + multiplier * JavaUtils.byteStringAs(input, unit) + } + + def byteToString(v: Long, unit: ByteUnit): String = unit.convertTo(v, ByteUnit.BYTE) + "b" + + def regexFromString(str: String, key: String): Regex = { + try str.r catch { + case e: PatternSyntaxException => + throw new IllegalArgumentException(s"$key should be a regex, but was $str", e) + } + } + +} + +class TypedConfigBuilder[T]( + val parent: ConfigBuilder, + val converter: String => T, + val stringConverter: T => String) { + + import ConfigHelpers._ + + def this(parent: ConfigBuilder, converter: String => T) = { + this(parent, converter, Option(_).map(_.toString).orNull) + } + + /** Apply a transformation to the user-provided values of the config entry. */ + def transform(fn: T => T): TypedConfigBuilder[T] = { + new TypedConfigBuilder(parent, s => fn(converter(s)), stringConverter) + } + + /** Checks if the user-provided value for the config matches the validator. */ + def checkValue(validator: T => Boolean, errorMsg: String): TypedConfigBuilder[T] = { + transform { v => + if (!validator(v)) throw new IllegalArgumentException(errorMsg) + v + } + } + + /** Check that user-provided values for the config match a pre-defined set. */ + def checkValues(validValues: Set[T]): TypedConfigBuilder[T] = { + transform { v => + if (!validValues.contains(v)) { + throw new IllegalArgumentException( + s"The value of ${parent.key} should be one of ${validValues.mkString(", ")}, but was $v") + } + v + } + } + + /** Turns the config entry into a sequence of values of the underlying type. */ + def toSequence: TypedConfigBuilder[Seq[T]] = { + new TypedConfigBuilder(parent, stringToSeq(_, converter), seqToString(_, stringConverter)) + } + + /** Creates a [[ConfigEntry]] that does not have a default value. */ + def createOptional: OptionalConfigEntry[T] = { + val entry = new OptionalConfigEntry[T](parent.key, parent._alternatives, converter, + stringConverter, parent._doc, parent._public) + parent._onCreate.foreach(_(entry)) + entry + } + + /** Creates a [[ConfigEntry]] that has a default value. */ + def createWithDefault(default: T): ConfigEntry[T] = { + // Treat "String" as a special case, so that both createWithDefault and createWithDefaultString + // behave the same w.r.t. variable expansion of default values. + if (default.isInstanceOf[String]) { + createWithDefaultString(default.asInstanceOf[String]) + } else { + val transformedDefault = converter(stringConverter(default)) + val entry = new ConfigEntryWithDefault[T](parent.key, parent._alternatives, + transformedDefault, converter, stringConverter, parent._doc, parent._public) + parent._onCreate.foreach(_(entry)) + entry + } + } + + /** Creates a [[ConfigEntry]] with a function to determine the default value */ + def createWithDefaultFunction(defaultFunc: () => T): ConfigEntry[T] = { + val entry = new ConfigEntryWithDefaultFunction[T](parent.key, parent._alternatives, defaultFunc, + converter, stringConverter, parent._doc, parent._public) + parent._onCreate.foreach(_ (entry)) + entry + } + + /** + * Creates a [[ConfigEntry]] that has a default value. The default value is provided as a + * [[String]] and must be a valid value for the entry. + */ + def createWithDefaultString(default: String): ConfigEntry[T] = { + val entry = new ConfigEntryWithDefaultString[T](parent.key, parent._alternatives, default, + converter, stringConverter, parent._doc, parent._public) + parent._onCreate.foreach(_(entry)) + entry + } + +} + +/** + * Basic builder for Spark configurations. Provides methods for creating type-specific builders. + * + * @see TypedConfigBuilder + */ +case class ConfigBuilder(key: String) { + + import ConfigHelpers._ + + private[config] var _public = true + private[config] var _doc = "" + private[config] var _onCreate: Option[ConfigEntry[_] => Unit] = None + private[config] var _alternatives = List.empty[String] + + def internal(): ConfigBuilder = { + _public = false + this + } + + def doc(s: String): ConfigBuilder = { + _doc = s + this + } + + /** + * Registers a callback for when the config entry is finally instantiated. Currently used by + * SQLConf to keep track of SQL configuration entries. + */ + def onCreate(callback: ConfigEntry[_] => Unit): ConfigBuilder = { + _onCreate = Option(callback) + this + } + + def withAlternative(key: String): ConfigBuilder = { + _alternatives = _alternatives :+ key + this + } + + def intConf: TypedConfigBuilder[Int] = { + new TypedConfigBuilder(this, toNumber(_, _.toInt, key, "int")) + } + + def longConf: TypedConfigBuilder[Long] = { + new TypedConfigBuilder(this, toNumber(_, _.toLong, key, "long")) + } + + def doubleConf: TypedConfigBuilder[Double] = { + new TypedConfigBuilder(this, toNumber(_, _.toDouble, key, "double")) + } + + def booleanConf: TypedConfigBuilder[Boolean] = { + new TypedConfigBuilder(this, toBoolean(_, key)) + } + + def stringConf: TypedConfigBuilder[String] = { + new TypedConfigBuilder(this, v => v) + } + + def timeConf(unit: TimeUnit): TypedConfigBuilder[Long] = { + new TypedConfigBuilder(this, timeFromString(_, unit), timeToString(_, unit)) + } + + def bytesConf(unit: ByteUnit): TypedConfigBuilder[Long] = { + new TypedConfigBuilder(this, byteFromString(_, unit), byteToString(_, unit)) + } + + def fallbackConf[T](fallback: ConfigEntry[T]): ConfigEntry[T] = { + val entry = new FallbackConfigEntry(key, _alternatives, _doc, _public, fallback) + _onCreate.foreach(_(entry)) + entry + } + + def regexConf: TypedConfigBuilder[Regex] = { + new TypedConfigBuilder(this, regexFromString(_, this.key), _.toString) + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/internal/config/ConfigEntry.scala b/common/src/main/scala/com/bytedance/css/common/internal/config/ConfigEntry.scala new file mode 100644 index 0000000..f54a4ed --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/internal/config/ConfigEntry.scala @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.internal.config + +/** + * An entry contains all meta information for a configuration. + * + * When applying variable substitution to config values, only references starting with "spark." are + * considered in the default namespace. For known Spark configuration keys (i.e. those created using + * `ConfigBuilder`), references will also consider the default value when it exists. + * + * Variable expansion is also applied to the default values of config entries that have a default + * value declared as a string. + * + * @param key the key for the configuration + * @param valueConverter how to convert a string to the value. It should throw an exception if the + * string does not have the required format. + * @param stringConverter how to convert a value to a string that the user can use it as a valid + * string value. It's usually `toString`. But sometimes, a custom converter + * is necessary. E.g., if T is List[String], `a, b, c` is better than + * `List(a, b, c)`. + * @param doc the documentation for the configuration + * @param isPublic if this configuration is public to the user. If it's `false`, this + * configuration is only used internally and we should not expose it to users. + * @tparam T the value type + */ +abstract class ConfigEntry[T] ( + val key: String, + val alternatives: List[String], + val valueConverter: String => T, + val stringConverter: T => String, + val doc: String, + val isPublic: Boolean) { + + import ConfigEntry._ + + registerEntry(this) + + def defaultValueString: String + + protected def readString(reader: ConfigReader): Option[String] = { + alternatives.foldLeft(reader.get(key))((res, nextKey) => res.orElse(reader.get(nextKey))) + } + + def readFrom(reader: ConfigReader): T + + def defaultValue: Option[T] = None + + override def toString: String = { + s"ConfigEntry(key=$key, defaultValue=$defaultValueString, doc=$doc, public=$isPublic)" + } +} + +private class ConfigEntryWithDefault[T] ( + key: String, + alternatives: List[String], + _defaultValue: T, + valueConverter: String => T, + stringConverter: T => String, + doc: String, + isPublic: Boolean) + extends ConfigEntry(key, alternatives, valueConverter, stringConverter, doc, isPublic) { + + override def defaultValue: Option[T] = Some(_defaultValue) + + override def defaultValueString: String = stringConverter(_defaultValue) + + def readFrom(reader: ConfigReader): T = { + readString(reader).map(valueConverter).getOrElse(_defaultValue) + } +} + +private class ConfigEntryWithDefaultFunction[T] ( + key: String, + alternatives: List[String], + _defaultFunction: () => T, + valueConverter: String => T, + stringConverter: T => String, + doc: String, + isPublic: Boolean) + extends ConfigEntry(key, alternatives, valueConverter, stringConverter, doc, isPublic) { + + override def defaultValue: Option[T] = Some(_defaultFunction()) + + override def defaultValueString: String = stringConverter(_defaultFunction()) + + def readFrom(reader: ConfigReader): T = { + readString(reader).map(valueConverter).getOrElse(_defaultFunction()) + } +} + +private class ConfigEntryWithDefaultString[T] ( + key: String, + alternatives: List[String], + _defaultValue: String, + valueConverter: String => T, + stringConverter: T => String, + doc: String, + isPublic: Boolean) + extends ConfigEntry(key, alternatives, valueConverter, stringConverter, doc, isPublic) { + + override def defaultValue: Option[T] = Some(valueConverter(_defaultValue)) + + override def defaultValueString: String = _defaultValue + + def readFrom(reader: ConfigReader): T = { + val value = readString(reader).getOrElse(reader.substitute(_defaultValue)) + valueConverter(value) + } +} + + +/** + * A config entry that does not have a default value. + */ +class OptionalConfigEntry[T]( + key: String, + alternatives: List[String], + val rawValueConverter: String => T, + val rawStringConverter: T => String, + doc: String, + isPublic: Boolean) + extends ConfigEntry[Option[T]](key, alternatives, + s => Some(rawValueConverter(s)), + v => v.map(rawStringConverter).orNull, doc, isPublic) { + + override def defaultValueString: String = ConfigEntry.UNDEFINED + + override def readFrom(reader: ConfigReader): Option[T] = { + readString(reader).map(rawValueConverter) + } +} + +/** + * A config entry whose default value is defined by another config entry. + */ +class FallbackConfigEntry[T] ( + key: String, + alternatives: List[String], + doc: String, + isPublic: Boolean, + val fallback: ConfigEntry[T]) + extends ConfigEntry[T](key, alternatives, + fallback.valueConverter, fallback.stringConverter, doc, isPublic) { + + override def defaultValueString: String = s"" + + override def readFrom(reader: ConfigReader): T = { + readString(reader).map(valueConverter).getOrElse(fallback.readFrom(reader)) + } +} + +object ConfigEntry { + + val UNDEFINED = "" + + private val knownConfigs = new java.util.concurrent.ConcurrentHashMap[String, ConfigEntry[_]]() + + def registerEntry(entry: ConfigEntry[_]): Unit = { + val existing = knownConfigs.putIfAbsent(entry.key, entry) + require(existing == null, s"Config entry ${entry.key} already registered!") + } + + def findEntry(key: String): ConfigEntry[_] = knownConfigs.get(key) + +} diff --git a/common/src/main/scala/com/bytedance/css/common/internal/config/ConfigProvider.scala b/common/src/main/scala/com/bytedance/css/common/internal/config/ConfigProvider.scala new file mode 100644 index 0000000..be0fc8c --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/internal/config/ConfigProvider.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.internal.config + +import java.util.{Map => JMap} + +import com.bytedance.css.common.CssConf + +/** + * A source of configuration values. + */ +trait ConfigProvider { + + def get(key: String): Option[String] + +} + +class EnvProvider extends ConfigProvider { + + override def get(key: String): Option[String] = sys.env.get(key) + +} + +class SystemProvider extends ConfigProvider { + + override def get(key: String): Option[String] = sys.props.get(key) + +} + +class MapProvider(conf: JMap[String, String]) extends ConfigProvider { + + override def get(key: String): Option[String] = Option(conf.get(key)) + +} + +/** + * A config provider that only reads Spark config keys. + */ +class CssConfigProvider(conf: JMap[String, String]) extends ConfigProvider { + + override def get(key: String): Option[String] = { + if (key.startsWith("css.")) { + Option(conf.get(key)).orElse(CssConf.getDeprecatedConfig(key, conf)) + } else { + None + } + } + +} diff --git a/common/src/main/scala/com/bytedance/css/common/internal/config/ConfigReader.scala b/common/src/main/scala/com/bytedance/css/common/internal/config/ConfigReader.scala new file mode 100644 index 0000000..a741996 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/internal/config/ConfigReader.scala @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.internal.config + +import java.util.{Map => JMap} + +import scala.collection.mutable.HashMap +import scala.util.matching.Regex + +private object ConfigReader { + + private val REF_RE = "\\$\\{(?:(\\w+?):)?(\\S+?)\\}".r + +} + +/** + * A helper class for reading config entries and performing variable substitution. + * + * If a config value contains variable references of the form "${prefix:variableName}", the + * reference will be replaced with the value of the variable depending on the prefix. By default, + * the following prefixes are handled: + * + * - no prefix: use the default config provider + * - system: looks for the value in the system properties + * - env: looks for the value in the environment + * + * Different prefixes can be bound to a `ConfigProvider`, which is used to read configuration + * values from the data source for the prefix, and both the system and env providers can be + * overridden. + * + * If the reference cannot be resolved, the original string will be retained. + * + * @param conf The config provider for the default namespace (no prefix). + */ +class ConfigReader(conf: ConfigProvider) { + + def this(conf: JMap[String, String]) = this(new MapProvider(conf)) + + private val bindings = new HashMap[String, ConfigProvider]() + bind(null, conf) + bindEnv(new EnvProvider()) + bindSystem(new SystemProvider()) + + /** + * Binds a prefix to a provider. This method is not thread-safe and should be called + * before the instance is used to expand values. + */ + def bind(prefix: String, provider: ConfigProvider): ConfigReader = { + bindings(prefix) = provider + this + } + + def bind(prefix: String, values: JMap[String, String]): ConfigReader = { + bind(prefix, new MapProvider(values)) + } + + def bindEnv(provider: ConfigProvider): ConfigReader = bind("env", provider) + + def bindSystem(provider: ConfigProvider): ConfigReader = bind("system", provider) + + /** + * Reads a configuration key from the default provider, and apply variable substitution. + */ + def get(key: String): Option[String] = conf.get(key).map(substitute) + + /** + * Perform variable substitution on the given input string. + */ + def substitute(input: String): String = substitute(input, Set()) + + private def substitute(input: String, usedRefs: Set[String]): String = { + if (input != null) { + ConfigReader.REF_RE.replaceAllIn(input, { m => + val prefix = m.group(1) + val name = m.group(2) + val ref = if (prefix == null) name else s"$prefix:$name" + require(!usedRefs.contains(ref), s"Circular reference in $input: $ref") + + val replacement = bindings.get(prefix) + .flatMap(getOrDefault(_, name)) + .map { v => substitute(v, usedRefs + ref) } + .getOrElse(m.matched) + Regex.quoteReplacement(replacement) + }) + } else { + input + } + } + + /** + * Gets the value of a config from the given `ConfigProvider`. If no value is found for this + * config, and the `ConfigEntry` defines this config has default value, return the default value. + */ + private def getOrDefault(conf: ConfigProvider, key: String): Option[String] = { + conf.get(key).orElse { + ConfigEntry.findEntry(key) match { + case e: ConfigEntryWithDefault[_] => Option(e.defaultValueString) + case e: ConfigEntryWithDefaultString[_] => Option(e.defaultValueString) + case e: ConfigEntryWithDefaultFunction[_] => Option(e.defaultValueString) + case e: FallbackConfigEntry[_] => getOrDefault(conf, e.fallback.key) + case _ => None + } + } + } + +} diff --git a/common/src/main/scala/com/bytedance/css/common/internal/config/package.scala b/common/src/main/scala/com/bytedance/css/common/internal/config/package.scala new file mode 100644 index 0000000..2b7c5c9 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/internal/config/package.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.internal + +package object config { +} diff --git a/common/src/main/scala/com/bytedance/css/common/metrics/MetricsConfig.scala b/common/src/main/scala/com/bytedance/css/common/metrics/MetricsConfig.scala new file mode 100644 index 0000000..7b5196a --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/metrics/MetricsConfig.scala @@ -0,0 +1,155 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.metrics + +import java.io.{File, FileInputStream, InputStream} +import java.util.Properties + +import scala.collection.{mutable, Map} +import scala.collection.JavaConverters._ +import scala.util.matching.Regex + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.util.Utils + +class MetricsConfig(conf: CssConf) extends Logging { + + private val DEFAULT_PREFIX = "*" + private val INSTANCE_REGEX = "^(\\*|[a-zA-Z]+)\\.(.+)".r + private val DEFAULT_METRICS_CONF_FILENAME = "css-metrics.properties" + + val properties = new Properties() + var perInstanceSubProperties: mutable.HashMap[String, Properties] = null + + def getDefaultMetricsFile(env: Map[String, String] = sys.env): Option[String] = { + env.get("CSS_CONF_DIR") + .orElse(env.get("CSS_HOME").map { t => s"$t${File.separator}conf" }) + .map { t => new File(s"$t${File.separator}${DEFAULT_METRICS_CONF_FILENAME}") } + .filter(_.isFile) + .map(_.getAbsolutePath) + } + + /** + * Load properties from various places, based on precedence + * If the same property is set again latter on in the method, it overwrites the previous value + */ + def initialize(): Unit = { + + val metricsFile = conf.getOption("css.metrics.conf") + .orElse(getDefaultMetricsFile()) + + loadPropertiesFromFile(metricsFile) + + // Also look for the properties in provided Css configuration + val prefix = "css.metrics.conf." + conf.getAll.foreach { + case (k, v) if k.startsWith(prefix) => + properties.setProperty(k.substring(prefix.length()), v) + case _ => + } + + // Now, let's populate a list of sub-properties per instance, instance being the prefix that + // appears before the first dot in the property name. + // Add to the sub-properties per instance, the default properties (those with prefix "*"), if + // they don't have that exact same sub-property already defined. + // + // For example, if properties has ("*.class"->"default_class", "*.path"->"default_path, + // "driver.path"->"driver_path"), for driver specific sub-properties, we'd like the output to be + // ("driver"->Map("path"->"driver_path", "class"->"default_class") + // Note how class got added to based on the default property, but path remained the same + // since "driver.path" already existed and took precedence over "*.path" + // + perInstanceSubProperties = subProperties(properties, INSTANCE_REGEX) + if (perInstanceSubProperties.contains(DEFAULT_PREFIX)) { + val defaultSubProperties = perInstanceSubProperties(DEFAULT_PREFIX).asScala + for ((instance, prop) <- perInstanceSubProperties if (instance != DEFAULT_PREFIX); + (k, v) <- defaultSubProperties if (prop.get(k) == null)) { + prop.put(k, v) + } + } + } + + /** + * Take a simple set of properties and a regex that the instance names (part before the first dot) + * have to conform to. And, return a map of the first order prefix (before the first dot) to the + * sub-properties under that prefix. + * + * For example, if the properties sent were Properties("*.sink.servlet.class"->"class1", + * "*.sink.servlet.path"->"path1"), the returned map would be + * Map("*" -> Properties("sink.servlet.class" -> "class1", "sink.servlet.path" -> "path1")) + * Note in the subProperties (value of the returned Map), only the suffixes are used as property + * keys. + * If, in the passed properties, there is only one property with a given prefix, it is still + * "unflattened". For example, if the input was Properties("*.sink.servlet.class" -> "class1" + * the returned Map would contain one key-value pair + * Map("*" -> Properties("sink.servlet.class" -> "class1")) + * Any passed in properties, not complying with the regex are ignored. + * + * @param prop the flat list of properties to "unflatten" based on prefixes + * @param regex the regex that the prefix has to comply with + * @return an unflattened map, mapping prefix with sub-properties under that prefix + */ + def subProperties(prop: Properties, regex: Regex): mutable.HashMap[String, Properties] = { + val subProperties = new mutable.HashMap[String, Properties] + prop.asScala.foreach { kv => + if (regex.findPrefixOf(kv._1.toString).isDefined) { + val regex(prefix, suffix) = kv._1.toString + subProperties.getOrElseUpdate(prefix, new Properties).setProperty(suffix, kv._2.toString) + } + } + subProperties + } + + def getInstance(inst: String): Properties = { + perInstanceSubProperties.get(inst) match { + case Some(s) => s + case None => perInstanceSubProperties.getOrElse(DEFAULT_PREFIX, new Properties) + } + } + + /** + * Loads configuration from a config file. If no config file is provided, try to get file + * in class path. + */ + private[this] def loadPropertiesFromFile(path: Option[String]): Unit = { + logInfo(s"Using Css's metrics file: " + + path.getOrElse(Utils.getContextOrClassLoader.getResource(DEFAULT_METRICS_CONF_FILENAME))) + var is: InputStream = null + try { + is = path match { + case Some(f) => new FileInputStream(f) + case None => Utils.getContextOrClassLoader.getResourceAsStream(DEFAULT_METRICS_CONF_FILENAME) + } + if (is != null) { + properties.load(is) + } + } catch { + case e: Exception => + val file = path.getOrElse(DEFAULT_METRICS_CONF_FILENAME) + logError(s"Error loading configuration file $file", e) + } finally { + if (is != null) { + is.close() + } + } + } + +} diff --git a/common/src/main/scala/com/bytedance/css/common/metrics/MetricsSystem.scala b/common/src/main/scala/com/bytedance/css/common/metrics/MetricsSystem.scala new file mode 100644 index 0000000..4bef396 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/metrics/MetricsSystem.scala @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.metrics + +import java.util.Properties +import java.util.concurrent.TimeUnit + +import scala.collection.mutable + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.metrics.sink.Sink +import com.bytedance.css.common.metrics.source.Source +import com.bytedance.css.common.util.Utils +import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry} + +/** + * Spark Metrics System, created by a specific "instance", combined by source, + * sink, periodically polls source metrics data to sink destinations. + * + * "instance" specifies "who" (the role) uses the metrics system. In Spark, there are several roles + * like master, worker, executor, client driver. These roles will create metrics system + * for monitoring. So, "instance" represents these roles. Currently in Spark, several instances + * have already implemented: master, worker, executor, driver, applications. + * + * "source" specifies "where" (source) to collect metrics data from. In metrics system, there exists + * two kinds of source: + * 1. Spark internal source, like MasterSource, WorkerSource, etc, which will collect + * Spark component's internal state, these sources are related to instance and will be + * added after a specific metrics system is created. + * 2. Common source, like JvmSource, which will collect low level state, is configured by + * configuration and loaded through reflection. + * + * "sink" specifies "where" (destination) to output metrics data to. Several sinks can + * coexist and metrics can be flushed to all these sinks. + * + * Metrics configuration format is like below: + * [instance].[sink|source].[name].[options] = xxxx + * + * [instance] can be "master", "worker", "executor", "driver", "applications" which means only + * the specified instance has this property. + * wild card "*" can be used to replace instance name, which means all the instances will have + * this property. + * + * [sink|source] means this property belongs to source or sink. This field can only be + * source or sink. + * + * [name] specify the name of sink or source, if it is custom defined. + * + * [options] represent the specific property of this source or sink. + */ +class MetricsSystem(val instance: String, val conf: CssConf) extends Logging { + + private[this] val metricsConfig = new MetricsConfig(conf) + + private val sinks = new mutable.ArrayBuffer[Sink] + private val sources = new mutable.ArrayBuffer[Source] + private val registry = new MetricRegistry() + + private var running: Boolean = false + + metricsConfig.initialize() + + def start(): Unit = { + require(!running, "Attempting to start a MetricsSystem that is already running") + running = true + registerSources() + registerSinks() + sinks.foreach(_.start()) + } + + def stop(): Unit = { + if (running) { + sinks.foreach(_.stop()) + } else { + logWarning("Stopping a MetricsSystem that is not running") + } + running = false + } + + def report(): Unit = { + sinks.foreach(_.report()) + } + + def getMetricRegistry: MetricRegistry = { + registry + } + + private def buildRegistryName(source: Source): String = { + val defaultName = MetricRegistry.name(source.sourceName) + defaultName + } + + def getSourcesByName(sourceName: String): Seq[Source] = + sources.filter(_.sourceName == sourceName) + + def registerSource(source: Source) { + sources += source + try { + val regName = buildRegistryName(source) + registry.register(regName, source.metricRegistry) + } catch { + case e: IllegalArgumentException => logInfo("Metrics already registered", e) + } + } + + def removeSource(source: Source) { + sources -= source + val regName = buildRegistryName(source) + registry.removeMatching(new MetricFilter { + def matches(name: String, metric: Metric): Boolean = name.startsWith(regName) + }) + } + + private def registerSources() { + val instConfig = metricsConfig.getInstance(instance) + val sourceConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SOURCE_REGEX) + + // Register all the sources related to instance + sourceConfigs.foreach { kv => + val classPath = kv._2.getProperty("class") + try { + val source = Utils.classForName(classPath).newInstance() + registerSource(source.asInstanceOf[Source]) + } catch { + case e: Exception => logError("Source class " + classPath + " cannot be instantiated", e) + } + } + } + + private def registerSinks() { + val instConfig = metricsConfig.getInstance(instance) + val sinkConfigs = metricsConfig.subProperties(instConfig, MetricsSystem.SINK_REGEX) + + sinkConfigs.foreach { kv => + val classPath = kv._2.getProperty("class") + if (null != classPath) { + try { + val sink = Utils.classForName(classPath) + .getConstructor(classOf[Properties], classOf[MetricRegistry]) + .newInstance(kv._2, registry) + if (kv._1 == "servlet") { + // metricsServlet = Some(sink.asInstanceOf[MetricsServlet]) + } else { + sinks += sink.asInstanceOf[Sink] + } + } catch { + case e: Exception => + logError("Sink class " + classPath + " cannot be instantiated") + throw e + } + } + } + } +} + +object MetricsSystem { + val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r + val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r + + private[this] val MINIMAL_POLL_UNIT = TimeUnit.SECONDS + private[this] val MINIMAL_POLL_PERIOD = 1 + + def checkMinimalPollingPeriod(pollUnit: TimeUnit, pollPeriod: Int): Unit = { + val period = MINIMAL_POLL_UNIT.convert(pollPeriod, pollUnit) + if (period < MINIMAL_POLL_PERIOD) { + throw new IllegalArgumentException("Polling period " + pollPeriod + " " + pollUnit + + " below than minimal polling period ") + } + } + + def createMetricsSystem(instance: String, conf: CssConf): MetricsSystem = { + new MetricsSystem(instance, conf) + } + + val CLIENT = "client" + val MASTER = "master" + val WORKER = "worker" +} diff --git a/common/src/main/scala/com/bytedance/css/common/metrics/sink/BytedanceSink.scala b/common/src/main/scala/com/bytedance/css/common/metrics/sink/BytedanceSink.scala new file mode 100644 index 0000000..fc8a797 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/metrics/sink/BytedanceSink.scala @@ -0,0 +1,69 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.metrics.sink + +import java.util.Properties +import java.util.concurrent.TimeUnit + +import com.bytedance.css.common.metrics.{ByteDanceMetricsEmitter, ByteDanceMetricsReporter, MetricsSystem} +import com.codahale.metrics.{MetricFilter, MetricRegistry} + +class BytedanceSink(val property: Properties, val registry: MetricRegistry) extends Sink { + val KEY_PERIOD = "period" + val KEY_UNIT = "unit" + val PREFIX = "prefix" + + val DEFAULT_PERIOD = 5 + val DEFAULT_UNIT = "SECONDS" + val DEFAULT_PREFIX = "inf.css.v2" + + val pollPeriod = Option(property.getProperty(KEY_PERIOD)) match { + case Some(s) => s.toInt + case None => DEFAULT_PERIOD + } + + val pollUnit: TimeUnit = Option(property.getProperty(KEY_UNIT)) match { + case Some(s) => TimeUnit.valueOf(s.toUpperCase()) + case None => TimeUnit.valueOf(DEFAULT_UNIT) + } + + val prefix: String = Option(property.getProperty(PREFIX)) match { + case Some(s) => s + case None => DEFAULT_PREFIX + } + + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) + + val emitter = ByteDanceMetricsEmitter.getEmitter(prefix) + val reporter = new ByteDanceMetricsReporter(emitter, registry, MetricFilter.ALL, + TimeUnit.SECONDS, TimeUnit.MILLISECONDS) + + override def start() { + reporter.start(pollPeriod, pollUnit) + } + + override def stop() { + reporter.stop() + } + + override def report() { + reporter.report() + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/metrics/sink/ConsoleSink.scala b/common/src/main/scala/com/bytedance/css/common/metrics/sink/ConsoleSink.scala new file mode 100644 index 0000000..8f379d7 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/metrics/sink/ConsoleSink.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.metrics.sink + +import java.util.{Locale, Properties} +import java.util.concurrent.TimeUnit + +import com.bytedance.css.common.metrics.MetricsSystem +import com.codahale.metrics.{ConsoleReporter, MetricRegistry} + +class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { + + val CONSOLE_DEFAULT_PERIOD = 10 + val CONSOLE_DEFAULT_UNIT = "SECONDS" + + val CONSOLE_KEY_PERIOD = "period" + val CONSOLE_KEY_UNIT = "unit" + + val pollPeriod = Option(property.getProperty(CONSOLE_KEY_PERIOD)) match { + case Some(s) => s.toInt + case None => CONSOLE_DEFAULT_PERIOD + } + + val pollUnit: TimeUnit = Option(property.getProperty(CONSOLE_KEY_UNIT)) match { + case Some(s) => TimeUnit.valueOf(s.toUpperCase(Locale.ROOT)) + case None => TimeUnit.valueOf(CONSOLE_DEFAULT_UNIT) + } + + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) + + val reporter: ConsoleReporter = ConsoleReporter.forRegistry(registry) + .convertDurationsTo(TimeUnit.MILLISECONDS) + .convertRatesTo(TimeUnit.SECONDS) + .build() + + override def start(): Unit = { + reporter.start(pollPeriod, pollUnit) + } + + override def stop(): Unit = { + reporter.stop() + } + + override def report(): Unit = { + reporter.report() + } +} + diff --git a/common/src/main/scala/com/bytedance/css/common/metrics/sink/CsvSink.scala b/common/src/main/scala/com/bytedance/css/common/metrics/sink/CsvSink.scala new file mode 100644 index 0000000..4c4dce0 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/metrics/sink/CsvSink.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.metrics.sink + +import java.io.File +import java.util.{Locale, Properties} +import java.util.concurrent.TimeUnit + +import com.bytedance.css.common.metrics.MetricsSystem +import com.codahale.metrics.{CsvReporter, MetricRegistry} + +class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { + + val CSV_KEY_PERIOD = "period" + val CSV_KEY_UNIT = "unit" + val CSV_KEY_DIR = "directory" + + val CSV_DEFAULT_PERIOD = 10 + val CSV_DEFAULT_UNIT = "SECONDS" + val CSV_DEFAULT_DIR = "/tmp/" + + val pollPeriod = Option(property.getProperty(CSV_KEY_PERIOD)) match { + case Some(s) => s.toInt + case None => CSV_DEFAULT_PERIOD + } + + val pollUnit: TimeUnit = Option(property.getProperty(CSV_KEY_UNIT)) match { + case Some(s) => TimeUnit.valueOf(s.toUpperCase(Locale.ROOT)) + case None => TimeUnit.valueOf(CSV_DEFAULT_UNIT) + } + + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) + + val pollDir = Option(property.getProperty(CSV_KEY_DIR)) match { + case Some(s) => s + case None => CSV_DEFAULT_DIR + } + + val reporter: CsvReporter = CsvReporter.forRegistry(registry) + .formatFor(Locale.US) + .convertDurationsTo(TimeUnit.MILLISECONDS) + .convertRatesTo(TimeUnit.SECONDS) + .build(new File(pollDir)) + + override def start(): Unit = { + reporter.start(pollPeriod, pollUnit) + } + + override def stop(): Unit = { + reporter.stop() + } + + override def report(): Unit = { + reporter.report() + } +} + diff --git a/common/src/main/scala/com/bytedance/css/common/metrics/sink/GraphiteSink.scala b/common/src/main/scala/com/bytedance/css/common/metrics/sink/GraphiteSink.scala new file mode 100644 index 0000000..7ccb3d2 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/metrics/sink/GraphiteSink.scala @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.metrics.sink + +import java.util.{Locale, Properties} +import java.util.concurrent.TimeUnit + +import com.bytedance.css.common.metrics.MetricsSystem +import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry} +import com.codahale.metrics.graphite.{Graphite, GraphiteReporter, GraphiteUDP} + +class GraphiteSink(val property: Properties, val registry: MetricRegistry) extends Sink { + + val GRAPHITE_DEFAULT_PERIOD = 10 + val GRAPHITE_DEFAULT_UNIT = "SECONDS" + val GRAPHITE_DEFAULT_PREFIX = "" + + val GRAPHITE_KEY_HOST = "host" + val GRAPHITE_KEY_PORT = "port" + val GRAPHITE_KEY_PERIOD = "period" + val GRAPHITE_KEY_UNIT = "unit" + val GRAPHITE_KEY_PREFIX = "prefix" + val GRAPHITE_KEY_PROTOCOL = "protocol" + val GRAPHITE_KEY_REGEX = "regex" + + def propertyToOption(prop: String): Option[String] = Option(property.getProperty(prop)) + + if (!propertyToOption(GRAPHITE_KEY_HOST).isDefined) { + throw new Exception("Graphite sink requires 'host' property.") + } + + if (!propertyToOption(GRAPHITE_KEY_PORT).isDefined) { + throw new Exception("Graphite sink requires 'port' property.") + } + + val host = propertyToOption(GRAPHITE_KEY_HOST).get + val port = propertyToOption(GRAPHITE_KEY_PORT).get.toInt + + val pollPeriod = propertyToOption(GRAPHITE_KEY_PERIOD) match { + case Some(s) => s.toInt + case None => GRAPHITE_DEFAULT_PERIOD + } + + val pollUnit: TimeUnit = propertyToOption(GRAPHITE_KEY_UNIT) match { + case Some(s) => TimeUnit.valueOf(s.toUpperCase(Locale.ROOT)) + case None => TimeUnit.valueOf(GRAPHITE_DEFAULT_UNIT) + } + + val prefix = propertyToOption(GRAPHITE_KEY_PREFIX).getOrElse(GRAPHITE_DEFAULT_PREFIX) + + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) + + val graphite = propertyToOption(GRAPHITE_KEY_PROTOCOL).map(_.toLowerCase(Locale.ROOT)) match { + case Some("udp") => new GraphiteUDP(host, port) + case Some("tcp") | None => new Graphite(host, port) + case Some(p) => throw new Exception(s"Invalid Graphite protocol: $p") + } + + val filter = propertyToOption(GRAPHITE_KEY_REGEX) match { + case Some(pattern) => new MetricFilter() { + override def matches(name: String, metric: Metric): Boolean = { + pattern.r.findFirstMatchIn(name).isDefined + } + } + case None => MetricFilter.ALL + } + + val reporter: GraphiteReporter = GraphiteReporter.forRegistry(registry) + .convertDurationsTo(TimeUnit.MILLISECONDS) + .convertRatesTo(TimeUnit.SECONDS) + .prefixedWith(prefix) + .filter(filter) + .build(graphite) + + override def start(): Unit = { + reporter.start(pollPeriod, pollUnit) + } + + override def stop(): Unit = { + reporter.stop() + } + + override def report(): Unit = { + reporter.report() + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/metrics/sink/JmxSink.scala b/common/src/main/scala/com/bytedance/css/common/metrics/sink/JmxSink.scala new file mode 100644 index 0000000..87d290c --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/metrics/sink/JmxSink.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.metrics.sink + +import java.util.Properties + +import com.codahale.metrics.MetricRegistry +import com.codahale.metrics.jmx.JmxReporter + +class JmxSink(val property: Properties, val registry: MetricRegistry) extends Sink { + + val reporter: JmxReporter = JmxReporter.forRegistry(registry).build() + + override def start(): Unit = { + reporter.start() + } + + override def stop(): Unit = { + reporter.stop() + } + + override def report(): Unit = { } + +} diff --git a/common/src/main/scala/com/bytedance/css/common/metrics/sink/Sink.scala b/common/src/main/scala/com/bytedance/css/common/metrics/sink/Sink.scala new file mode 100644 index 0000000..c276860 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/metrics/sink/Sink.scala @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.metrics.sink + +trait Sink { + def start(): Unit + def stop(): Unit + def report(): Unit +} diff --git a/common/src/main/scala/com/bytedance/css/common/metrics/sink/Slf4jSink.scala b/common/src/main/scala/com/bytedance/css/common/metrics/sink/Slf4jSink.scala new file mode 100644 index 0000000..4437659 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/metrics/sink/Slf4jSink.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.metrics.sink + +import java.util.{Locale, Properties} +import java.util.concurrent.TimeUnit + +import com.bytedance.css.common.metrics.MetricsSystem +import com.codahale.metrics.{MetricRegistry, Slf4jReporter} + +class Slf4jSink(val property: Properties, val registry: MetricRegistry) extends Sink { + val SLF4J_DEFAULT_PERIOD = 10 + val SLF4J_DEFAULT_UNIT = "SECONDS" + + val SLF4J_KEY_PERIOD = "period" + val SLF4J_KEY_UNIT = "unit" + + val pollPeriod = Option(property.getProperty(SLF4J_KEY_PERIOD)) match { + case Some(s) => s.toInt + case None => SLF4J_DEFAULT_PERIOD + } + + val pollUnit: TimeUnit = Option(property.getProperty(SLF4J_KEY_UNIT)) match { + case Some(s) => TimeUnit.valueOf(s.toUpperCase(Locale.ROOT)) + case None => TimeUnit.valueOf(SLF4J_DEFAULT_UNIT) + } + + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) + + val reporter: Slf4jReporter = Slf4jReporter.forRegistry(registry) + .convertDurationsTo(TimeUnit.MILLISECONDS) + .convertRatesTo(TimeUnit.SECONDS) + .build() + + override def start(): Unit = { + reporter.start(pollPeriod, pollUnit) + } + + override def stop(): Unit = { + reporter.stop() + } + + override def report(): Unit = { + reporter.report() + } +} + diff --git a/common/src/main/scala/com/bytedance/css/common/metrics/sink/StatsdReporter.scala b/common/src/main/scala/com/bytedance/css/common/metrics/sink/StatsdReporter.scala new file mode 100644 index 0000000..bc37f23 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/metrics/sink/StatsdReporter.scala @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.metrics.sink + +import java.io.IOException +import java.net.{DatagramPacket, DatagramSocket, InetSocketAddress} +import java.nio.charset.StandardCharsets.UTF_8 +import java.util.SortedMap +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ +import scala.util.{Failure, Success, Try} + +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.util.Utils +import com.codahale.metrics._ + +/** + * @see + * StatsD metric types + */ +object StatsdMetricType { + val COUNTER = "c" + val GAUGE = "g" + val TIMER = "ms" + val Set = "s" +} + +class StatsdReporter( + registry: MetricRegistry, + host: String = "127.0.0.1", + port: Int = 8125, + prefix: String = "", + filter: MetricFilter = MetricFilter.ALL, + rateUnit: TimeUnit = TimeUnit.SECONDS, + durationUnit: TimeUnit = TimeUnit.MILLISECONDS) + extends ScheduledReporter(registry, "statsd-reporter", filter, rateUnit, durationUnit) + with Logging { + + import StatsdMetricType._ + + private val address = new InetSocketAddress(host, port) + private val whitespace = "[\\s]+".r + + override def report( + gauges: SortedMap[String, Gauge[_]], + counters: SortedMap[String, Counter], + histograms: SortedMap[String, Histogram], + meters: SortedMap[String, Meter], + timers: SortedMap[String, Timer]): Unit = + Try(new DatagramSocket) match { + case Failure(ioe: IOException) => logWarning(s"StatsD datagram socket construction failed" + + s" host ${host} port ${port} localhost ${Utils.localHostName()} port ${0}", ioe) + case Failure(e) => logWarning("StatsD datagram socket construction failed", e) + case Success(s) => + implicit val socket = s + val localAddress = Try(socket.getLocalAddress).map(_.getHostAddress).getOrElse(null) + val localPort = socket.getLocalPort + Try { + gauges.entrySet.asScala.foreach(e => reportGauge(e.getKey, e.getValue)) + counters.entrySet.asScala.foreach(e => reportCounter(e.getKey, e.getValue)) + histograms.entrySet.asScala.foreach(e => reportHistogram(e.getKey, e.getValue)) + meters.entrySet.asScala.foreach(e => reportMetered(e.getKey, e.getValue)) + timers.entrySet.asScala.foreach(e => reportTimer(e.getKey, e.getValue)) + } recover { + case ioe: IOException => + logDebug(s"Unable to send packets to StatsD host ${address.getHostString} port ${address.getPort}" + + s" localhost ${localAddress} port ${localPort}", ioe) + case e: Throwable => logDebug(s"Unable to send packets to StatsD at '$host:$port'", e) + } + Try(socket.close()) recover { + case ioe: IOException => + logDebug("Error when close socket to StatsD host ${address.getHostString} port ${address.getPort}" + + s" localhost ${localAddress} port ${localPort}", ioe) + case e: Throwable => logDebug("Error when close socket to StatsD", e) + } + } + + private def reportGauge(name: String, gauge: Gauge[_])(implicit socket: DatagramSocket): Unit = + formatAny(gauge.getValue).foreach(v => send(fullName(name), v, GAUGE)) + + private def reportCounter(name: String, counter: Counter)(implicit socket: DatagramSocket): Unit = + send(fullName(name), format(counter.getCount), COUNTER) + + private def reportHistogram(name: String, histogram: Histogram) + (implicit socket: DatagramSocket): Unit = { + val snapshot = histogram.getSnapshot + send(fullName(name, "count"), format(histogram.getCount), GAUGE) + send(fullName(name, "max"), format(snapshot.getMax), TIMER) + send(fullName(name, "mean"), format(snapshot.getMean), TIMER) + send(fullName(name, "min"), format(snapshot.getMin), TIMER) + send(fullName(name, "stddev"), format(snapshot.getStdDev), TIMER) + send(fullName(name, "p50"), format(snapshot.getMedian), TIMER) + send(fullName(name, "p75"), format(snapshot.get75thPercentile), TIMER) + send(fullName(name, "p95"), format(snapshot.get95thPercentile), TIMER) + send(fullName(name, "p98"), format(snapshot.get98thPercentile), TIMER) + send(fullName(name, "p99"), format(snapshot.get99thPercentile), TIMER) + send(fullName(name, "p999"), format(snapshot.get999thPercentile), TIMER) + } + + private def reportMetered(name: String, meter: Metered)(implicit socket: DatagramSocket): Unit = { + send(fullName(name, "count"), format(meter.getCount), GAUGE) + send(fullName(name, "m1_rate"), format(convertRate(meter.getOneMinuteRate)), TIMER) + send(fullName(name, "m5_rate"), format(convertRate(meter.getFiveMinuteRate)), TIMER) + send(fullName(name, "m15_rate"), format(convertRate(meter.getFifteenMinuteRate)), TIMER) + send(fullName(name, "mean_rate"), format(convertRate(meter.getMeanRate)), TIMER) + } + + private def reportTimer(name: String, timer: Timer)(implicit socket: DatagramSocket): Unit = { + val snapshot = timer.getSnapshot + send(fullName(name, "max"), format(convertDuration(snapshot.getMax)), TIMER) + send(fullName(name, "mean"), format(convertDuration(snapshot.getMean)), TIMER) + send(fullName(name, "min"), format(convertDuration(snapshot.getMin)), TIMER) + send(fullName(name, "stddev"), format(convertDuration(snapshot.getStdDev)), TIMER) + send(fullName(name, "p50"), format(convertDuration(snapshot.getMedian)), TIMER) + send(fullName(name, "p75"), format(convertDuration(snapshot.get75thPercentile)), TIMER) + send(fullName(name, "p95"), format(convertDuration(snapshot.get95thPercentile)), TIMER) + send(fullName(name, "p98"), format(convertDuration(snapshot.get98thPercentile)), TIMER) + send(fullName(name, "p99"), format(convertDuration(snapshot.get99thPercentile)), TIMER) + send(fullName(name, "p999"), format(convertDuration(snapshot.get999thPercentile)), TIMER) + + reportMetered(name, timer) + } + + private def send(name: String, value: String, metricType: String) + (implicit socket: DatagramSocket): Unit = { + val bytes = sanitize(s"$name:$value|$metricType").getBytes(UTF_8) + val packet = new DatagramPacket(bytes, bytes.length, address) + socket.send(packet) + } + + private def fullName(names: String*): String = MetricRegistry.name(prefix, names : _*) + + private def sanitize(s: String): String = whitespace.replaceAllIn(s, "-") + + private def format(v: Any): String = formatAny(v).getOrElse("") + + private def formatAny(v: Any): Option[String] = + v match { + case f: Float => Some("%2.2f".format(f)) + case d: Double => Some("%2.2f".format(d)) + case b: BigDecimal => Some("%2.2f".format(b)) + case n: Number => Some(v.toString) + case _ => None + } +} + diff --git a/common/src/main/scala/com/bytedance/css/common/metrics/sink/StatsdSink.scala b/common/src/main/scala/com/bytedance/css/common/metrics/sink/StatsdSink.scala new file mode 100644 index 0000000..0e5c4b0 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/metrics/sink/StatsdSink.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.metrics.sink + +import java.util.{Locale, Properties} +import java.util.concurrent.TimeUnit + +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.metrics.MetricsSystem +import com.codahale.metrics.MetricRegistry + +object StatsdSink { + val STATSD_KEY_HOST = "host" + val STATSD_KEY_PORT = "port" + val STATSD_KEY_PERIOD = "period" + val STATSD_KEY_UNIT = "unit" + val STATSD_KEY_PREFIX = "prefix" + + val STATSD_DEFAULT_HOST = "127.0.0.1" + val STATSD_DEFAULT_PORT = "8125" + val STATSD_DEFAULT_PERIOD = "10" + val STATSD_DEFAULT_UNIT = "SECONDS" + val STATSD_DEFAULT_PREFIX = "" +} + +class StatsdSink( + val property: Properties, val registry: MetricRegistry) extends Sink with Logging { + import StatsdSink._ + + val host = property.getProperty(STATSD_KEY_HOST, STATSD_DEFAULT_HOST) + val port = property.getProperty(STATSD_KEY_PORT, STATSD_DEFAULT_PORT).toInt + + val pollPeriod = property.getProperty(STATSD_KEY_PERIOD, STATSD_DEFAULT_PERIOD).toInt + val pollUnit = + TimeUnit.valueOf( + property.getProperty(STATSD_KEY_UNIT, STATSD_DEFAULT_UNIT).toUpperCase(Locale.ROOT)) + + val prefix = property.getProperty(STATSD_KEY_PREFIX, STATSD_DEFAULT_PREFIX) + + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) + + val reporter = new StatsdReporter(registry, host, port, prefix) + + override def start(): Unit = { + reporter.start(pollPeriod, pollUnit) + logInfo(s"StatsdSink started with prefix: '$prefix'") + } + + override def stop(): Unit = { + reporter.stop() + logInfo("StatsdSink stopped.") + } + + override def report(): Unit = reporter.report() +} + diff --git a/common/src/main/scala/com/bytedance/css/common/metrics/source/Source.scala b/common/src/main/scala/com/bytedance/css/common/metrics/source/Source.scala new file mode 100644 index 0000000..d7c9eb0 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/metrics/source/Source.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.metrics.source + +import com.codahale.metrics.MetricRegistry + +trait Source { + def sourceName: String + def metricRegistry: MetricRegistry +} diff --git a/common/src/main/scala/com/bytedance/css/common/protocol/CssRpcMessage.scala b/common/src/main/scala/com/bytedance/css/common/protocol/CssRpcMessage.scala new file mode 100644 index 0000000..570a289 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/protocol/CssRpcMessage.scala @@ -0,0 +1,119 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.bytedance.css.common.protocol + +import java.util + +import com.bytedance.css.common.rpc.RpcEndpointRef + +sealed trait Message extends Serializable + +object CssRpcMessage { + + /** + * Rpc Message For Cluster Service With Master & Worker. + */ + // sent from worker to master + // register push port, fetch port + // and additional rpcRef for master to call back + case class RegisterWorker( + name: String, + host: String, + rpcPort: Int, + pushPort: Int, + fetchPort: Int, + workerRef: RpcEndpointRef) extends Message + + case class RegisterWorkerResponse(statusCode: CssStatusCode) extends Message + + case class HeartbeatFromWorker( + name: String, + rttAvgStat: Long, + shuffleKeys: util.HashSet[String]) extends Message + + case class HeartbeatResponse( + expiredShuffleKeys: util.HashSet[String], + expiredAppIds: util.HashSet[String]) extends Message + + /** + * Rpc Message For Master & App. + */ + case class HeartbeatFromApp(appId: String) extends Message + + case class ApplicationLost(applicationId: String) extends Message + + /** + * Rpc Message For Shuffle Flow Control Message. + */ + // For test only, internal + case class BreakPartition(shuffleKey: String, reducerId: Int, epochId: Int) extends Message + case class BreakPartitionResponse(statusCode: CssStatusCode) extends Message + + case class RegisterPartitionGroup( + applicationId: String, + shuffleId: Int, + numMappers: Int, + numPartitions: Int, + maxPartitionsPerGroup: Int) extends Message + + case class RegisterPartitionGroupResponse( + statusCode: CssStatusCode, + partitionGroups: util.List[PartitionGroup]) extends Message + + case class ReallocatePartitionGroup( + applicationId: String, + shuffleId: Int, + mapId: Int, + attemptId: Int, + oldPartitionGroup: PartitionGroup) extends Message + + case class ReallocatePartitionGroupResponse(statusCode: CssStatusCode, partitionGroup: PartitionGroup) extends Message + + case class MapperEnd( + applicationId: String, + shuffleId: Int, + mapId: Int, + attemptId: Int, + numMappers: Int, + epochList: util.List[PartitionInfo], + batchBlacklist: util.List[FailedPartitionInfoBatch]) extends Message + + case class MapperEndResponse(statusCode: CssStatusCode) extends Message + + case class StageEnd(applicationId: String, shuffleId: Int) extends Message + case class StageEndResponse(statusCode: CssStatusCode) extends Message + + case class CommitFiles(shuffleKey: String) extends Message + case class CommitFilesResponse(committed: util.List[CommittedPartitionInfo]) extends Message + + case class CloseFile(shuffleKey: String, partition: PartitionInfo) extends Message + case object CloseFileResponse extends Message + + case class GetReducerFileGroups(applicationId: String, shuffleId: Int) extends Message + case class GetReducerFileGroupsResponse( + status: CssStatusCode, + fileGroup: Array[Array[CommittedPartitionInfo]], + attempts: Array[Int], + batchBlacklist: util.HashSet[FailedPartitionInfoBatch]) extends Message + + case class UnregisterShuffle(applicationId: String, shuffleId: Int) extends Message + case class UnregisterShuffleResponse(statusCode: CssStatusCode) extends Message + +} diff --git a/common/src/main/scala/com/bytedance/css/common/rpc/RpcAddress.scala b/common/src/main/scala/com/bytedance/css/common/rpc/RpcAddress.scala new file mode 100644 index 0000000..667af48 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/rpc/RpcAddress.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc + +import com.bytedance.css.common.util.Utils + + +/** + * Address for an RPC environment, with hostname and port. + */ +case class RpcAddress(host: String, port: Int) { + + def hostPort: String = host + ":" + port + + /** Returns a string in the form of "css://host:port". */ + def toCssURL: String = "css://" + hostPort + + override def toString: String = hostPort +} + + +object RpcAddress { + + /** Return the [[RpcAddress]] represented by `uri`. */ + def fromURIString(uri: String): RpcAddress = { + val uriObj = new java.net.URI(uri) + RpcAddress(uriObj.getHost, uriObj.getPort) + } + + /** Returns the [[RpcAddress]] encoded in the form of "css://host:port" */ + def fromCssURL(cssUrl: String): RpcAddress = { + val (host, port) = Utils.extractHostPortFromCssUrl(cssUrl) + RpcAddress(host, port) + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/rpc/RpcCallContext.scala b/common/src/main/scala/com/bytedance/css/common/rpc/RpcCallContext.scala new file mode 100644 index 0000000..da26984 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/rpc/RpcCallContext.scala @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc + +/** + * A callback that [[RpcEndpoint]] can use to send back a message or failure. It's thread-safe + * and can be called in any thread. + */ +trait RpcCallContext { + + /** + * Reply a message to the sender. If the sender is [[RpcEndpoint]], its `RpcEndpoint.receive` + * will be called. + */ + def reply(response: Any): Unit + + /** + * Report a failure to the sender. + */ + def sendFailure(e: Throwable): Unit + + /** + * The sender of this message. + */ + def senderAddress: RpcAddress +} diff --git a/common/src/main/scala/com/bytedance/css/common/rpc/RpcEndpoint.scala b/common/src/main/scala/com/bytedance/css/common/rpc/RpcEndpoint.scala new file mode 100644 index 0000000..6680807 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/rpc/RpcEndpoint.scala @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc + +import com.bytedance.css.common.exception.CssException + +/** + * A factory class to create the [[RpcEnv]]. It must have an empty constructor so that it can be + * created using Reflection. + */ +trait RpcEnvFactory { + + def create(config: RpcEnvConfig): RpcEnv +} + +/** + * An end point for the RPC that defines what functions to trigger given a message. + * + * It is guaranteed that `onStart`, `receive` and `onStop` will be called in sequence. + * + * The life-cycle of an endpoint is: + * + * {@code constructor -> onStart -> receive* -> onStop} + * + * Note: `receive` can be called concurrently. If you want `receive` to be thread-safe, please use + * [[ThreadSafeRpcEndpoint]] + * + * If any error is thrown from one of [[RpcEndpoint]] methods except `onError`, `onError` will be + * invoked with the cause. If `onError` throws an error, [[RpcEnv]] will ignore it. + */ +trait RpcEndpoint { + + /** + * The [[RpcEnv]] that this [[RpcEndpoint]] is registered to. + */ + val rpcEnv: RpcEnv + + /** + * The [[RpcEndpointRef]] of this [[RpcEndpoint]]. `self` will become valid when `onStart` is + * called. And `self` will become `null` when `onStop` is called. + * + * Note: Because before `onStart`, [[RpcEndpoint]] has not yet been registered and there is not + * valid [[RpcEndpointRef]] for it. So don't call `self` before `onStart` is called. + */ + final def self: RpcEndpointRef = { + require(rpcEnv != null, "rpcEnv has not been initialized") + rpcEnv.endpointRef(this) + } + + /** + * Process messages from `RpcEndpointRef.send` or `RpcCallContext.reply`. If receiving a + * unmatched message, `CssException` will be thrown and sent to `onError`. + */ + def receive: PartialFunction[Any, Unit] = { + case _ => throw new CssException(self + " does not implement 'receive'") + } + + /** + * Process messages from `RpcEndpointRef.ask`. If receiving a unmatched message, + * `CssException` will be thrown and sent to `onError`. + */ + def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case _ => context.sendFailure(new CssException(self + " won't reply anything")) + } + + /** + * Invoked when any exception is thrown during handling messages. + */ + def onError(cause: Throwable): Unit = { + // By default, throw e and let RpcEnv handle it + throw cause + } + + /** + * Invoked when `remoteAddress` is connected to the current node. + */ + def onConnected(remoteAddress: RpcAddress): Unit = { + // By default, do nothing. + } + + /** + * Invoked when `remoteAddress` is lost. + */ + def onDisconnected(remoteAddress: RpcAddress): Unit = { + // By default, do nothing. + } + + /** + * Invoked when some network error happens in the connection between the current node and + * `remoteAddress`. + */ + def onNetworkError(cause: Throwable, remoteAddress: RpcAddress): Unit = { + // By default, do nothing. + } + + /** + * Invoked before [[RpcEndpoint]] starts to handle any message. + */ + def onStart(): Unit = { + // By default, do nothing. + } + + /** + * Invoked when [[RpcEndpoint]] is stopping. `self` will be `null` in this method and you cannot + * use it to send or ask messages. + */ + def onStop(): Unit = { + // By default, do nothing. + } + + /** + * A convenient method to stop [[RpcEndpoint]]. + */ + final def stop(): Unit = { + val _self = self + if (_self != null) { + rpcEnv.stop(_self) + } + } +} + +/** + * A trait that requires RpcEnv thread-safely sending messages to it. + * + * Thread-safety means processing of one message happens before processing of the next message by + * the same [[ThreadSafeRpcEndpoint]]. In the other words, changes to internal fields of a + * [[ThreadSafeRpcEndpoint]] are visible when processing the next message, and fields in the + * [[ThreadSafeRpcEndpoint]] need not be volatile or equivalent. + * + * However, there is no guarantee that the same thread will be executing the same + * [[ThreadSafeRpcEndpoint]] for different messages. + */ +trait ThreadSafeRpcEndpoint extends RpcEndpoint diff --git a/common/src/main/scala/com/bytedance/css/common/rpc/RpcEndpointAddress.scala b/common/src/main/scala/com/bytedance/css/common/rpc/RpcEndpointAddress.scala new file mode 100644 index 0000000..bdd9ded --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/rpc/RpcEndpointAddress.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc + +import com.bytedance.css.common.exception.CssException + +/** + * An address identifier for an RPC endpoint. + * + * The `rpcAddress` may be null, in which case the endpoint is registered via a client-only + * connection and can only be reached via the client that sent the endpoint reference. + * + * @param rpcAddress The socket address of the endpoint. It's `null` when this address pointing to + * an endpoint in a client `NettyRpcEnv`. + * @param name Name of the endpoint. + */ +case class RpcEndpointAddress(rpcAddress: RpcAddress, name: String) { + + require(name != null, "RpcEndpoint name must be provided.") + + def this(host: String, port: Int, name: String) = { + this(RpcAddress(host, port), name) + } + + override val toString = if (rpcAddress != null) { + s"css://$name@${rpcAddress.host}:${rpcAddress.port}" + } else { + s"css-client://$name" + } +} + +object RpcEndpointAddress { + + def apply(host: String, port: Int, name: String): RpcEndpointAddress = { + new RpcEndpointAddress(host, port, name) + } + + def apply(cssUrl: String): RpcEndpointAddress = { + try { + val uri = new java.net.URI(cssUrl) + val host = uri.getHost + val port = uri.getPort + val name = uri.getUserInfo + if (uri.getScheme != "css" || + host == null || + port < 0 || + name == null || + (uri.getPath != null && !uri.getPath.isEmpty) || // uri.getPath returns "" instead of null + uri.getFragment != null || + uri.getQuery != null) { + throw new CssException("Invalid Css URL: " + cssUrl) + } + new RpcEndpointAddress(host, port, name) + } catch { + case e: java.net.URISyntaxException => + throw new CssException("Invalid Css URL: " + cssUrl, e) + } + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/rpc/RpcEndpointNotFoundException.scala b/common/src/main/scala/com/bytedance/css/common/rpc/RpcEndpointNotFoundException.scala new file mode 100644 index 0000000..733b220 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/rpc/RpcEndpointNotFoundException.scala @@ -0,0 +1,25 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc + +import com.bytedance.css.common.exception.CssException + +private[rpc] class RpcEndpointNotFoundException(uri: String) + extends CssException(s"Cannot find endpoint: $uri") diff --git a/common/src/main/scala/com/bytedance/css/common/rpc/RpcEndpointRef.scala b/common/src/main/scala/com/bytedance/css/common/rpc/RpcEndpointRef.scala new file mode 100644 index 0000000..f2ef960 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/rpc/RpcEndpointRef.scala @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc + +import scala.concurrent.Future +import scala.reflect.ClassTag + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.util.RpcUtils + +/** + * A reference for a remote [[RpcEndpoint]]. [[RpcEndpointRef]] is thread-safe. + */ +abstract class RpcEndpointRef(conf: CssConf) + extends Serializable with Logging { + + private[this] val maxRetries = RpcUtils.numRetries(conf) + private[this] val retryWaitMs = RpcUtils.retryWaitMs(conf) + private[this] val defaultAskTimeout = RpcUtils.askRpcTimeout(conf) + + /** + * return the address for the [[RpcEndpointRef]] + */ + def address: RpcAddress + + def name: String + + /** + * Sends a one-way asynchronous message. Fire-and-forget semantics. + */ + def send(message: Any): Unit + + /** + * Send a message to the corresponding [[RpcEndpoint.receiveAndReply)]] and return a [[Future]] to + * receive the reply within the specified timeout. + * + * This method only sends the message once and never retries. + */ + def ask[T: ClassTag](message: Any, timeout: RpcTimeout): Future[T] + + /** + * Send a message to the corresponding [[RpcEndpoint.receiveAndReply)]] and return a [[Future]] to + * receive the reply within a default timeout. + * + * This method only sends the message once and never retries. + */ + def ask[T: ClassTag](message: Any): Future[T] = ask(message, defaultAskTimeout) + + /** + * Send a message to the corresponding [[RpcEndpoint.receiveAndReply]] and get its result within a + * default timeout, throw an exception if this fails. + * + * Note: this is a blocking action which may cost a lot of time, so don't call it in a message + * loop of [[RpcEndpoint]]. + + * @param message the message to send + * @tparam T type of the reply message + * @return the reply message from the corresponding [[RpcEndpoint]] + */ + def askSync[T: ClassTag](message: Any): T = askSync(message, defaultAskTimeout) + + /** + * Send a message to the corresponding [[RpcEndpoint.receiveAndReply]] and get its result within a + * specified timeout, throw an exception if this fails. + * + * Note: this is a blocking action which may cost a lot of time, so don't call it in a message + * loop of [[RpcEndpoint]]. + * + * @param message the message to send + * @param timeout the timeout duration + * @tparam T type of the reply message + * @return the reply message from the corresponding [[RpcEndpoint]] + */ + def askSync[T: ClassTag](message: Any, timeout: RpcTimeout): T = { + val future = ask[T](message, timeout) + timeout.awaitResult(future) + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/rpc/RpcEnv.scala b/common/src/main/scala/com/bytedance/css/common/rpc/RpcEnv.scala new file mode 100644 index 0000000..6418c91 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/rpc/RpcEnv.scala @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc + +import java.io.File +import java.nio.channels.ReadableByteChannel + +import scala.concurrent.Future + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.rpc.netty.NettyRpcEnvFactory +import com.bytedance.css.common.util.RpcUtils + + +/** + * A RpcEnv implementation must have a [[RpcEnvFactory]] implementation with an empty constructor + * so that it can be created via Reflection. + */ +object RpcEnv { + + def create( + name: String, + host: String, + port: Int, + conf: CssConf, + clientMode: Boolean = false): RpcEnv = { + create(name, host, host, port, conf, 0, clientMode) + } + + def create( + name: String, + bindAddress: String, + advertiseAddress: String, + port: Int, + conf: CssConf, + numUsableCores: Int, + clientMode: Boolean): RpcEnv = { + val config = RpcEnvConfig(conf, name, bindAddress, advertiseAddress, port, numUsableCores, clientMode) + new NettyRpcEnvFactory().create(config) + } +} + + +/** + * An RPC environment. [[RpcEndpoint]]s need to register itself with a name to [[RpcEnv]] to + * receives messages. Then [[RpcEnv]] will process messages sent from [[RpcEndpointRef]] or remote + * nodes, and deliver them to corresponding [[RpcEndpoint]]s. For uncaught exceptions caught by + * [[RpcEnv]], [[RpcEnv]] will use [[RpcCallContext.sendFailure]] to send exceptions back to the + * sender, or logging them if no such sender or `NotSerializableException`. + * + * [[RpcEnv]] also provides some methods to retrieve [[RpcEndpointRef]]s given name or uri. + */ +abstract class RpcEnv(conf: CssConf) { + + val defaultLookupTimeout = RpcUtils.lookupRpcTimeout(conf) + + /** + * Return RpcEndpointRef of the registered [[RpcEndpoint]]. Will be used to implement + * [[RpcEndpoint.self]]. Return `null` if the corresponding [[RpcEndpointRef]] does not exist. + */ + def endpointRef(endpoint: RpcEndpoint): RpcEndpointRef + + /** + * Return the address that [[RpcEnv]] is listening to. + */ + def address: RpcAddress + + /** + * Register a [[RpcEndpoint]] with a name and return its [[RpcEndpointRef]]. [[RpcEnv]] does not + * guarantee thread-safety. + */ + def setupEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef + + /** + * Retrieve the [[RpcEndpointRef]] represented by `uri` asynchronously. + */ + def asyncSetupEndpointRefByURI(uri: String): Future[RpcEndpointRef] + + /** + * Retrieve the [[RpcEndpointRef]] represented by `uri`. This is a blocking action. + */ + def setupEndpointRefByURI(uri: String): RpcEndpointRef = { + defaultLookupTimeout.awaitResult(asyncSetupEndpointRefByURI(uri)) + } + + /** + * Retrieve the [[RpcEndpointRef]] represented by `address` and `endpointName`. + * This is a blocking action. + */ + def setupEndpointRef(address: RpcAddress, endpointName: String): RpcEndpointRef = { + setupEndpointRefByURI(RpcEndpointAddress(address, endpointName).toString) + } + + /** + * Stop [[RpcEndpoint]] specified by `endpoint`. + */ + def stop(endpoint: RpcEndpointRef): Unit + + /** + * Shutdown this [[RpcEnv]] asynchronously. If need to make sure [[RpcEnv]] exits successfully, + * call [[awaitTermination()]] straight after [[shutdown()]]. + */ + def shutdown(): Unit + + /** + * Wait until [[RpcEnv]] exits. + * + * TODO do we need a timeout parameter? + */ + def awaitTermination(): Unit + + /** + * [[RpcEndpointRef]] cannot be deserialized without [[RpcEnv]]. So when deserializing any object + * that contains [[RpcEndpointRef]]s, the deserialization codes should be wrapped by this method. + */ + def deserialize[T](deserializationAction: () => T): T + + /** + * Return the instance of the file server used to serve files. This may be `null` if the + * RpcEnv is not operating in server mode. + */ + def fileServer: RpcEnvFileServer + + /** + * Open a channel to download a file from the given URI. If the URIs returned by the + * RpcEnvFileServer use the "spark" scheme, this method will be called by the Utils class to + * retrieve the files. + * + * @param uri URI with location of the file. + */ + def openChannel(uri: String): ReadableByteChannel +} + +/** + * A server used by the RpcEnv to server files to other processes owned by the application. + * + * The file server can return URIs handled by common libraries (such as "http" or "hdfs"), or + * it can return "spark" URIs which will be handled by `RpcEnv#fetchFile`. + */ +trait RpcEnvFileServer { + + /** + * Adds a file to be served by this RpcEnv. This is used to serve files from the driver + * to executors when they're stored on the driver's local file system. + * + * @param file Local file to serve. + * @return A URI for the location of the file. + */ + def addFile(file: File): String + + /** + * Adds a jar to be served by this RpcEnv. Similar to `addFile` but for jars added using + * `SparkContext.addJar`. + * + * @param file Local file to serve. + * @return A URI for the location of the file. + */ + def addJar(file: File): String + + /** + * Adds a local directory to be served via this file server. + * + * @param baseUri Leading URI path (files can be retrieved by appending their relative + * path to this base URI). This cannot be "files" nor "jars". + * @param path Path to the local directory. + * @return URI for the root of the directory in the file server. + */ + def addDirectory(baseUri: String, path: File): String + + /** Validates and normalizes the base URI for directories. */ + protected def validateDirectoryUri(baseUri: String): String = { + val fixedBaseUri = "/" + baseUri.stripPrefix("/").stripSuffix("/") + require(fixedBaseUri != "/files" && fixedBaseUri != "/jars", + "Directory URI cannot be /files nor /jars.") + fixedBaseUri + } + +} + +case class RpcEnvConfig( + conf: CssConf, + name: String, + bindAddress: String, + advertiseAddress: String, + port: Int, + numUsableCores: Int, + clientMode: Boolean) diff --git a/common/src/main/scala/com/bytedance/css/common/rpc/RpcEnvStoppedException.scala b/common/src/main/scala/com/bytedance/css/common/rpc/RpcEnvStoppedException.scala new file mode 100644 index 0000000..fb869c1 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/rpc/RpcEnvStoppedException.scala @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc + +private[rpc] class RpcEnvStoppedException() + extends IllegalStateException("RpcEnv already stopped.") diff --git a/common/src/main/scala/com/bytedance/css/common/rpc/RpcTimeout.scala b/common/src/main/scala/com/bytedance/css/common/rpc/RpcTimeout.scala new file mode 100644 index 0000000..23befd3 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/rpc/RpcTimeout.scala @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc + +import java.util.concurrent.TimeoutException + +import scala.concurrent.Future +import scala.concurrent.duration._ + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.util.{ThreadUtils, Utils} + +/** + * An exception thrown if RpcTimeout modifies a `TimeoutException`. + */ +private[rpc] class RpcTimeoutException(message: String, cause: TimeoutException) + extends TimeoutException(message) { initCause(cause) } + + +/** + * Associates a timeout with a description so that a when a TimeoutException occurs, additional + * context about the timeout can be amended to the exception message. + * + * @param duration timeout duration in seconds + * @param timeoutProp the configuration property that controls this timeout + */ +class RpcTimeout(val duration: FiniteDuration, val timeoutProp: String) + extends Serializable { + + /** Amends the standard message of TimeoutException to include the description */ + private def createRpcTimeoutException(te: TimeoutException): RpcTimeoutException = { + new RpcTimeoutException(te.getMessage + ". This timeout is controlled by " + timeoutProp, te) + } + + /** + * PartialFunction to match a TimeoutException and add the timeout description to the message + * + * @note This can be used in the recover callback of a Future to add to a TimeoutException + * Example: + * val timeout = new RpcTimeout(5 millis, "short timeout") + * Future(throw new TimeoutException).recover(timeout.addMessageIfTimeout) + */ + def addMessageIfTimeout[T]: PartialFunction[Throwable, T] = { + // The exception has already been converted to a RpcTimeoutException so just raise it + case rte: RpcTimeoutException => throw rte + // Any other TimeoutException get converted to a RpcTimeoutException with modified message + case te: TimeoutException => throw createRpcTimeoutException(te) + } + + /** + * Wait for the completed result and return it. If the result is not available within this + * timeout, throw a [[RpcTimeoutException]] to indicate which configuration controls the timeout. + * + * @param future the `Future` to be awaited + * @throws RpcTimeoutException if after waiting for the specified time `future` + * is still not ready + */ + def awaitResult[T](future: Future[T]): T = { + try { + ThreadUtils.awaitResult(future, duration) + } catch addMessageIfTimeout + } +} + + +object RpcTimeout { + + /** + * Lookup the timeout property in the configuration and create + * a RpcTimeout with the property key in the description. + * + * @param conf configuration properties containing the timeout + * @param timeoutProp property key for the timeout in seconds + * @throws NoSuchElementException if property is not set + */ + def apply(conf: CssConf, timeoutProp: String): RpcTimeout = { + val timeout = { conf.getTimeAsSeconds(timeoutProp).seconds } + new RpcTimeout(timeout, timeoutProp) + } + + /** + * Lookup the timeout property in the configuration and create + * a RpcTimeout with the property key in the description. + * Uses the given default value if property is not set + * + * @param conf configuration properties containing the timeout + * @param timeoutProp property key for the timeout in seconds + * @param defaultValue default timeout value in seconds if property not found + */ + def apply(conf: CssConf, timeoutProp: String, defaultValue: String): RpcTimeout = { + val timeout = { conf.getTimeAsSeconds(timeoutProp, defaultValue).seconds } + new RpcTimeout(timeout, timeoutProp) + } + + /** + * Lookup prioritized list of timeout properties in the configuration + * and create a RpcTimeout with the first set property key in the + * description. + * Uses the given default value if property is not set + * + * @param conf configuration properties containing the timeout + * @param timeoutPropList prioritized list of property keys for the timeout in seconds + * @param defaultValue default timeout value in seconds if no properties found + */ + def apply(conf: CssConf, timeoutPropList: Seq[String], defaultValue: String): RpcTimeout = { + require(timeoutPropList.nonEmpty) + + // Find the first set property or use the default value with the first property + val itr = timeoutPropList.iterator + var foundProp: Option[(String, String)] = None + while (itr.hasNext && foundProp.isEmpty) { + val propKey = itr.next() + conf.getOption(propKey).foreach { prop => foundProp = Some((propKey, prop)) } + } + val finalProp = foundProp.getOrElse((timeoutPropList.head, defaultValue)) + val timeout = { Utils.timeStringAsSeconds(finalProp._2).seconds } + new RpcTimeout(timeout, finalProp._1) + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/rpc/netty/Dispatcher.scala b/common/src/main/scala/com/bytedance/css/common/rpc/netty/Dispatcher.scala new file mode 100644 index 0000000..332d7fe --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/rpc/netty/Dispatcher.scala @@ -0,0 +1,244 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc.netty + +import java.util.concurrent._ +import javax.annotation.concurrent.GuardedBy + +import scala.collection.JavaConverters._ +import scala.concurrent.Promise +import scala.util.control.NonFatal + +import com.bytedance.css.common.exception.CssException +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.rpc.{RpcEndpoint, RpcEndpointAddress, RpcEndpointRef, RpcEnvStoppedException} +import com.bytedance.css.common.util.ThreadUtils +import com.bytedance.css.network.client.RpcResponseCallback + +/** + * A message dispatcher, responsible for routing RPC messages to the appropriate endpoint(s). + * + * @param numUsableCores Number of CPU cores allocated to the process, for sizing the thread pool. + * If 0, will consider the available CPUs on the host. + */ +class Dispatcher(nettyEnv: NettyRpcEnv, numUsableCores: Int) extends Logging { + + private class EndpointData( + val name: String, + val endpoint: RpcEndpoint, + val ref: NettyRpcEndpointRef) { + val inbox = new Inbox(ref, endpoint) + } + + private val endpoints: ConcurrentMap[String, EndpointData] = + new ConcurrentHashMap[String, EndpointData] + private val endpointRefs: ConcurrentMap[RpcEndpoint, RpcEndpointRef] = + new ConcurrentHashMap[RpcEndpoint, RpcEndpointRef] + + // Track the receivers whose inboxes may contain messages. + private val receivers = new LinkedBlockingQueue[EndpointData] + + /** + * True if the dispatcher has been stopped. Once stopped, all messages posted will be bounced + * immediately. + */ + @GuardedBy("this") + private var stopped = false + + def registerRpcEndpoint(name: String, endpoint: RpcEndpoint): NettyRpcEndpointRef = { + val addr = RpcEndpointAddress(nettyEnv.address, name) + val endpointRef = new NettyRpcEndpointRef(nettyEnv.conf, addr, nettyEnv) + synchronized { + if (stopped) { + throw new IllegalStateException("RpcEnv has been stopped") + } + if (endpoints.putIfAbsent(name, new EndpointData(name, endpoint, endpointRef)) != null) { + throw new IllegalArgumentException(s"There is already an RpcEndpoint called $name") + } + val data = endpoints.get(name) + endpointRefs.put(data.endpoint, data.ref) + receivers.offer(data) // for the OnStart message + } + endpointRef + } + + def getRpcEndpointRef(endpoint: RpcEndpoint): RpcEndpointRef = endpointRefs.get(endpoint) + + def removeRpcEndpointRef(endpoint: RpcEndpoint): Unit = endpointRefs.remove(endpoint) + + // Should be idempotent + private def unregisterRpcEndpoint(name: String): Unit = { + val data = endpoints.remove(name) + if (data != null) { + data.inbox.stop() + receivers.offer(data) // for the OnStop message + } + // Don't clean `endpointRefs` here because it's possible that some messages are being processed + // now and they can use `getRpcEndpointRef`. So `endpointRefs` will be cleaned in Inbox via + // `removeRpcEndpointRef`. + } + + def stop(rpcEndpointRef: RpcEndpointRef): Unit = { + synchronized { + if (stopped) { + // This endpoint will be stopped by Dispatcher.stop() method. + return + } + unregisterRpcEndpoint(rpcEndpointRef.name) + } + } + + /** + * Send a message to all registered [[RpcEndpoint]]s in this process. + * + * This can be used to make network events known to all end points (e.g. "a new node connected"). + */ + def postToAll(message: InboxMessage): Unit = { + val iter = endpoints.keySet().iterator() + while (iter.hasNext) { + val name = iter.next + postMessage(name, message, (e) => { e match { + case e: RpcEnvStoppedException => logDebug (s"Message $message dropped. ${e.getMessage}") + case e: Throwable => logWarning(s"Message $message dropped. ${e.getMessage}") + }} + )} + } + + /** Posts a message sent by a remote endpoint. */ + def postRemoteMessage(message: RequestMessage, callback: RpcResponseCallback): Unit = { + val rpcCallContext = + new RemoteNettyRpcCallContext(nettyEnv, callback, message.senderAddress) + val rpcMessage = RpcMessage(message.senderAddress, message.content, rpcCallContext) + postMessage(message.receiver.name, rpcMessage, (e) => callback.onFailure(e)) + } + + /** Posts a message sent by a local endpoint. */ + def postLocalMessage(message: RequestMessage, p: Promise[Any]): Unit = { + val rpcCallContext = + new LocalNettyRpcCallContext(message.senderAddress, p) + val rpcMessage = RpcMessage(message.senderAddress, message.content, rpcCallContext) + postMessage(message.receiver.name, rpcMessage, (e) => p.tryFailure(e)) + } + + /** Posts a one-way message. */ + def postOneWayMessage(message: RequestMessage): Unit = { + postMessage(message.receiver.name, OneWayMessage(message.senderAddress, message.content), + (e) => throw e) + } + + /** + * Posts a message to a specific endpoint. + * + * @param endpointName name of the endpoint. + * @param message the message to post + * @param callbackIfStopped callback function if the endpoint is stopped. + */ + def postMessage( + endpointName: String, + message: InboxMessage, + callbackIfStopped: (Exception) => Unit): Unit = { + val error = synchronized { + val data = endpoints.get(endpointName) + if (stopped) { + Some(new RpcEnvStoppedException()) + } else if (data == null) { + Some(new CssException(s"Could not find $endpointName.")) + } else { + data.inbox.post(message) + receivers.offer(data) + None + } + } + // We don't need to call `onStop` in the `synchronized` block + error.foreach(callbackIfStopped) + } + + def stop(): Unit = { + synchronized { + if (stopped) { + return + } + stopped = true + } + // Stop all endpoints. This will queue all endpoints for processing by the message loops. + endpoints.keySet().asScala.foreach(unregisterRpcEndpoint) + // Enqueue a message that tells the message loops to stop. + receivers.offer(PoisonPill) + threadpool.shutdown() + } + + def awaitTermination(): Unit = { + threadpool.awaitTermination(Long.MaxValue, TimeUnit.MILLISECONDS) + } + + /** + * Return if the endpoint exists + */ + def verify(name: String): Boolean = { + endpoints.containsKey(name) + } + + /** Thread pool used for dispatching messages. */ + private val threadpool: ThreadPoolExecutor = { + val availableCores = + if (numUsableCores > 0) numUsableCores else Runtime.getRuntime.availableProcessors() + val numThreads = nettyEnv.conf.getInt("css.rpc.netty.dispatcher.numThreads", + math.max(2, availableCores)) + val pool = ThreadUtils.newDaemonFixedThreadPool(numThreads, "dispatcher-event-loop") + for (i <- 0 until numThreads) { + pool.execute(new MessageLoop) + } + pool + } + + /** Message loop used for dispatching messages. */ + private class MessageLoop extends Runnable { + override def run(): Unit = { + try { + while (true) { + try { + val data = receivers.take() + if (data == PoisonPill) { + // Put PoisonPill back so that other MessageLoops can see it. + receivers.offer(PoisonPill) + return + } + data.inbox.process(Dispatcher.this) + } catch { + case NonFatal(e) => logError(e.getMessage, e) + } + } + } catch { + case _: InterruptedException => // exit + case t: Throwable => + try { + // Re-submit a MessageLoop so that Dispatcher will still work if + // UncaughtExceptionHandler decides to not kill JVM. + threadpool.execute(new MessageLoop) + } finally { + throw t + } + } + } + } + + /** A poison endpoint that indicates MessageLoop should exit its message loop. */ + private val PoisonPill = new EndpointData(null, null, null) +} diff --git a/common/src/main/scala/com/bytedance/css/common/rpc/netty/Inbox.scala b/common/src/main/scala/com/bytedance/css/common/rpc/netty/Inbox.scala new file mode 100644 index 0000000..41122c7 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/rpc/netty/Inbox.scala @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc.netty + +import javax.annotation.concurrent.GuardedBy + +import scala.util.control.NonFatal + +import com.bytedance.css.common.exception.CssException +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.rpc.{RpcAddress, RpcEndpoint, ThreadSafeRpcEndpoint} + +sealed trait InboxMessage + +case class OneWayMessage( + senderAddress: RpcAddress, + content: Any) extends InboxMessage + +case class RpcMessage( + senderAddress: RpcAddress, + content: Any, + context: NettyRpcCallContext) extends InboxMessage + +case object OnStart extends InboxMessage + +case object OnStop extends InboxMessage + +/** A message to tell all endpoints that a remote process has connected. */ +case class RemoteProcessConnected(remoteAddress: RpcAddress) extends InboxMessage + +/** A message to tell all endpoints that a remote process has disconnected. */ +case class RemoteProcessDisconnected(remoteAddress: RpcAddress) extends InboxMessage + +/** A message to tell all endpoints that a network error has happened. */ +case class RemoteProcessConnectionError(cause: Throwable, remoteAddress: RpcAddress) + extends InboxMessage + +/** + * An inbox that stores messages for an [[RpcEndpoint]] and posts messages to it thread-safely. + */ +class Inbox( + val endpointRef: NettyRpcEndpointRef, + val endpoint: RpcEndpoint) + extends Logging { + + inbox => // Give this an alias so we can use it more clearly in closures. + + @GuardedBy("this") + protected val messages = new java.util.LinkedList[InboxMessage]() + + /** True if the inbox (and its associated endpoint) is stopped. */ + @GuardedBy("this") + private var stopped = false + + /** Allow multiple threads to process messages at the same time. */ + @GuardedBy("this") + private var enableConcurrent = false + + /** The number of threads processing messages for this inbox. */ + @GuardedBy("this") + private var numActiveThreads = 0 + + // OnStart should be the first message to process + inbox.synchronized { + messages.add(OnStart) + } + + /** + * Process stored messages. + */ + def process(dispatcher: Dispatcher): Unit = { + var message: InboxMessage = null + inbox.synchronized { + if (!enableConcurrent && numActiveThreads != 0) { + return + } + message = messages.poll() + if (message != null) { + numActiveThreads += 1 + } else { + return + } + } + while (true) { + safelyCall(endpoint) { + message match { + case RpcMessage(_sender, content, context) => + try { + endpoint.receiveAndReply(context).applyOrElse[Any, Unit](content, { msg => + throw new CssException(s"Unsupported message $message from ${_sender}") + }) + } catch { + case e: Throwable => + context.sendFailure(e) + // Throw the exception -- this exception will be caught by the safelyCall function. + // The endpoint's onError function will be called. + throw e + } + + case OneWayMessage(_sender, content) => + endpoint.receive.applyOrElse[Any, Unit](content, { msg => + throw new CssException(s"Unsupported message $message from ${_sender}") + }) + + case OnStart => + endpoint.onStart() + if (!endpoint.isInstanceOf[ThreadSafeRpcEndpoint]) { + inbox.synchronized { + if (!stopped) { + enableConcurrent = true + } + } + } + + case OnStop => + val activeThreads = inbox.synchronized { inbox.numActiveThreads } + assert(activeThreads == 1, + s"There should be only a single active thread but found $activeThreads threads.") + dispatcher.removeRpcEndpointRef(endpoint) + endpoint.onStop() + assert(isEmpty, "OnStop should be the last message") + + case RemoteProcessConnected(remoteAddress) => + endpoint.onConnected(remoteAddress) + + case RemoteProcessDisconnected(remoteAddress) => + endpoint.onDisconnected(remoteAddress) + + case RemoteProcessConnectionError(cause, remoteAddress) => + endpoint.onNetworkError(cause, remoteAddress) + } + } + + inbox.synchronized { + // "enableConcurrent" will be set to false after `onStop` is called, so we should check it + // every time. + if (!enableConcurrent && numActiveThreads != 1) { + // If we are not the only one worker, exit + numActiveThreads -= 1 + return + } + message = messages.poll() + if (message == null) { + numActiveThreads -= 1 + return + } + } + } + } + + def post(message: InboxMessage): Unit = inbox.synchronized { + if (stopped) { + // We already put "OnStop" into "messages", so we should drop further messages + onDrop(message) + } else { + messages.add(message) + false + } + } + + def stop(): Unit = inbox.synchronized { + // The following codes should be in `synchronized` so that we can make sure "OnStop" is the last + // message + if (!stopped) { + // We should disable concurrent here. Then when RpcEndpoint.onStop is called, it's the only + // thread that is processing messages. So `RpcEndpoint.onStop` can release its resources + // safely. + enableConcurrent = false + stopped = true + messages.add(OnStop) + // Note: The concurrent events in messages will be processed one by one. + } + } + + def isEmpty: Boolean = inbox.synchronized { messages.isEmpty } + + /** + * Called when we are dropping a message. Test cases override this to test message dropping. + * Exposed for testing. + */ + protected def onDrop(message: InboxMessage): Unit = { + logWarning(s"Drop $message because $endpointRef is stopped") + } + + /** + * Calls action closure, and calls the endpoint's onError function in the case of exceptions. + */ + private def safelyCall(endpoint: RpcEndpoint)(action: => Unit): Unit = { + try action catch { + case NonFatal(e) => + try endpoint.onError(e) catch { + case NonFatal(ee) => + if (stopped) { + logDebug("Ignoring error", ee) + } else { + logError("Ignoring error", ee) + } + } + } + } + +} diff --git a/common/src/main/scala/com/bytedance/css/common/rpc/netty/NettyRpcCallContext.scala b/common/src/main/scala/com/bytedance/css/common/rpc/netty/NettyRpcCallContext.scala new file mode 100644 index 0000000..3d15f5e --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/rpc/netty/NettyRpcCallContext.scala @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc.netty + +import java.nio.ByteBuffer + +import scala.concurrent.Promise + +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.rpc.{RpcAddress, RpcCallContext} +import com.bytedance.css.network.client.RpcResponseCallback + +abstract class NettyRpcCallContext(override val senderAddress: RpcAddress) + extends RpcCallContext with Logging { + + protected def send(message: Any): Unit + + override def reply(response: Any): Unit = { + send(response) + } + + override def sendFailure(e: Throwable): Unit = { + send(RpcFailure(e)) + } + +} + +/** + * If the sender and the receiver are in the same process, the reply can be sent back via `Promise`. + */ +class LocalNettyRpcCallContext( + senderAddress: RpcAddress, + p: Promise[Any]) + extends NettyRpcCallContext(senderAddress) { + + override protected def send(message: Any): Unit = { + p.success(message) + } +} + +/** + * A [[RpcCallContext]] that will call [[RpcResponseCallback]] to send the reply back. + */ +class RemoteNettyRpcCallContext( + val nettyEnv: NettyRpcEnv, + callback: RpcResponseCallback, + senderAddress: RpcAddress) + extends NettyRpcCallContext(senderAddress) { + + override protected def send(message: Any): Unit = { + val reply: ByteBuffer = if (message.isInstanceOf[ByteBuffer]) { + message.asInstanceOf[ByteBuffer] + } else { + nettyEnv.serialize(message) + } + callback.onSuccess(reply) + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/rpc/netty/NettyRpcEnv.scala b/common/src/main/scala/com/bytedance/css/common/rpc/netty/NettyRpcEnv.scala new file mode 100644 index 0000000..3fb0372 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/rpc/netty/NettyRpcEnv.scala @@ -0,0 +1,724 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ +package com.bytedance.css.common.rpc.netty + +import java.io._ +import java.net.{InetSocketAddress, URI} +import java.nio.ByteBuffer +import java.nio.channels.{Pipe, ReadableByteChannel, WritableByteChannel} +import java.util.concurrent._ +import java.util.concurrent.atomic.AtomicBoolean +import javax.annotation.Nullable + +import scala.concurrent.{Future, Promise} +import scala.reflect.ClassTag +import scala.util.{DynamicVariable, Failure, Success, Try} +import scala.util.control.NonFatal + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.rpc.{RpcAddress, RpcEndpoint, RpcEndpointAddress, RpcEndpointNotFoundException} +import com.bytedance.css.common.rpc.{RpcEndpointRef, RpcEnv, RpcEnvConfig, RpcEnvFactory} +import com.bytedance.css.common.rpc.{RpcEnvFileServer, RpcEnvStoppedException, RpcTimeout} +import com.bytedance.css.common.serializer.{JavaSerializer, JavaSerializerInstance, SerializationStream} +import com.bytedance.css.common.util.{ByteBufferInputStream, ByteBufferOutputStream, ThreadUtils, Utils} +import com.bytedance.css.network.TransportContext +import com.bytedance.css.network.client.{RpcResponseCallback, StreamCallback, TransportClient} +import com.bytedance.css.network.client.{TransportClientBootstrap, TransportClientFactory} +import com.bytedance.css.network.server.{NoOpRpcHandler, RpcHandler, StreamManager} +import com.bytedance.css.network.server.{TransportServer, TransportServerBootstrap} + +class NettyRpcEnv( + val conf: CssConf, + javaSerializerInstance: JavaSerializerInstance, + host: String, + numUsableCores: Int) extends RpcEnv(conf) with Logging { + + val transportConf = Utils.fromCssConf( + conf.clone.set("spark.rpc.io.numConnectionsPerPeer", "1"), + "rpc", + conf.getInt("spark.rpc.io.threads", numUsableCores)) + + private val dispatcher: Dispatcher = new Dispatcher(this, numUsableCores) + + private val streamManager = new NettyStreamManager(this) + + private val transportContext = new TransportContext(transportConf, + new NettyRpcHandler(dispatcher, this, streamManager)) + + private def createClientBootstraps(): java.util.List[TransportClientBootstrap] = { + java.util.Collections.emptyList[TransportClientBootstrap] + } + + private val clientFactory = transportContext.createClientFactory(createClientBootstraps()) + + /** + * A separate client factory for file downloads. This avoids using the same RPC handler as + * the main RPC context, so that events caused by these clients are kept isolated from the + * main RPC traffic. + * + * It also allows for different configuration of certain properties, such as the number of + * connections per peer. + */ + @volatile private var fileDownloadFactory: TransportClientFactory = _ + + val timeoutScheduler = ThreadUtils.newDaemonSingleThreadScheduledExecutor("netty-rpc-env-timeout") + + // Because TransportClientFactory.createClient is blocking, we need to run it in this thread pool + // to implement non-blocking send/ask. + // TODO: a non-blocking TransportClientFactory.createClient in future + val clientConnectionExecutor = ThreadUtils.newDaemonCachedThreadPool( + "netty-rpc-connection", + conf.getInt("spark.rpc.connect.threads", 64)) + + @volatile private var server: TransportServer = _ + + private val stopped = new AtomicBoolean(false) + + /** + * A map for [[RpcAddress]] and [[Outbox]]. When we are connecting to a remote [[RpcAddress]], + * we just put messages to its [[Outbox]] to implement a non-blocking `send` method. + */ + private val outboxes = new ConcurrentHashMap[RpcAddress, Outbox]() + + /** + * Remove the address's Outbox and stop it. + */ + def removeOutbox(address: RpcAddress): Unit = { + val outbox = outboxes.remove(address) + if (outbox != null) { + outbox.stop() + } + } + + def startServer(bindAddress: String, port: Int): Unit = { + val bootstraps: java.util.List[TransportServerBootstrap] = + java.util.Collections.emptyList() + server = transportContext.createServer(bindAddress, port, bootstraps) + dispatcher.registerRpcEndpoint( + RpcEndpointVerifier.NAME, new RpcEndpointVerifier(this, dispatcher)) + } + + @Nullable + override lazy val address: RpcAddress = { + if (server != null) RpcAddress(host, server.getPort()) else null + } + + override def setupEndpoint(name: String, endpoint: RpcEndpoint): RpcEndpointRef = { + dispatcher.registerRpcEndpoint(name, endpoint) + } + + def asyncSetupEndpointRefByURI(uri: String): Future[RpcEndpointRef] = { + val addr = RpcEndpointAddress(uri) + val endpointRef = new NettyRpcEndpointRef(conf, addr, this) + val verifier = new NettyRpcEndpointRef( + conf, RpcEndpointAddress(addr.rpcAddress, RpcEndpointVerifier.NAME), this) + verifier.ask[Boolean](RpcEndpointVerifier.CheckExistence(endpointRef.name)).flatMap { find => + if (find) { + Future.successful(endpointRef) + } else { + Future.failed(new RpcEndpointNotFoundException(uri)) + } + }(ThreadUtils.sameThread) + } + + override def stop(endpointRef: RpcEndpointRef): Unit = { + require(endpointRef.isInstanceOf[NettyRpcEndpointRef]) + dispatcher.stop(endpointRef) + } + + private def postToOutbox(receiver: NettyRpcEndpointRef, message: OutboxMessage): Unit = { + if (receiver.client != null) { + message.sendWith(receiver.client) + } else { + require(receiver.address != null, + "Cannot send message to client endpoint with no listen address.") + val targetOutbox = { + val outbox = outboxes.get(receiver.address) + if (outbox == null) { + val newOutbox = new Outbox(this, receiver.address) + val oldOutbox = outboxes.putIfAbsent(receiver.address, newOutbox) + if (oldOutbox == null) { + newOutbox + } else { + oldOutbox + } + } else { + outbox + } + } + if (stopped.get) { + // It's possible that we put `targetOutbox` after stopping. So we need to clean it. + outboxes.remove(receiver.address) + targetOutbox.stop() + } else { + targetOutbox.send(message) + } + } + } + + def send(message: RequestMessage): Unit = { + val remoteAddr = message.receiver.address + if (remoteAddr == address) { + // Message to a local RPC endpoint. + try { + dispatcher.postOneWayMessage(message) + } catch { + case e: RpcEnvStoppedException => logDebug(e.getMessage) + } + } else { + // Message to a remote RPC endpoint. + postToOutbox(message.receiver, OneWayOutboxMessage(message.serialize(this))) + } + } + + def createClient(address: RpcAddress): TransportClient = { + clientFactory.createClient(address.host, address.port) + } + + def ask[T: ClassTag](message: RequestMessage, timeout: RpcTimeout): Future[T] = { + val promise = Promise[Any]() + val remoteAddr = message.receiver.address + + def onFailure(e: Throwable): Unit = { + if (!promise.tryFailure(e)) { + e match { + case e : RpcEnvStoppedException => logDebug (s"Ignored failure: $e") + case _ => logWarning(s"Ignored failure: $e") + } + } + } + + def onSuccess(reply: Any): Unit = reply match { + case RpcFailure(e) => onFailure(e) + case rpcReply => + if (!promise.trySuccess(rpcReply)) { + logWarning(s"Ignored message: $reply") + } + } + + try { + if (remoteAddr == address) { + val p = Promise[Any]() + p.future.onComplete { + case Success(response) => onSuccess(response) + case Failure(e) => onFailure(e) + }(ThreadUtils.sameThread) + dispatcher.postLocalMessage(message, p) + } else { + val rpcMessage = RpcOutboxMessage(message.serialize(this), + onFailure, + (client, response) => onSuccess(deserialize[Any](client, response))) + postToOutbox(message.receiver, rpcMessage) + promise.future.failed.foreach { + case _: TimeoutException => rpcMessage.onTimeout() + case _ => + }(ThreadUtils.sameThread) + } + + val timeoutCancelable = timeoutScheduler.schedule(new Runnable { + override def run(): Unit = { + val remoteRecAddr = if (remoteAddr == null) { + Try { + message.receiver.client.getChannel.remoteAddress() + }.toOption.orNull + } else { + remoteAddr + } + onFailure(new TimeoutException(s"Cannot receive any reply from ${remoteRecAddr} " + + s"in ${timeout.duration}")) + } + }, timeout.duration.toNanos, TimeUnit.NANOSECONDS) + promise.future.onComplete { v => + timeoutCancelable.cancel(true) + }(ThreadUtils.sameThread) + } catch { + case NonFatal(e) => + onFailure(e) + } + promise.future.mapTo[T].recover(timeout.addMessageIfTimeout)(ThreadUtils.sameThread) + } + + def serialize(content: Any): ByteBuffer = { + javaSerializerInstance.serialize(content) + } + + /** + * Returns [[SerializationStream]] that forwards the serialized bytes to `out`. + */ + def serializeStream(out: OutputStream): SerializationStream = { + javaSerializerInstance.serializeStream(out) + } + + def deserialize[T: ClassTag](client: TransportClient, bytes: ByteBuffer): T = { + NettyRpcEnv.currentClient.withValue(client) { + deserialize { () => + javaSerializerInstance.deserialize[T](bytes) + } + } + } + + override def endpointRef(endpoint: RpcEndpoint): RpcEndpointRef = { + dispatcher.getRpcEndpointRef(endpoint) + } + + override def shutdown(): Unit = { + cleanup() + } + + override def awaitTermination(): Unit = { + dispatcher.awaitTermination() + } + + private def cleanup(): Unit = { + if (!stopped.compareAndSet(false, true)) { + return + } + + val iter = outboxes.values().iterator() + while (iter.hasNext()) { + val outbox = iter.next() + outboxes.remove(outbox.address) + outbox.stop() + } + if (timeoutScheduler != null) { + timeoutScheduler.shutdownNow() + } + if (dispatcher != null) { + dispatcher.stop() + } + if (server != null) { + server.close() + } + if (clientFactory != null) { + clientFactory.close() + } + if (clientConnectionExecutor != null) { + clientConnectionExecutor.shutdownNow() + } + if (fileDownloadFactory != null) { + fileDownloadFactory.close() + } + } + + override def deserialize[T](deserializationAction: () => T): T = { + NettyRpcEnv.currentEnv.withValue(this) { + deserializationAction() + } + } + + override def fileServer: RpcEnvFileServer = streamManager + + override def openChannel(uri: String): ReadableByteChannel = { + val parsedUri = new URI(uri) + require(parsedUri.getHost() != null, "Host name must be defined.") + require(parsedUri.getPort() > 0, "Port must be defined.") + require(parsedUri.getPath() != null && parsedUri.getPath().nonEmpty, "Path must be defined.") + + val pipe = Pipe.open() + val source = new FileDownloadChannel(pipe.source()) + Utils.tryWithSafeFinallyAndFailureCallbacks(block = { + val client = downloadClient(parsedUri.getHost(), parsedUri.getPort()) + val callback = new FileDownloadCallback(pipe.sink(), source, client) + client.stream(parsedUri.getPath(), callback) + })(catchBlock = { + pipe.sink().close() + source.close() + }) + + source + } + + private def downloadClient(host: String, port: Int): TransportClient = { + if (fileDownloadFactory == null) synchronized { + if (fileDownloadFactory == null) { + val module = "files" + val prefix = "spark.rpc.io." + val clone = conf.clone() + + // Copy any RPC configuration that is not overridden in the spark.files namespace. + conf.getAll.foreach { case (key, value) => + if (key.startsWith(prefix)) { + val opt = key.substring(prefix.length()) + clone.setIfMissing(s"spark.$module.io.$opt", value) + } + } + + val ioThreads = clone.getInt("spark.files.io.threads", 1) + val downloadConf = Utils.fromCssConf(clone, module, ioThreads) + val downloadContext = new TransportContext(downloadConf, new NoOpRpcHandler(), true) + fileDownloadFactory = downloadContext.createClientFactory(createClientBootstraps()) + } + } + fileDownloadFactory.createClient(host, port) + } + + private class FileDownloadChannel(source: Pipe.SourceChannel) extends ReadableByteChannel { + + @volatile private var error: Throwable = _ + + def setError(e: Throwable): Unit = { + // This setError callback is invoked by internal RPC threads in order to propagate remote + // exceptions to application-level threads which are reading from this channel. When an + // RPC error occurs, the RPC system will call setError() and then will close the + // Pipe.SinkChannel corresponding to the other end of the `source` pipe. Closing of the pipe + // sink will cause `source.read()` operations to return EOF, unblocking the application-level + // reading thread. Thus there is no need to actually call `source.close()` here in the + // onError() callback and, in fact, calling it here would be dangerous because the close() + // would be asynchronous with respect to the read() call and could trigger race-conditions + // that lead to data corruption. See the PR for SPARK-22982 for more details on this topic. + error = e + } + + override def read(dst: ByteBuffer): Int = { + Try(source.read(dst)) match { + // See the documentation above in setError(): if an RPC error has occurred then setError() + // will be called to propagate the RPC error and then `source`'s corresponding + // Pipe.SinkChannel will be closed, unblocking this read. In that case, we want to propagate + // the remote RPC exception (and not any exceptions triggered by the pipe close, such as + // ChannelClosedException), hence this `error != null` check: + case _ if error != null => throw error + case Success(bytesRead) => bytesRead + case Failure(readErr) => throw readErr + } + } + + override def close(): Unit = source.close() + + override def isOpen(): Boolean = source.isOpen() + + } + + private class FileDownloadCallback( + sink: WritableByteChannel, + source: FileDownloadChannel, + client: TransportClient) extends StreamCallback { + + override def onData(streamId: String, buf: ByteBuffer): Unit = { + while (buf.remaining() > 0) { + sink.write(buf) + } + } + + override def onComplete(streamId: String): Unit = { + sink.close() + } + + override def onFailure(streamId: String, cause: Throwable): Unit = { + logDebug(s"Error downloading stream $streamId.", cause) + source.setError(cause) + sink.close() + } + + } +} + +object NettyRpcEnv extends Logging { + /** + * When deserializing the [[NettyRpcEndpointRef]], it needs a reference to [[NettyRpcEnv]]. + * Use `currentEnv` to wrap the deserialization codes. E.g., + * + * {{{ + * NettyRpcEnv.currentEnv.withValue(this) { + * your deserialization codes + * } + * }}} + */ + val currentEnv = new DynamicVariable[NettyRpcEnv](null) + + /** + * Similar to `currentEnv`, this variable references the client instance associated with an + * RPC, in case it's needed to find out the remote address during deserialization. + */ + val currentClient = new DynamicVariable[TransportClient](null) + +} + +private[rpc] class NettyRpcEnvFactory extends RpcEnvFactory with Logging { + + def create(config: RpcEnvConfig): RpcEnv = { + val CssConf = config.conf + // Use JavaSerializerInstance in multiple threads is safe. However, if we plan to support + // KryoSerializer in future, we have to use ThreadLocal to store SerializerInstance + val javaSerializerInstance = + new JavaSerializer(CssConf).newInstance().asInstanceOf[JavaSerializerInstance] + val nettyEnv = + new NettyRpcEnv(CssConf, javaSerializerInstance, config.advertiseAddress, + config.numUsableCores) + if (!config.clientMode) { + val startNettyRpcEnv: Int => (NettyRpcEnv, Int) = { actualPort => + nettyEnv.startServer(config.bindAddress, actualPort) + (nettyEnv, nettyEnv.address.port) + } + try { + Utils.startServiceOnPort(config.port, startNettyRpcEnv, CssConf, config.name)._1 + } catch { + case NonFatal(e) => + nettyEnv.shutdown() + throw e + } + } + nettyEnv + } +} + +/** + * The NettyRpcEnv version of RpcEndpointRef. + * + * This class behaves differently depending on where it's created. On the node that "owns" the + * RpcEndpoint, it's a simple wrapper around the RpcEndpointAddress instance. + * + * On other machines that receive a serialized version of the reference, the behavior changes. The + * instance will keep track of the TransportClient that sent the reference, so that messages + * to the endpoint are sent over the client connection, instead of needing a new connection to + * be opened. + * + * The RpcAddress of this ref can be null; what that means is that the ref can only be used through + * a client connection, since the process hosting the endpoint is not listening for incoming + * connections. These refs should not be shared with 3rd parties, since they will not be able to + * send messages to the endpoint. + * + * @param conf Spark configuration. + * @param endpointAddress The address where the endpoint is listening. + * @param nettyEnv The RpcEnv associated with this ref. + */ +class NettyRpcEndpointRef( + @transient private val conf: CssConf, + private val endpointAddress: RpcEndpointAddress, + @transient @volatile private var nettyEnv: NettyRpcEnv) extends RpcEndpointRef(conf) { + + @transient @volatile var client: TransportClient = _ + + override def address: RpcAddress = + if (endpointAddress.rpcAddress != null) endpointAddress.rpcAddress else null + + private def readObject(in: ObjectInputStream): Unit = { + in.defaultReadObject() + nettyEnv = NettyRpcEnv.currentEnv.value + client = NettyRpcEnv.currentClient.value + } + + private def writeObject(out: ObjectOutputStream): Unit = { + out.defaultWriteObject() + } + + override def name: String = endpointAddress.name + + override def ask[T: ClassTag](message: Any, timeout: RpcTimeout): Future[T] = { + nettyEnv.ask(new RequestMessage(nettyEnv.address, this, message), timeout) + } + + override def send(message: Any): Unit = { + require(message != null, "Message is null") + nettyEnv.send(new RequestMessage(nettyEnv.address, this, message)) + } + + override def toString: String = s"NettyRpcEndpointRef(${endpointAddress})" + + final override def equals(that: Any): Boolean = that match { + case other: NettyRpcEndpointRef => endpointAddress == other.endpointAddress + case _ => false + } + + final override def hashCode(): Int = + if (endpointAddress == null) 0 else endpointAddress.hashCode() +} + +/** + * The message that is sent from the sender to the receiver. + * + * @param senderAddress the sender address. It's `null` if this message is from a client + * `NettyRpcEnv`. + * @param receiver the receiver of this message. + * @param content the message content. + */ +class RequestMessage( + val senderAddress: RpcAddress, + val receiver: NettyRpcEndpointRef, + val content: Any) { + + /** Manually serialize [[RequestMessage]] to minimize the size. */ + def serialize(nettyEnv: NettyRpcEnv): ByteBuffer = { + val bos = new ByteBufferOutputStream() + val out = new DataOutputStream(bos) + try { + writeRpcAddress(out, senderAddress) + writeRpcAddress(out, receiver.address) + out.writeUTF(receiver.name) + val s = nettyEnv.serializeStream(out) + try { + s.writeObject(content) + } finally { + s.close() + } + } finally { + out.close() + } + bos.toByteBuffer + } + + private def writeRpcAddress(out: DataOutputStream, rpcAddress: RpcAddress): Unit = { + if (rpcAddress == null) { + out.writeBoolean(false) + } else { + out.writeBoolean(true) + out.writeUTF(rpcAddress.host) + out.writeInt(rpcAddress.port) + } + } + + override def toString: String = s"RequestMessage($senderAddress, $receiver, $content)" +} + +object RequestMessage { + + private def readRpcAddress(in: DataInputStream): RpcAddress = { + val hasRpcAddress = in.readBoolean() + if (hasRpcAddress) { + RpcAddress(in.readUTF(), in.readInt()) + } else { + null + } + } + + def apply(nettyEnv: NettyRpcEnv, client: TransportClient, bytes: ByteBuffer): RequestMessage = { + val bis = new ByteBufferInputStream(bytes) + val in = new DataInputStream(bis) + try { + val senderAddress = readRpcAddress(in) + val endpointAddress = RpcEndpointAddress(readRpcAddress(in), in.readUTF()) + val ref = new NettyRpcEndpointRef(nettyEnv.conf, endpointAddress, nettyEnv) + ref.client = client + new RequestMessage( + senderAddress, + ref, + // The remaining bytes in `bytes` are the message content. + nettyEnv.deserialize(client, bytes)) + } finally { + in.close() + } + } +} + +/** + * A response that indicates some failure happens in the receiver side. + */ +case class RpcFailure(e: Throwable) + +/** + * Dispatches incoming RPCs to registered endpoints. + * + * The handler keeps track of all client instances that communicate with it, so that the RpcEnv + * knows which `TransportClient` instance to use when sending RPCs to a client endpoint (i.e., + * one that is not listening for incoming connections, but rather needs to be contacted via the + * client socket). + * + * Events are sent on a per-connection basis, so if a client opens multiple connections to the + * RpcEnv, multiple connection / disconnection events will be created for that client (albeit + * with different `RpcAddress` information). + */ +class NettyRpcHandler( + dispatcher: Dispatcher, + nettyEnv: NettyRpcEnv, + streamManager: StreamManager) extends RpcHandler with Logging { + + // A variable to track the remote RpcEnv addresses of all clients + private val remoteAddresses = new ConcurrentHashMap[RpcAddress, RpcAddress]() + + override def receive( + client: TransportClient, + message: ByteBuffer, + callback: RpcResponseCallback): Unit = { + val messageToDispatch = internalReceive(client, message) + dispatcher.postRemoteMessage(messageToDispatch, callback) + } + + override def receive( + client: TransportClient, + message: ByteBuffer): Unit = { + val messageToDispatch = internalReceive(client, message) + dispatcher.postOneWayMessage(messageToDispatch) + } + + private def internalReceive(client: TransportClient, message: ByteBuffer): RequestMessage = { + val addr = client.getChannel().remoteAddress().asInstanceOf[InetSocketAddress] + assert(addr != null) + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) + val requestMessage = RequestMessage(nettyEnv, client, message) + if (requestMessage.senderAddress == null) { + // Create a new message with the socket address of the client as the sender. + new RequestMessage(clientAddr, requestMessage.receiver, requestMessage.content) + } else { + // The remote RpcEnv listens to some port, we should also fire a RemoteProcessConnected for + // the listening address + val remoteEnvAddress = requestMessage.senderAddress + if (remoteAddresses.putIfAbsent(clientAddr, remoteEnvAddress) == null) { + dispatcher.postToAll(RemoteProcessConnected(remoteEnvAddress)) + } + requestMessage + } + } + + override def getStreamManager: StreamManager = streamManager + + override def exceptionCaught(cause: Throwable, client: TransportClient): Unit = { + val addr = client.getChannel.remoteAddress().asInstanceOf[InetSocketAddress] + if (addr != null) { + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) + dispatcher.postToAll(RemoteProcessConnectionError(cause, clientAddr)) + // If the remove RpcEnv listens to some address, we should also fire a + // RemoteProcessConnectionError for the remote RpcEnv listening address + val remoteEnvAddress = remoteAddresses.get(clientAddr) + if (remoteEnvAddress != null) { + dispatcher.postToAll(RemoteProcessConnectionError(cause, remoteEnvAddress)) + } + } else { + // If the channel is closed before connecting, its remoteAddress will be null. + // See java.net.Socket.getRemoteSocketAddress + // Because we cannot get a RpcAddress, just log it + logError("Exception before connecting to the client", cause) + } + } + + override def channelActive(client: TransportClient): Unit = { + val addr = client.getChannel().remoteAddress().asInstanceOf[InetSocketAddress] + assert(addr != null) + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) + dispatcher.postToAll(RemoteProcessConnected(clientAddr)) + } + + override def channelInactive(client: TransportClient): Unit = { + val addr = client.getChannel.remoteAddress().asInstanceOf[InetSocketAddress] + if (addr != null) { + val clientAddr = RpcAddress(addr.getHostString, addr.getPort) + nettyEnv.removeOutbox(clientAddr) + dispatcher.postToAll(RemoteProcessDisconnected(clientAddr)) + val remoteEnvAddress = remoteAddresses.remove(clientAddr) + // If the remove RpcEnv listens to some address, we should also fire a + // RemoteProcessDisconnected for the remote RpcEnv listening address + if (remoteEnvAddress != null) { + dispatcher.postToAll(RemoteProcessDisconnected(remoteEnvAddress)) + } + } else { + // If the channel is closed before connecting, its remoteAddress will be null. In this case, + // we can ignore it since we don't fire "Associated". + // See java.net.Socket.getRemoteSocketAddress + } + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/rpc/netty/NettyStreamManager.scala b/common/src/main/scala/com/bytedance/css/common/rpc/netty/NettyStreamManager.scala new file mode 100644 index 0000000..b68a629 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/rpc/netty/NettyStreamManager.scala @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ +package com.bytedance.css.common.rpc.netty + +import java.io.File +import java.util.concurrent.ConcurrentHashMap + +import com.bytedance.css.common.rpc.RpcEnvFileServer +import com.bytedance.css.common.util.Utils +import com.bytedance.css.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} +import com.bytedance.css.network.server.StreamManager + +class NettyStreamManager(rpcEnv: NettyRpcEnv) + extends StreamManager with RpcEnvFileServer { + + private val files = new ConcurrentHashMap[String, File]() + private val jars = new ConcurrentHashMap[String, File]() + private val dirs = new ConcurrentHashMap[String, File]() + + override def getChunk(streamId: Long, chunkIndex: Int): ManagedBuffer = { + throw new UnsupportedOperationException() + } + + override def openStream(streamId: String): ManagedBuffer = { + val Array(ftype, fname) = streamId.stripPrefix("/").split("/", 2) + val file = ftype match { + case "files" => files.get(fname) + case "jars" => jars.get(fname) + case other => + val dir = dirs.get(ftype) + require(dir != null, s"Invalid stream URI: $ftype not found.") + new File(dir, fname) + } + + if (file != null && file.isFile()) { + new FileSegmentManagedBuffer(rpcEnv.transportConf, file, 0, file.length()) + } else { + null + } + } + + override def addFile(file: File): String = { + val existingPath = files.putIfAbsent(file.getName, file) + require(existingPath == null || existingPath == file, + s"File ${file.getName} was already registered with a different path " + + s"(old path = $existingPath, new path = $file") + s"${rpcEnv.address.toCssURL}/files/${Utils.encodeFileNameToURIRawPath(file.getName())}" + } + + override def addJar(file: File): String = { + val existingPath = jars.putIfAbsent(file.getName, file) + require(existingPath == null || existingPath == file, + s"File ${file.getName} was already registered with a different path " + + s"(old path = $existingPath, new path = $file") + s"${rpcEnv.address.toCssURL}/jars/${Utils.encodeFileNameToURIRawPath(file.getName())}" + } + + override def addDirectory(baseUri: String, path: File): String = { + val fixedBaseUri = validateDirectoryUri(baseUri) + require(dirs.putIfAbsent(fixedBaseUri.stripPrefix("/"), path) == null, + s"URI '$fixedBaseUri' already registered.") + s"${rpcEnv.address.toCssURL}$fixedBaseUri" + } + +} diff --git a/common/src/main/scala/com/bytedance/css/common/rpc/netty/Outbox.scala b/common/src/main/scala/com/bytedance/css/common/rpc/netty/Outbox.scala new file mode 100644 index 0000000..f15fea7 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/rpc/netty/Outbox.scala @@ -0,0 +1,277 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc.netty + +import java.nio.ByteBuffer +import java.util.concurrent.Callable +import javax.annotation.concurrent.GuardedBy + +import scala.util.control.NonFatal + +import com.bytedance.css.common.exception.CssException +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.rpc.{RpcAddress, RpcEnvStoppedException} +import com.bytedance.css.network.client.{RpcResponseCallback, TransportClient} + +sealed trait OutboxMessage { + + def sendWith(client: TransportClient): Unit + + def onFailure(e: Throwable): Unit + +} + +case class OneWayOutboxMessage(content: ByteBuffer) extends OutboxMessage + with Logging { + + override def sendWith(client: TransportClient): Unit = { + client.send(content) + } + + override def onFailure(e: Throwable): Unit = { + e match { + case e1: RpcEnvStoppedException => logDebug(e1.getMessage) + case e1: Throwable => logWarning(s"Failed to send one-way RPC.", e1) + } + } + +} + +case class RpcOutboxMessage( + content: ByteBuffer, + _onFailure: (Throwable) => Unit, + _onSuccess: (TransportClient, ByteBuffer) => Unit) + extends OutboxMessage with RpcResponseCallback with Logging { + + private var client: TransportClient = _ + private var requestId: Long = _ + + override def sendWith(client: TransportClient): Unit = { + this.client = client + this.requestId = client.sendRpc(content, this) + } + + def onTimeout(): Unit = { + if (client != null) { + client.removeRpcRequest(requestId) + } else { + logError("Ask timeout before connecting successfully") + } + } + + override def onFailure(e: Throwable): Unit = { + _onFailure(e) + } + + override def onSuccess(response: ByteBuffer): Unit = { + _onSuccess(client, response) + } + +} + +class Outbox(nettyEnv: NettyRpcEnv, val address: RpcAddress) { + + outbox => // Give this an alias so we can use it more clearly in closures. + + @GuardedBy("this") + private val messages = new java.util.LinkedList[OutboxMessage] + + @GuardedBy("this") + private var client: TransportClient = null + + /** + * connectFuture points to the connect task. If there is no connect task, connectFuture will be + * null. + */ + @GuardedBy("this") + private var connectFuture: java.util.concurrent.Future[Unit] = null + + @GuardedBy("this") + private var stopped = false + + /** + * If there is any thread draining the message queue + */ + @GuardedBy("this") + private var draining = false + + /** + * Send a message. If there is no active connection, cache it and launch a new connection. If + * [[Outbox]] is stopped, the sender will be notified with a [[SparkException]]. + */ + def send(message: OutboxMessage): Unit = { + val dropped = synchronized { + if (stopped) { + true + } else { + messages.add(message) + false + } + } + if (dropped) { + message.onFailure(new CssException("Message is dropped because Outbox is stopped")) + } else { + drainOutbox() + } + } + + /** + * Drain the message queue. If there is other draining thread, just exit. If the connection has + * not been established, launch a task in the `nettyEnv.clientConnectionExecutor` to setup the + * connection. + */ + private def drainOutbox(): Unit = { + var message: OutboxMessage = null + synchronized { + if (stopped) { + return + } + if (connectFuture != null) { + // We are connecting to the remote address, so just exit + return + } + if (client == null) { + // There is no connect task but client is null, so we need to launch the connect task. + launchConnectTask() + return + } + if (draining) { + // There is some thread draining, so just exit + return + } + message = messages.poll() + if (message == null) { + return + } + draining = true + } + while (true) { + try { + val _client = synchronized { client } + if (_client != null) { + message.sendWith(_client) + } else { + assert(stopped == true) + } + } catch { + case NonFatal(e) => + handleNetworkFailure(e) + return + } + synchronized { + if (stopped) { + return + } + message = messages.poll() + if (message == null) { + draining = false + return + } + } + } + } + + private def launchConnectTask(): Unit = { + connectFuture = nettyEnv.clientConnectionExecutor.submit(new Callable[Unit] { + + override def call(): Unit = { + try { + val _client = nettyEnv.createClient(address) + outbox.synchronized { + client = _client + if (stopped) { + closeClient() + } + } + } catch { + case ie: InterruptedException => + // exit + return + case NonFatal(e) => + outbox.synchronized { connectFuture = null } + handleNetworkFailure(e) + return + } + outbox.synchronized { connectFuture = null } + // It's possible that no thread is draining now. If we don't drain here, we cannot send the + // messages until the next message arrives. + drainOutbox() + } + }) + } + + /** + * Stop [[Inbox]] and notify the waiting messages with the cause. + */ + private def handleNetworkFailure(e: Throwable): Unit = { + synchronized { + assert(connectFuture == null) + if (stopped) { + return + } + stopped = true + closeClient() + } + // Remove this Outbox from nettyEnv so that the further messages will create a new Outbox along + // with a new connection + nettyEnv.removeOutbox(address) + + // Notify the connection failure for the remaining messages + // + // We always check `stopped` before updating messages, so here we can make sure no thread will + // update messages and it's safe to just drain the queue. + var message = messages.poll() + while (message != null) { + message.onFailure(e) + message = messages.poll() + } + assert(messages.isEmpty) + } + + private def closeClient(): Unit = synchronized { + // Just set client to null. Don't close it in order to reuse the connection. + client = null + } + + /** + * Stop [[Outbox]]. The remaining messages in the [[Outbox]] will be notified with a + * [[SparkException]]. + */ + def stop(): Unit = { + synchronized { + if (stopped) { + return + } + stopped = true + if (connectFuture != null) { + connectFuture.cancel(true) + } + closeClient() + } + + // We always check `stopped` before updating messages, so here we can make sure no thread will + // update messages and it's safe to just drain the queue. + var message = messages.poll() + while (message != null) { + message.onFailure(new CssException("Message is dropped because Outbox is stopped")) + message = messages.poll() + } + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/rpc/netty/RpcEndpointVerifier.scala b/common/src/main/scala/com/bytedance/css/common/rpc/netty/RpcEndpointVerifier.scala new file mode 100644 index 0000000..1a616ff --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/rpc/netty/RpcEndpointVerifier.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc.netty + +import com.bytedance.css.common.rpc.{RpcCallContext, RpcEndpoint, RpcEnv} + +/** + * An [[RpcEndpoint]] for remote [[RpcEnv]]s to query if an `RpcEndpoint` exists. + * + * This is used when setting up a remote endpoint reference. + */ +class RpcEndpointVerifier(override val rpcEnv: RpcEnv, dispatcher: Dispatcher) + extends RpcEndpoint { + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case RpcEndpointVerifier.CheckExistence(name) => context.reply(dispatcher.verify(name)) + } +} + +object RpcEndpointVerifier { + val NAME = "endpoint-verifier" + + /** A message used to ask the remote [[RpcEndpointVerifier]] if an `RpcEndpoint` exists. */ + case class CheckExistence(name: String) +} diff --git a/common/src/main/scala/com/bytedance/css/common/serializer/JavaSerializer.scala b/common/src/main/scala/com/bytedance/css/common/serializer/JavaSerializer.scala new file mode 100644 index 0000000..1952087 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/serializer/JavaSerializer.scala @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.serializer + +import java.io._ +import java.nio.ByteBuffer + +import scala.reflect.ClassTag + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.util.{ByteBufferInputStream, ByteBufferOutputStream, Utils} + +class JavaSerializationStream( + out: OutputStream, counterReset: Int, extraDebugInfo: Boolean) + extends SerializationStream { + private val objOut = new ObjectOutputStream(out) + private var counter = 0 + + /** + * Calling reset to avoid memory leak: + * http://stackoverflow.com/questions/1281549/memory-leak-traps-in-the-java-standard-api + * But only call it every 100th time to avoid bloated serialization streams (when + * the stream 'resets' object class descriptions have to be re-written) + */ + def writeObject[T: ClassTag](t: T): SerializationStream = { + try { + objOut.writeObject(t) + } catch { + case e: NotSerializableException if extraDebugInfo => + throw SerializationDebugger.improveException(t, e) + } + counter += 1 + if (counterReset > 0 && counter >= counterReset) { + objOut.reset() + counter = 0 + } + this + } + + def flush() { objOut.flush() } + def close() { objOut.close() } +} + +class JavaDeserializationStream(in: InputStream, loader: ClassLoader) + extends DeserializationStream { + + private val objIn = new ObjectInputStream(in) { + override def resolveClass(desc: ObjectStreamClass): Class[_] = + try { + // scalastyle:off classforname + Class.forName(desc.getName, false, loader) + // scalastyle:on classforname + } catch { + case e: ClassNotFoundException => + JavaDeserializationStream.primitiveMappings.getOrElse(desc.getName, throw e) + } + } + + def readObject[T: ClassTag](): T = objIn.readObject().asInstanceOf[T] + def close() { objIn.close() } +} + +private object JavaDeserializationStream { + val primitiveMappings = Map[String, Class[_]]( + "boolean" -> classOf[Boolean], + "byte" -> classOf[Byte], + "char" -> classOf[Char], + "short" -> classOf[Short], + "int" -> classOf[Int], + "long" -> classOf[Long], + "float" -> classOf[Float], + "double" -> classOf[Double], + "void" -> classOf[Void] + ) +} + +class JavaSerializerInstance( + counterReset: Int, extraDebugInfo: Boolean, defaultClassLoader: ClassLoader) + extends SerializerInstance { + + override def serialize[T: ClassTag](t: T): ByteBuffer = { + val bos = new ByteBufferOutputStream() + val out = serializeStream(bos) + out.writeObject(t) + out.close() + bos.toByteBuffer + } + + override def deserialize[T: ClassTag](bytes: ByteBuffer): T = { + val bis = new ByteBufferInputStream(bytes) + val in = deserializeStream(bis) + in.readObject() + } + + override def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { + val bis = new ByteBufferInputStream(bytes) + val in = deserializeStream(bis, loader) + in.readObject() + } + + override def serializeStream(s: OutputStream): SerializationStream = { + new JavaSerializationStream(s, counterReset, extraDebugInfo) + } + + override def deserializeStream(s: InputStream): DeserializationStream = { + new JavaDeserializationStream(s, defaultClassLoader) + } + + def deserializeStream(s: InputStream, loader: ClassLoader): DeserializationStream = { + new JavaDeserializationStream(s, loader) + } +} + +/** + * :: DeveloperApi :: + * A Spark serializer that uses Java's built-in serialization. + * + * @note This serializer is not guaranteed to be wire-compatible across different versions of + * Spark. It is intended to be used to serialize/de-serialize data within a single + * Spark application. + */ +class JavaSerializer(conf: CssConf) extends Serializer with Externalizable { + private var counterReset = conf.getInt("css.serializer.objectStreamReset", 100) + private var extraDebugInfo = conf.getBoolean("css.serializer.extraDebugInfo", true) + + protected def this() = this(new CssConf()) // For deserialization only + + override def newInstance(): SerializerInstance = { + val classLoader = defaultClassLoader.getOrElse(Thread.currentThread.getContextClassLoader) + new JavaSerializerInstance(counterReset, extraDebugInfo, classLoader) + } + + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { + out.writeInt(counterReset) + out.writeBoolean(extraDebugInfo) + } + + override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { + counterReset = in.readInt() + extraDebugInfo = in.readBoolean() + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/serializer/SerializationDebugger.scala b/common/src/main/scala/com/bytedance/css/common/serializer/SerializationDebugger.scala new file mode 100644 index 0000000..dfa60a5 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/serializer/SerializationDebugger.scala @@ -0,0 +1,425 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.serializer + +import java.io._ +import java.lang.reflect.{Field, Method} +import java.security.AccessController + +import scala.annotation.tailrec +import scala.collection.mutable +import scala.util.control.NonFatal + +import com.bytedance.css.common.internal.Logging + +object SerializationDebugger extends Logging { + + /** + * Improve the given NotSerializableException with the serialization path leading from the given + * object to the problematic object. This is turned off automatically if + * `sun.io.serialization.extendedDebugInfo` flag is turned on for the JVM. + */ + def improveException(obj: Any, e: NotSerializableException): NotSerializableException = { + if (enableDebugging && reflect != null) { + try { + new NotSerializableException( + e.getMessage + "\nSerialization stack:\n" + find(obj).map("\t- " + _).mkString("\n")) + } catch { + case NonFatal(t) => + // Fall back to old exception + logWarning("Exception in serialization debugger", t) + e + } + } else { + e + } + } + + /** + * Find the path leading to a not serializable object. This method is modeled after OpenJDK's + * serialization mechanism, and handles the following cases: + * + * - primitives + * - arrays of primitives + * - arrays of non-primitive objects + * - Serializable objects + * - Externalizable objects + * - writeReplace + * + * It does not yet handle writeObject override, but that shouldn't be too hard to do either. + */ + private[serializer] def find(obj: Any): List[String] = { + new SerializationDebugger().visit(obj, List.empty) + } + + private[serializer] var enableDebugging: Boolean = { + !AccessController.doPrivileged(new sun.security.action.GetBooleanAction( + "sun.io.serialization.extendedDebugInfo")).booleanValue() + } + + private class SerializationDebugger { + + /** A set to track the list of objects we have visited, to avoid cycles in the graph. */ + private val visited = new mutable.HashSet[Any] + + /** + * Visit the object and its fields and stop when we find an object that is not serializable. + * Return the path as a list. If everything can be serialized, return an empty list. + */ + def visit(o: Any, stack: List[String]): List[String] = { + if (o == null) { + List.empty + } else if (visited.contains(o)) { + List.empty + } else { + visited += o + o match { + // Primitive value, string, and primitive arrays are always serializable + case _ if o.getClass.isPrimitive => List.empty + case _: String => List.empty + case _ if o.getClass.isArray && o.getClass.getComponentType.isPrimitive => List.empty + + // Traverse non primitive array. + case a: Array[_] if o.getClass.isArray && !o.getClass.getComponentType.isPrimitive => + val elem = s"array (class ${a.getClass.getName}, size ${a.length})" + visitArray(o.asInstanceOf[Array[_]], elem :: stack) + + case e: java.io.Externalizable => + val elem = s"externalizable object (class ${e.getClass.getName}, $e)" + visitExternalizable(e, elem :: stack) + + case s: Object with java.io.Serializable => + val elem = s"object (class ${s.getClass.getName}, $s)" + visitSerializable(s, elem :: stack) + + case _ => + // Found an object that is not serializable! + s"object not serializable (class: ${o.getClass.getName}, value: $o)" :: stack + } + } + } + + private def visitArray(o: Array[_], stack: List[String]): List[String] = { + var i = 0 + while (i < o.length) { + val childStack = visit(o(i), s"element of array (index: $i)" :: stack) + if (childStack.nonEmpty) { + return childStack + } + i += 1 + } + return List.empty + } + + /** + * Visit an externalizable object. + * Since writeExternal() can choose to add arbitrary objects at the time of serialization, + * the only way to capture all the objects it will serialize is by using a + * dummy ObjectOutput that collects all the relevant objects for further testing. + */ + private def visitExternalizable(o: java.io.Externalizable, stack: List[String]): List[String] = + { + val fieldList = new ListObjectOutput + o.writeExternal(fieldList) + val childObjects = fieldList.outputArray + var i = 0 + while (i < childObjects.length) { + val childStack = visit(childObjects(i), "writeExternal data" :: stack) + if (childStack.nonEmpty) { + return childStack + } + i += 1 + } + return List.empty + } + + private def visitSerializable(o: Object, stack: List[String]): List[String] = { + // An object contains multiple slots in serialization. + // Get the slots and visit fields in all of them. + val (finalObj, desc) = findObjectAndDescriptor(o) + + // If the object has been replaced using writeReplace(), + // then call visit() on it again to test its type again. + if (finalObj.getClass != o.getClass) { + return visit(finalObj, s"writeReplace data (class: ${finalObj.getClass.getName})" :: stack) + } + + // Every class is associated with one or more "slots", each slot refers to the parent + // classes of this class. These slots are used by the ObjectOutputStream + // serialization code to recursively serialize the fields of an object and + // its parent classes. For example, if there are the following classes. + // + // class ParentClass(parentField: Int) + // class ChildClass(childField: Int) extends ParentClass(1) + // + // Then serializing the an object Obj of type ChildClass requires first serializing the fields + // of ParentClass (that is, parentField), and then serializing the fields of ChildClass + // (that is, childField). Correspondingly, there will be two slots related to this object: + // + // 1. ParentClass slot, which will be used to serialize parentField of Obj + // 2. ChildClass slot, which will be used to serialize childField fields of Obj + // + // The following code uses the description of each slot to find the fields in the + // corresponding object to visit. + // + val slotDescs = desc.getSlotDescs + var i = 0 + while (i < slotDescs.length) { + val slotDesc = slotDescs(i) + if (slotDesc.hasWriteObjectMethod) { + // If the class type corresponding to current slot has writeObject() defined, + // then its not obvious which fields of the class will be serialized as the writeObject() + // can choose arbitrary fields for serialization. This case is handled separately. + val elem = s"writeObject data (class: ${slotDesc.getName})" + val childStack = visitSerializableWithWriteObjectMethod(finalObj, elem :: stack) + if (childStack.nonEmpty) { + return childStack + } + } else { + // Visit all the fields objects of the class corresponding to the current slot. + val fields: Array[ObjectStreamField] = slotDesc.getFields + val objFieldValues: Array[Object] = new Array[Object](slotDesc.getNumObjFields) + val numPrims = fields.length - objFieldValues.length + slotDesc.getObjFieldValues(finalObj, objFieldValues) + + var j = 0 + while (j < objFieldValues.length) { + val fieldDesc = fields(numPrims + j) + val elem = s"field (class: ${slotDesc.getName}" + + s", name: ${fieldDesc.getName}" + + s", type: ${fieldDesc.getType})" + val childStack = visit(objFieldValues(j), elem :: stack) + if (childStack.nonEmpty) { + return childStack + } + j += 1 + } + } + i += 1 + } + return List.empty + } + + /** + * Visit a serializable object which has the writeObject() defined. + * Since writeObject() can choose to add arbitrary objects at the time of serialization, + * the only way to capture all the objects it will serialize is by using a + * dummy ObjectOutputStream that collects all the relevant fields for further testing. + * This is similar to how externalizable objects are visited. + */ + private def visitSerializableWithWriteObjectMethod( + o: Object, stack: List[String]): List[String] = { + val innerObjectsCatcher = new ListObjectOutputStream + var notSerializableFound = false + try { + innerObjectsCatcher.writeObject(o) + } catch { + case io: IOException => + notSerializableFound = true + } + + // If something was not serializable, then visit the captured objects. + // Otherwise, all the captured objects are safely serializable, so no need to visit them. + // As an optimization, just added them to the visited list. + if (notSerializableFound) { + val innerObjects = innerObjectsCatcher.outputArray + var k = 0 + while (k < innerObjects.length) { + val childStack = visit(innerObjects(k), stack) + if (childStack.nonEmpty) { + return childStack + } + k += 1 + } + } else { + visited ++= innerObjectsCatcher.outputArray + } + return List.empty + } + } + + /** + * Find the object to serialize and the associated [[ObjectStreamClass]]. This method handles + * writeReplace in Serializable. It starts with the object itself, and keeps calling the + * writeReplace method until there is no more. + */ + @tailrec + private def findObjectAndDescriptor(o: Object): (Object, ObjectStreamClass) = { + val cl = o.getClass + val desc = ObjectStreamClass.lookupAny(cl) + if (!desc.hasWriteReplaceMethod) { + (o, desc) + } else { + val replaced = desc.invokeWriteReplace(o) + // `writeReplace` recursion stops when the returned object has the same class. + if (replaced.getClass == o.getClass) { + (replaced, desc) + } else { + findObjectAndDescriptor(replaced) + } + } + } + + /** + * A dummy [[ObjectOutput]] that simply saves the list of objects written by a writeExternal + * call, and returns them through `outputArray`. + */ + private class ListObjectOutput extends ObjectOutput { + private val output = new mutable.ArrayBuffer[Any] + def outputArray: Array[Any] = output.toArray + override def writeObject(o: Any): Unit = output += o + override def flush(): Unit = {} + override def write(i: Int): Unit = {} + override def write(bytes: Array[Byte]): Unit = {} + override def write(bytes: Array[Byte], i: Int, i1: Int): Unit = {} + override def close(): Unit = {} + override def writeFloat(v: Float): Unit = {} + override def writeChars(s: String): Unit = {} + override def writeDouble(v: Double): Unit = {} + override def writeUTF(s: String): Unit = {} + override def writeShort(i: Int): Unit = {} + override def writeInt(i: Int): Unit = {} + override def writeBoolean(b: Boolean): Unit = {} + override def writeBytes(s: String): Unit = {} + override def writeChar(i: Int): Unit = {} + override def writeLong(l: Long): Unit = {} + override def writeByte(i: Int): Unit = {} + } + + /** An output stream that emulates /dev/null */ + private class NullOutputStream extends OutputStream { + override def write(b: Int) { } + } + + /** + * A dummy [[ObjectOutputStream]] that saves the list of objects written to it and returns + * them through `outputArray`. This works by using the [[ObjectOutputStream]]'s `replaceObject()` + * method which gets called on every object, only if replacing is enabled. So this subclass + * of [[ObjectOutputStream]] enabled replacing, and uses replaceObject to get the objects that + * are being serializabled. The serialized bytes are ignored by sending them to a + * [[NullOutputStream]], which acts like a /dev/null. + */ + private class ListObjectOutputStream extends ObjectOutputStream(new NullOutputStream) { + private val output = new mutable.ArrayBuffer[Any] + this.enableReplaceObject(true) + + def outputArray: Array[Any] = output.toArray + + override def replaceObject(obj: Object): Object = { + output += obj + obj + } + } + + /** An implicit class that allows us to call private methods of ObjectStreamClass. */ + implicit class ObjectStreamClassMethods(val desc: ObjectStreamClass) extends AnyVal { + def getSlotDescs: Array[ObjectStreamClass] = { + reflect.GetClassDataLayout.invoke(desc).asInstanceOf[Array[Object]].map { + classDataSlot => reflect.DescField.get(classDataSlot).asInstanceOf[ObjectStreamClass] + } + } + + def hasWriteObjectMethod: Boolean = { + reflect.HasWriteObjectMethod.invoke(desc).asInstanceOf[Boolean] + } + + def hasWriteReplaceMethod: Boolean = { + reflect.HasWriteReplaceMethod.invoke(desc).asInstanceOf[Boolean] + } + + def invokeWriteReplace(obj: Object): Object = { + reflect.InvokeWriteReplace.invoke(desc, obj) + } + + def getNumObjFields: Int = { + reflect.GetNumObjFields.invoke(desc).asInstanceOf[Int] + } + + def getObjFieldValues(obj: Object, out: Array[Object]): Unit = { + reflect.GetObjFieldValues.invoke(desc, obj, out) + } + } + + /** + * Object to hold all the reflection objects. If we run on a JVM that we cannot understand, + * this field will be null and this the debug helper should be disabled. + */ + private val reflect: ObjectStreamClassReflection = try { + new ObjectStreamClassReflection + } catch { + case e: Exception => + logWarning("Cannot find private methods using reflection", e) + null + } + + private class ObjectStreamClassReflection { + /** ObjectStreamClass.getClassDataLayout */ + val GetClassDataLayout: Method = { + val f = classOf[ObjectStreamClass].getDeclaredMethod("getClassDataLayout") + f.setAccessible(true) + f + } + + /** ObjectStreamClass.hasWriteObjectMethod */ + val HasWriteObjectMethod: Method = { + val f = classOf[ObjectStreamClass].getDeclaredMethod("hasWriteObjectMethod") + f.setAccessible(true) + f + } + + /** ObjectStreamClass.hasWriteReplaceMethod */ + val HasWriteReplaceMethod: Method = { + val f = classOf[ObjectStreamClass].getDeclaredMethod("hasWriteReplaceMethod") + f.setAccessible(true) + f + } + + /** ObjectStreamClass.invokeWriteReplace */ + val InvokeWriteReplace: Method = { + val f = classOf[ObjectStreamClass].getDeclaredMethod("invokeWriteReplace", classOf[Object]) + f.setAccessible(true) + f + } + + /** ObjectStreamClass.getNumObjFields */ + val GetNumObjFields: Method = { + val f = classOf[ObjectStreamClass].getDeclaredMethod("getNumObjFields") + f.setAccessible(true) + f + } + + /** ObjectStreamClass.getObjFieldValues */ + val GetObjFieldValues: Method = { + val f = classOf[ObjectStreamClass].getDeclaredMethod( + "getObjFieldValues", classOf[Object], classOf[Array[Object]]) + f.setAccessible(true) + f + } + + /** ObjectStreamClass$ClassDataSlot.desc field */ + val DescField: Field = { + // scalastyle:off classforname + val f = Class.forName("java.io.ObjectStreamClass$ClassDataSlot").getDeclaredField("desc") + // scalastyle:on classforname + f.setAccessible(true) + f + } + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/serializer/Serializer.scala b/common/src/main/scala/com/bytedance/css/common/serializer/Serializer.scala new file mode 100644 index 0000000..e0fd8fb --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/serializer/Serializer.scala @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.serializer + +import java.io._ +import java.nio.ByteBuffer +import javax.annotation.concurrent.NotThreadSafe + +import scala.reflect.ClassTag + +import com.bytedance.css.common.util.NextIterator + +abstract class Serializer { + + /** + * Default ClassLoader to use in deserialization. Implementations of [[Serializer]] should + * make sure it is using this when set. + */ + @volatile protected var defaultClassLoader: Option[ClassLoader] = None + + /** + * Sets a class loader for the serializer to use in deserialization. + * + * @return this Serializer object + */ + def setDefaultClassLoader(classLoader: ClassLoader): Serializer = { + defaultClassLoader = Some(classLoader) + this + } + + /** Creates a new [[SerializerInstance]]. */ + def newInstance(): SerializerInstance + + /** + * :: Private :: + * Returns true if this serializer supports relocation of its serialized objects and false + * otherwise. This should return true if and only if reordering the bytes of serialized objects + * in serialization stream output is equivalent to having re-ordered those elements prior to + * serializing them. More specifically, the following should hold if a serializer supports + * relocation: + * + * {{{ + * serOut.open() + * position = 0 + * serOut.write(obj1) + * serOut.flush() + * position = # of bytes written to stream so far + * obj1Bytes = output[0:position-1] + * serOut.write(obj2) + * serOut.flush() + * position2 = # of bytes written to stream so far + * obj2Bytes = output[position:position2-1] + * serIn.open([obj2bytes] concatenate [obj1bytes]) should return (obj2, obj1) + * }}} + * + * In general, this property should hold for serializers that are stateless and that do not + * write special metadata at the beginning or end of the serialization stream. + * + * This API is private to Spark; this method should not be overridden in third-party subclasses + * or called in user code and is subject to removal in future Spark releases. + * + * See SPARK-7311 for more details. + */ + def supportsRelocationOfSerializedObjects: Boolean = false +} + + +/** + * :: DeveloperApi :: + * An instance of a serializer, for use by one thread at a time. + * + * It is legal to create multiple serialization / deserialization streams from the same + * SerializerInstance as long as those streams are all used within the same thread. + */ +@NotThreadSafe +abstract class SerializerInstance { + def serialize[T: ClassTag](t: T): ByteBuffer + + def deserialize[T: ClassTag](bytes: ByteBuffer): T + + def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T + + def serializeStream(s: OutputStream): SerializationStream + + def deserializeStream(s: InputStream): DeserializationStream +} + +/** + * :: DeveloperApi :: + * A stream for writing serialized objects. + */ +abstract class SerializationStream extends Closeable { + /** The most general-purpose method to write an object. */ + def writeObject[T: ClassTag](t: T): SerializationStream + /** Writes the object representing the key of a key-value pair. */ + def writeKey[T: ClassTag](key: T): SerializationStream = writeObject(key) + /** Writes the object representing the value of a key-value pair. */ + def writeValue[T: ClassTag](value: T): SerializationStream = writeObject(value) + def flush(): Unit + override def close(): Unit + + def writeAll[T: ClassTag](iter: Iterator[T]): SerializationStream = { + while (iter.hasNext) { + writeObject(iter.next()) + } + this + } +} + + +/** + * :: DeveloperApi :: + * A stream for reading serialized objects. + */ +abstract class DeserializationStream extends Closeable { + /** The most general-purpose method to read an object. */ + def readObject[T: ClassTag](): T + /** Reads the object representing the key of a key-value pair. */ + def readKey[T: ClassTag](): T = readObject[T]() + /** Reads the object representing the value of a key-value pair. */ + def readValue[T: ClassTag](): T = readObject[T]() + override def close(): Unit + + /** + * Read the elements of this stream through an iterator. This can only be called once, as + * reading each element will consume data from the input source. + */ + def asIterator: Iterator[Any] = new NextIterator[Any] { + override protected def getNext() = { + try { + readObject[Any]() + } catch { + case eof: EOFException => + finished = true + null + } + } + + override protected def close() { + DeserializationStream.this.close() + } + } + + /** + * Read the elements of this stream through an iterator over key-value pairs. This can only be + * called once, as reading each element will consume data from the input source. + */ + def asKeyValueIterator: Iterator[(Any, Any)] = new NextIterator[(Any, Any)] { + override protected def getNext() = { + try { + (readKey[Any](), readValue[Any]()) + } catch { + case eof: EOFException => + finished = true + null + } + } + + override protected def close() { + DeserializationStream.this.close() + } + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/util/BitSet.scala b/common/src/main/scala/com/bytedance/css/common/util/BitSet.scala new file mode 100644 index 0000000..95a0bb2 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/util/BitSet.scala @@ -0,0 +1,247 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.util + +import java.util.Arrays + +/** + * A simple, fixed-size bit set implementation. This implementation is fast because it avoids + * safety/bound checking. + */ +class BitSet(numBits: Int) extends Serializable { + + private val words = new Array[Long](bit2words(numBits)) + private val numWords = words.length + + /** + * Compute the capacity (number of bits) that can be represented + * by this bitset. + */ + def capacity: Int = numWords * 64 + + /** + * Clear all set bits. + */ + def clear(): Unit = Arrays.fill(words, 0) + + /** + * Set all the bits up to a given index + */ + def setUntil(bitIndex: Int): Unit = { + val wordIndex = bitIndex >> 6 // divide by 64 + Arrays.fill(words, 0, wordIndex, -1) + if(wordIndex < words.length) { + // Set the remaining bits (note that the mask could still be zero) + val mask = ~(-1L << (bitIndex & 0x3f)) + words(wordIndex) |= mask + } + } + + /** + * Clear all the bits up to a given index + */ + def clearUntil(bitIndex: Int): Unit = { + val wordIndex = bitIndex >> 6 // divide by 64 + Arrays.fill(words, 0, wordIndex, 0) + if(wordIndex < words.length) { + // Clear the remaining bits + val mask = -1L << (bitIndex & 0x3f) + words(wordIndex) &= mask + } + } + + /** + * Compute the bit-wise AND of the two sets returning the + * result. + */ + def &(other: BitSet): BitSet = { + val newBS = new BitSet(math.max(capacity, other.capacity)) + val smaller = math.min(numWords, other.numWords) + assert(newBS.numWords >= numWords) + assert(newBS.numWords >= other.numWords) + var ind = 0 + while( ind < smaller ) { + newBS.words(ind) = words(ind) & other.words(ind) + ind += 1 + } + newBS + } + + def and(other: BitSet): BitSet = { + this & other + } + + /** + * Compute the bit-wise OR of the two sets returning the + * result. + */ + def |(other: BitSet): BitSet = { + val newBS = new BitSet(math.max(capacity, other.capacity)) + assert(newBS.numWords >= numWords) + assert(newBS.numWords >= other.numWords) + val smaller = math.min(numWords, other.numWords) + var ind = 0 + while( ind < smaller ) { + newBS.words(ind) = words(ind) | other.words(ind) + ind += 1 + } + while( ind < numWords ) { + newBS.words(ind) = words(ind) + ind += 1 + } + while( ind < other.numWords ) { + newBS.words(ind) = other.words(ind) + ind += 1 + } + newBS + } + + /** + * Compute the symmetric difference by performing bit-wise XOR of the two sets returning the + * result. + */ + def ^(other: BitSet): BitSet = { + val newBS = new BitSet(math.max(capacity, other.capacity)) + val smaller = math.min(numWords, other.numWords) + var ind = 0 + while (ind < smaller) { + newBS.words(ind) = words(ind) ^ other.words(ind) + ind += 1 + } + if (ind < numWords) { + Array.copy( words, ind, newBS.words, ind, numWords - ind ) + } + if (ind < other.numWords) { + Array.copy( other.words, ind, newBS.words, ind, other.numWords - ind ) + } + newBS + } + + /** + * Compute the difference of the two sets by performing bit-wise AND-NOT returning the + * result. + */ + def andNot(other: BitSet): BitSet = { + val newBS = new BitSet(capacity) + val smaller = math.min(numWords, other.numWords) + var ind = 0 + while (ind < smaller) { + newBS.words(ind) = words(ind) & ~other.words(ind) + ind += 1 + } + if (ind < numWords) { + Array.copy( words, ind, newBS.words, ind, numWords - ind ) + } + newBS + } + + /** + * Sets the bit at the specified index to true. + * @param index the bit index + */ + def set(index: Int): Unit = { + val bitmask = 1L << (index & 0x3f) // mod 64 and shift + words(index >> 6) |= bitmask // div by 64 and mask + } + + def unset(index: Int): Unit = { + val bitmask = 1L << (index & 0x3f) // mod 64 and shift + words(index >> 6) &= ~bitmask // div by 64 and mask + } + + /** + * Return the value of the bit with the specified index. The value is true if the bit with + * the index is currently set in this BitSet; otherwise, the result is false. + * + * @param index the bit index + * @return the value of the bit with the specified index + */ + def get(index: Int): Boolean = { + val bitmask = 1L << (index & 0x3f) // mod 64 and shift + (words(index >> 6) & bitmask) != 0 // div by 64 and mask + } + + /** + * Get an iterator over the set bits. + */ + def iterator: Iterator[Int] = new Iterator[Int] { + var ind = nextSetBit(0) + override def hasNext: Boolean = ind >= 0 + override def next(): Int = { + val tmp = ind + ind = nextSetBit(ind + 1) + tmp + } + } + + + /** Return the number of bits set to true in this BitSet. */ + def cardinality(): Int = { + var sum = 0 + var i = 0 + while (i < numWords) { + sum += java.lang.Long.bitCount(words(i)) + i += 1 + } + sum + } + + /** + * Returns the index of the first bit that is set to true that occurs on or after the + * specified starting index. If no such bit exists then -1 is returned. + * + * To iterate over the true bits in a BitSet, use the following loop: + * + * for (int i = bs.nextSetBit(0); i >= 0; i = bs.nextSetBit(i+1)) { + * // operate on index i here + * } + * + * @param fromIndex the index to start checking from (inclusive) + * @return the index of the next set bit, or -1 if there is no such bit + */ + def nextSetBit(fromIndex: Int): Int = { + var wordIndex = fromIndex >> 6 + if (wordIndex >= numWords) { + return -1 + } + + // Try to find the next set bit in the current word + val subIndex = fromIndex & 0x3f + var word = words(wordIndex) >> subIndex + if (word != 0) { + return (wordIndex << 6) + subIndex + java.lang.Long.numberOfTrailingZeros(word) + } + + // Find the next set bit in the rest of the words + wordIndex += 1 + while (wordIndex < numWords) { + word = words(wordIndex) + if (word != 0) { + return (wordIndex << 6) + java.lang.Long.numberOfTrailingZeros(word) + } + wordIndex += 1 + } + + -1 + } + + /** Return the number of longs it would take to hold numBits. */ + private def bit2words(numBits: Int) = ((numBits - 1) >> 6) + 1 +} diff --git a/common/src/main/scala/com/bytedance/css/common/util/ByteBufferInputStream.scala b/common/src/main/scala/com/bytedance/css/common/util/ByteBufferInputStream.scala new file mode 100644 index 0000000..9dba69e --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/util/ByteBufferInputStream.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.util + +import java.io.InputStream +import java.nio.ByteBuffer + +/** + * Reads data from a ByteBuffer. + */ +class ByteBufferInputStream(private var buffer: ByteBuffer) + extends InputStream { + + override def read(): Int = { + if (buffer == null || buffer.remaining() == 0) { + cleanUp() + -1 + } else { + buffer.get() & 0xFF + } + } + + override def read(dest: Array[Byte]): Int = { + read(dest, 0, dest.length) + } + + override def read(dest: Array[Byte], offset: Int, length: Int): Int = { + if (buffer == null || buffer.remaining() == 0) { + cleanUp() + -1 + } else { + val amountToGet = math.min(buffer.remaining(), length) + buffer.get(dest, offset, amountToGet) + amountToGet + } + } + + override def skip(bytes: Long): Long = { + if (buffer != null) { + val amountToSkip = math.min(bytes, buffer.remaining).toInt + buffer.position(buffer.position() + amountToSkip) + if (buffer.remaining() == 0) { + cleanUp() + } + amountToSkip + } else { + 0L + } + } + + /** + * Clean up the buffer, and potentially dispose of it using StorageUtils.dispose(). + */ + private def cleanUp() { + if (buffer != null) { + buffer = null + } + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/util/ByteBufferOutputStream.scala b/common/src/main/scala/com/bytedance/css/common/util/ByteBufferOutputStream.scala new file mode 100644 index 0000000..54d7ea5 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/util/ByteBufferOutputStream.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.util + +import java.io.ByteArrayOutputStream +import java.nio.ByteBuffer + +/** + * Provide a zero-copy way to convert data in ByteArrayOutputStream to ByteBuffer + */ +class ByteBufferOutputStream(capacity: Int) extends ByteArrayOutputStream(capacity) { + + def this() = this(32) + + def getCount(): Int = count + + private[this] var closed: Boolean = false + + override def write(b: Int): Unit = { + require(!closed, "cannot write to a closed ByteBufferOutputStream") + super.write(b) + } + + override def write(b: Array[Byte], off: Int, len: Int): Unit = { + require(!closed, "cannot write to a closed ByteBufferOutputStream") + super.write(b, off, len) + } + + override def reset(): Unit = { + require(!closed, "cannot reset a closed ByteBufferOutputStream") + super.reset() + } + + override def close(): Unit = { + if (!closed) { + super.close() + closed = true + } + } + + def toByteBuffer: ByteBuffer = { + require(closed, "can only call toByteBuffer() after ByteBufferOutputStream has been closed") + ByteBuffer.wrap(buf, 0, count) + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/util/Collections.scala b/common/src/main/scala/com/bytedance/css/common/util/Collections.scala new file mode 100644 index 0000000..4365d91 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/util/Collections.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.util + +import java.util +import java.util.function + +private[css] class WrappedMap[K, V](map: util.Map[K, V]) { + /** + * Implicit function for computeIfAbsent + */ + def computeWhenAbsent(key: K, fun: K => V): V = { + map.computeIfAbsent(key, new function.Function[K, V] { + override def apply(t: K): V = { + fun(key) + } + }) + } +} + +object Collections { + implicit def wrapMap[K, V](map: util.Map[K, V]): WrappedMap[K, V] = { + new WrappedMap(map) + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/util/IntParam.scala b/common/src/main/scala/com/bytedance/css/common/util/IntParam.scala new file mode 100644 index 0000000..2d117a9 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/util/IntParam.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.util + +/** + * An extractor object for parsing strings into integers. + */ +object IntParam { + def unapply(str: String): Option[Int] = { + try { + Some(str.toInt) + } catch { + case e: NumberFormatException => None + } + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/util/MemoryParam.scala b/common/src/main/scala/com/bytedance/css/common/util/MemoryParam.scala new file mode 100644 index 0000000..da7c009 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/util/MemoryParam.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.util + +/** + * An extractor object for parsing JVM memory strings, such as "10g", into an Int representing + * the number of megabytes. Supports the same formats as Utils.memoryStringToMb. + */ +object MemoryParam { + def unapply(str: String): Option[Long] = { + try { + Some(Utils.byteStringAsBytes(str)) + } catch { + case e: NumberFormatException => None + } + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/util/NextIterator.scala b/common/src/main/scala/com/bytedance/css/common/util/NextIterator.scala new file mode 100644 index 0000000..e110f00 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/util/NextIterator.scala @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.util + +/** Provides a basic/boilerplate Iterator implementation. */ +abstract class NextIterator[U] extends Iterator[U] { + + private var gotNext = false + private var nextValue: U = _ + private var closed = false + protected var finished = false + + /** + * Method for subclasses to implement to provide the next element. + * + * If no next element is available, the subclass should set `finished` + * to `true` and may return any value (it will be ignored). + * + * This convention is required because `null` may be a valid value, + * and using `Option` seems like it might create unnecessary Some/None + * instances, given some iterators might be called in a tight loop. + * + * @return U, or set 'finished' when done + */ + protected def getNext(): U + + /** + * Method for subclasses to implement when all elements have been successfully + * iterated, and the iteration is done. + * + * Note: `NextIterator` cannot guarantee that `close` will be + * called because it has no control over what happens when an exception + * happens in the user code that is calling hasNext/next. + * + * Ideally you should have another try/catch, as in HadoopRDD, that + * ensures any resources are closed should iteration fail. + */ + protected def close() + + /** + * Calls the subclass-defined close method, but only once. + * + * Usually calling `close` multiple times should be fine, but historically + * there have been issues with some InputFormats throwing exceptions. + */ + def closeIfNeeded() { + if (!closed) { + // Note: it's important that we set closed = true before calling close(), since setting it + // afterwards would permit us to call close() multiple times if close() threw an exception. + closed = true + close() + } + } + + override def hasNext: Boolean = { + if (!finished) { + if (!gotNext) { + nextValue = getNext() + if (finished) { + closeIfNeeded() + } + gotNext = true + } + } + !finished + } + + override def next(): U = { + if (!hasNext) { + throw new NoSuchElementException("End of stream") + } + gotNext = false + nextValue + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/util/RpcUtils.scala b/common/src/main/scala/com/bytedance/css/common/util/RpcUtils.scala new file mode 100644 index 0000000..7de7f0e --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/util/RpcUtils.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.util + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.rpc.RpcTimeout + +object RpcUtils { + + /** Returns the configured number of times to retry connecting */ + def numRetries(conf: CssConf): Int = { + conf.getInt("css.rpc.numRetries", 3) + } + + /** Returns the configured number of milliseconds to wait on each retry */ + def retryWaitMs(conf: CssConf): Long = { + conf.getTimeAsMs("css.rpc.retry.wait", "3s") + } + + /** Returns the default Spark timeout to use for RPC ask operations. */ + def askRpcTimeout(conf: CssConf): RpcTimeout = { + RpcTimeout(conf, Seq("css.rpc.askTimeout", "css.network.timeout"), "240s") + } + + /** Returns the default Spark timeout to use for RPC remote endpoint lookup. */ + def lookupRpcTimeout(conf: CssConf): RpcTimeout = { + RpcTimeout(conf, Seq("css.rpc.lookupTimeout", "css.network.timeout"), "240s") + } + + private val MAX_MESSAGE_SIZE_IN_MB = Int.MaxValue / 1024 / 1024 + + /** Returns the configured max message size for messages in bytes. */ + def maxMessageSizeBytes(conf: CssConf): Int = { + val maxSizeInMB = conf.getInt("css.rpc.message.maxSize", 128) + if (maxSizeInMB > MAX_MESSAGE_SIZE_IN_MB) { + throw new IllegalArgumentException( + s"css.rpc.message.maxSize should not be greater than $MAX_MESSAGE_SIZE_IN_MB MB") + } + maxSizeInMB * 1024 * 1024 + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/util/ThreadUtils.scala b/common/src/main/scala/com/bytedance/css/common/util/ThreadUtils.scala new file mode 100644 index 0000000..fc7c8a5 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/util/ThreadUtils.scala @@ -0,0 +1,296 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.util + +import java.util.concurrent._ + +import scala.collection.TraversableLike +import scala.collection.generic.CanBuildFrom +import scala.concurrent.{Awaitable, ExecutionContext, ExecutionContextExecutor, Future} +import scala.concurrent.duration.{Duration, FiniteDuration} +import scala.concurrent.forkjoin.{ForkJoinPool => SForkJoinPool, ForkJoinWorkerThread => SForkJoinWorkerThread} +import scala.language.higherKinds +import scala.util.control.NonFatal + +import com.bytedance.css.common.exception.CssException +import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} + +object ThreadUtils { + + private val sameThreadExecutionContext = + ExecutionContext.fromExecutorService(MoreExecutors.sameThreadExecutor()) + + /** + * An `ExecutionContextExecutor` that runs each task in the thread that invokes `execute/submit`. + * The caller should make sure the tasks running in this `ExecutionContextExecutor` are short and + * never block. + */ + def sameThread: ExecutionContextExecutor = sameThreadExecutionContext + + /** + * Create a thread factory that names threads with a prefix and also sets the threads to daemon. + */ + def namedThreadFactory(prefix: String): ThreadFactory = { + new ThreadFactoryBuilder().setDaemon(true).setNameFormat(prefix + "-%d").build() + } + + /** + * Wrapper over newCachedThreadPool. Thread names are formatted as prefix-ID, where ID is a + * unique, sequentially assigned integer. + */ + def newDaemonCachedThreadPool(prefix: String): ThreadPoolExecutor = { + val threadFactory = namedThreadFactory(prefix) + Executors.newCachedThreadPool(threadFactory).asInstanceOf[ThreadPoolExecutor] + } + + /** + * Create a cached thread pool whose max number of threads is `maxThreadNumber`. Thread names + * are formatted as prefix-ID, where ID is a unique, sequentially assigned integer. + */ + def newDaemonCachedThreadPool( + prefix: String, maxThreadNumber: Int, keepAliveSeconds: Int = 60): ThreadPoolExecutor = { + val threadFactory = namedThreadFactory(prefix) + val threadPool = new ThreadPoolExecutor( + maxThreadNumber, // corePoolSize: the max number of threads to create before queuing the tasks + maxThreadNumber, // maximumPoolSize: because we use LinkedBlockingDeque, this one is not used + keepAliveSeconds, + TimeUnit.SECONDS, + new LinkedBlockingQueue[Runnable], + threadFactory) + threadPool.allowCoreThreadTimeOut(true) + threadPool + } + + /** + * Wrapper over newFixedThreadPool. Thread names are formatted as prefix-ID, where ID is a + * unique, sequentially assigned integer. + */ + def newDaemonFixedThreadPool(nThreads: Int, prefix: String): ThreadPoolExecutor = { + val threadFactory = namedThreadFactory(prefix) + Executors.newFixedThreadPool(nThreads, threadFactory).asInstanceOf[ThreadPoolExecutor] + } + + /** + * Wrapper over newSingleThreadExecutor. + */ + def newDaemonSingleThreadExecutor(threadName: String): ExecutorService = { + val threadFactory = new ThreadFactoryBuilder().setDaemon(true).setNameFormat(threadName).build() + Executors.newSingleThreadExecutor(threadFactory) + } + + /** + * Wrapper over ScheduledThreadPoolExecutor. + */ + def newDaemonSingleThreadScheduledExecutor(threadName: String): ScheduledExecutorService = { + val threadFactory = new ThreadFactoryBuilder().setDaemon(true).setNameFormat(threadName).build() + val executor = new ScheduledThreadPoolExecutor(1, threadFactory) + // By default, a cancelled task is not automatically removed from the work queue until its delay + // elapses. We have to enable it manually. + executor.setRemoveOnCancelPolicy(true) + executor + } + + /** + * Wrapper over ScheduledThreadPoolExecutor. + */ + def newDaemonThreadPoolScheduledExecutor(threadNamePrefix: String, numThreads: Int) + : ScheduledExecutorService = { + val threadFactory = new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat(s"$threadNamePrefix-%d") + .build() + val executor = new ScheduledThreadPoolExecutor(numThreads, threadFactory) + // By default, a cancelled task is not automatically removed from the work queue until its delay + // elapses. We have to enable it manually. + executor.setRemoveOnCancelPolicy(true) + executor + } + + /** + * Run a piece of code in a new thread and return the result. Exception in the new thread is + * thrown in the caller thread with an adjusted stack trace that removes references to this + * method for clarity. The exception stack traces will be like the following + * + * SomeException: exception-message + * at CallerClass.body-method (sourcefile.scala) + * at ... run in separate thread using org.apache.spark.util.ThreadUtils ... () + * at CallerClass.caller-method (sourcefile.scala) + * ... + */ + def runInNewThread[T]( + threadName: String, + isDaemon: Boolean = true)(body: => T): T = { + @volatile var exception: Option[Throwable] = None + @volatile var result: T = null.asInstanceOf[T] + + val thread = new Thread(threadName) { + override def run(): Unit = { + try { + result = body + } catch { + case NonFatal(e) => + exception = Some(e) + } + } + } + thread.setDaemon(isDaemon) + thread.start() + thread.join() + + exception match { + case Some(realException) => + // Remove the part of the stack that shows method calls into this helper method + // This means drop everything from the top until the stack element + // ThreadUtils.runInNewThread(), and then drop that as well (hence the `drop(1)`). + val baseStackTrace = Thread.currentThread().getStackTrace().dropWhile( + ! _.getClassName.contains(this.getClass.getSimpleName)).drop(1) + + // Remove the part of the new thread stack that shows methods call from this helper method + val extraStackTrace = realException.getStackTrace.takeWhile( + ! _.getClassName.contains(this.getClass.getSimpleName)) + + // Combine the two stack traces, with a place holder just specifying that there + // was a helper method used, without any further details of the helper + val placeHolderStackElem = new StackTraceElement( + s"... run in separate thread using ${ThreadUtils.getClass.getName.stripSuffix("$")} ..", + " ", "", -1) + val finalStackTrace = extraStackTrace ++ Seq(placeHolderStackElem) ++ baseStackTrace + + // Update the stack trace and rethrow the exception in the caller thread + realException.setStackTrace(finalStackTrace) + throw realException + case None => + result + } + } + + /** + * Construct a new Scala ForkJoinPool with a specified max parallelism and name prefix. + */ + def newForkJoinPool(prefix: String, maxThreadNumber: Int): SForkJoinPool = { + // Custom factory to set thread names + val factory = new SForkJoinPool.ForkJoinWorkerThreadFactory { + override def newThread(pool: SForkJoinPool) = + new SForkJoinWorkerThread(pool) { + setName(prefix + "-" + super.getName) + } + } + new SForkJoinPool(maxThreadNumber, factory, + null, // handler + false // asyncMode + ) + } + + // scalastyle:off awaitresult + /** + * Preferred alternative to `Await.result()`. + * + * This method wraps and re-throws any exceptions thrown by the underlying `Await` call, ensuring + * that this thread's stack trace appears in logs. + * + * In addition, it calls `Awaitable.result` directly to avoid using `ForkJoinPool`'s + * `BlockingContext`. Codes running in the user's thread may be in a thread of Scala ForkJoinPool. + * As concurrent executions in ForkJoinPool may see some [[ThreadLocal]] value unexpectedly, this + * method basically prevents ForkJoinPool from running other tasks in the current waiting thread. + * In general, we should use this method because many places in Spark use [[ThreadLocal]] and it's + * hard to debug when [[ThreadLocal]]s leak to other tasks. + */ + @throws(classOf[CssException]) + def awaitResult[T](awaitable: Awaitable[T], atMost: Duration): T = { + try { + // `awaitPermission` is not actually used anywhere so it's safe to pass in null here. + // See SPARK-13747. + val awaitPermission = null.asInstanceOf[scala.concurrent.CanAwait] + awaitable.result(atMost)(awaitPermission) + } catch { + // TimeoutException is thrown in the current thread, so not need to warp the exception. + case NonFatal(t) if !t.isInstanceOf[TimeoutException] => + throw new CssException("Exception thrown in awaitResult: ", t) + case e: Throwable => + throw e + } + } + // scalastyle:on awaitresult + + // scalastyle:off awaitready + /** + * Preferred alternative to `Await.ready()`. + * + * @see [[awaitResult]] + */ + @throws(classOf[CssException]) + def awaitReady[T](awaitable: Awaitable[T], atMost: Duration): awaitable.type = { + try { + // `awaitPermission` is not actually used anywhere so it's safe to pass in null here. + // See SPARK-13747. + val awaitPermission = null.asInstanceOf[scala.concurrent.CanAwait] + awaitable.ready(atMost)(awaitPermission) + } catch { + // TimeoutException is thrown in the current thread, so not need to warp the exception. + case NonFatal(t) if !t.isInstanceOf[TimeoutException] => + throw new CssException("Exception thrown in awaitResult: ", t) + } + } + // scalastyle:on awaitready + + def shutdown( + executor: ExecutorService, + gracePeriod: Duration = FiniteDuration(30, TimeUnit.SECONDS)): Unit = { + executor.shutdown() + executor.awaitTermination(gracePeriod.toMillis, TimeUnit.MILLISECONDS) + if (!executor.isShutdown) { + executor.shutdownNow() + } + } + + /** + * Transforms input collection by applying the given function to each element in parallel fashion. + * Comparing to the map() method of Scala parallel collections, this method can be interrupted + * at any time. This is useful on canceling of task execution, for example. + * + * @param in - the input collection which should be transformed in parallel. + * @param prefix - the prefix assigned to the underlying thread pool. + * @param maxThreads - maximum number of thread can be created during execution. + * @param f - the lambda function will be applied to each element of `in`. + * @tparam I - the type of elements in the input collection. + * @tparam O - the type of elements in resulted collection. + * @return new collection in which each element was given from the input collection `in` by + * applying the lambda function `f`. + */ + def parmap[I, O, Col[X] <: TraversableLike[X, Col[X]]] + (in: Col[I], prefix: String, maxThreads: Int) + (f: I => O) + (implicit + cbf: CanBuildFrom[Col[I], Future[O], Col[Future[O]]], // For in.map + cbf2: CanBuildFrom[Col[Future[O]], O, Col[O]] // for Future.sequence + ): Col[O] = { + val pool = newForkJoinPool(prefix, maxThreads) + try { + implicit val ec = ExecutionContext.fromExecutor(pool) + + val futures = in.map(x => Future(f(x))) + val futureSeq = Future.sequence(futures) + + awaitResult(futureSeq, Duration.Inf) + } finally { + pool.shutdownNow() + } + } +} diff --git a/common/src/main/scala/com/bytedance/css/common/util/Utils.scala b/common/src/main/scala/com/bytedance/css/common/util/Utils.scala new file mode 100644 index 0000000..da1f6f6 --- /dev/null +++ b/common/src/main/scala/com/bytedance/css/common/util/Utils.scala @@ -0,0 +1,536 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.util + +import java.io.{File, FileInputStream, InputStreamReader, IOException} +import java.math.{MathContext, RoundingMode} +import java.net.{BindException, Inet4Address, InetAddress, NetworkInterface, URI} +import java.nio.charset.StandardCharsets +import java.util.{Locale, Properties} + +import scala.collection.JavaConverters._ +import scala.collection.Map +import scala.util.Try +import scala.util.control.{ControlThrowable, NonFatal} + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.exception.CssException +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.protocol.PartitionInfo +import com.bytedance.css.network.util.{ConfigProvider, JavaUtils, TransportConf} +import com.google.common.net.InetAddresses +import io.netty.channel.unix.Errors.NativeIoException +import org.apache.commons.lang3.SystemUtils + +object Utils extends Logging { + def checkCssConfLegality(conf: CssConf): Boolean = { + CssConf.minDiskBaseDirNum(conf) < 0 || CssConf.minDiskBaseDirNum(conf) < CssConf.diskFlusherBaseDirs(conf).length + } + + def stringToSeq(str: String): Seq[String] = { + str.split(",").map(_.trim()).filter(_.nonEmpty) + } + + def getSystemProperties: Map[String, String] = { + System.getProperties.stringPropertyNames().asScala + .map(key => (key, System.getProperty(key))).toMap + } + + def timeStringAsMs(str: String): Long = { + JavaUtils.timeStringAsMs(str) + } + + def timeStringAsSeconds(str: String): Long = { + JavaUtils.timeStringAsSec(str) + } + + def byteStringAsBytes(str: String): Long = { + JavaUtils.byteStringAsBytes(str) + } + + def byteStringAsKb(str: String): Long = { + JavaUtils.byteStringAsKb(str) + } + + def byteStringAsMb(str: String): Long = { + JavaUtils.byteStringAsMb(str) + } + + def byteStringAsGb(str: String): Long = { + JavaUtils.byteStringAsGb(str) + } + + def memoryStringToMb(str: String): Int = { + // Convert to bytes, rather than directly to MB, because when no units are specified the unit + // is assumed to be bytes + (JavaUtils.byteStringAsBytes(str) / 1024 / 1024).toInt + } + + def bytesToString(size: Long): String = bytesToString(BigInt(size)) + + def bytesToString(size: BigInt): String = { + val EB = 1L << 60 + val PB = 1L << 50 + val TB = 1L << 40 + val GB = 1L << 30 + val MB = 1L << 20 + val KB = 1L << 10 + + if (size >= BigInt(1L << 11) * EB) { + // The number is too large, show it in scientific notation. + BigDecimal(size, new MathContext(3, RoundingMode.HALF_UP)).toString() + " B" + } else { + val (value, unit) = { + if (size >= 2 * EB) { + (BigDecimal(size) / EB, "EB") + } else if (size >= 2 * PB) { + (BigDecimal(size) / PB, "PB") + } else if (size >= 2 * TB) { + (BigDecimal(size) / TB, "TB") + } else if (size >= 2 * GB) { + (BigDecimal(size) / GB, "GB") + } else if (size >= 2 * MB) { + (BigDecimal(size) / MB, "MB") + } else if (size >= 2 * KB) { + (BigDecimal(size) / KB, "KB") + } else { + (BigDecimal(size), "B") + } + } + "%.1f %s".formatLocal(Locale.US, value, unit) + } + } + + def megabytesToString(megabytes: Long): String = { + bytesToString(megabytes * 1024L * 1024L) + } + + def getClassLoader: ClassLoader = getClass.getClassLoader + + def getContextOrClassLoader: ClassLoader = + Option(Thread.currentThread().getContextClassLoader).getOrElse(getClassLoader) + + def classIsLoadable(clazz: String): Boolean = { + // scalastyle:off classforname + Try { + Class.forName(clazz, false, getContextOrClassLoader) + }.isSuccess + // scalastyle:on classforname + } + + // scalastyle:off classforname + /** Preferred alternative to Class.forName(className) */ + def classForName(className: String): Class[_] = { + Class.forName(className, true, getContextOrClassLoader) + // scalastyle:on classforname + } + + def tryOrIOException[T](block: => T): T = { + try { + block + } catch { + case e: IOException => + logError("Exception encountered", e) + throw e + case NonFatal(e) => + logError("Exception encountered", e) + throw new IOException(e) + } + } + + @throws(classOf[CssException]) + def extractHostPortFromCssUrl(cssUrl: String): (String, Int) = { + try { + val uri = new java.net.URI(cssUrl) + val host = uri.getHost + val port = uri.getPort + if (uri.getScheme != "css" || + host == null || + port < 0 || + (uri.getPath != null && !uri.getPath.isEmpty) || // uri.getPath returns "" instead of null + uri.getFragment != null || + uri.getQuery != null || + uri.getUserInfo != null) { + throw new CssException("Invalid master URL: " + cssUrl) + } + (host, port) + } catch { + case e: java.net.URISyntaxException => + throw new CssException("Invalid master URL: " + cssUrl, e) + } + } + + def isLocalBlockFetchable(enabled: Boolean, fileName: String): Boolean = { + enabled && new File(fileName).exists() && new File(fileName).canRead + } + + def encodeFileNameToURIRawPath(fileName: String): String = { + require(!fileName.contains("/") && !fileName.contains("\\")) + // `file` and `localhost` are not used. Just to prevent URI from parsing `fileName` as + // scheme or host. The prefix "/" is required because URI doesn't accept a relative path. + // We should remove it after we get the raw path. + new URI("file", null, "localhost", -1, "/" + fileName, null, null).getRawPath.substring(1) + } + + def tryWithSafeFinallyAndFailureCallbacks[T](block: => T) + (catchBlock: => Unit = (), finallyBlock: => Unit = ()): T = { + var originalThrowable: Throwable = null + try { + block + } catch { + case cause: Throwable => + // Purposefully not using NonFatal, because even fatal exceptions + // we don't want to have our finallyBlock suppress + originalThrowable = cause + try { + logError("Aborting task", originalThrowable) +// TaskContext.get().markTaskFailed(originalThrowable) + catchBlock + } catch { + case t: Throwable => + if (originalThrowable != t) { + originalThrowable.addSuppressed(t) + logWarning(s"Suppressing exception in catch: ${t.getMessage}", t) + } + } + throw originalThrowable + } finally { + try { + finallyBlock + } catch { + case t: Throwable if (originalThrowable != null && originalThrowable != t) => + originalThrowable.addSuppressed(t) + logWarning(s"Suppressing exception in finally: ${t.getMessage}", t) + throw originalThrowable + } + } + } + + def portMaxRetries(conf: CssConf): Int = { + val maxRetries = conf.getOption("css.port.maxRetries").map(_.toInt) + if (conf.contains("css.testing")) { + // Set a higher number of retries for tests... + maxRetries.getOrElse(100) + } else { + maxRetries.getOrElse(16) + } + } + + def userPort(base: Int, offset: Int): Int = { + (base + offset - 1024) % (65536 - 1024) + 1024 + } + + def isBindCollision(exception: Throwable): Boolean = { + exception match { + case e: BindException => + if (e.getMessage != null) { + return true + } + isBindCollision(e.getCause) + case e: NativeIoException => + (e.getMessage != null && e.getMessage.startsWith("bind() failed: ")) || + isBindCollision(e.getCause) + case e: Exception => isBindCollision(e.getCause) + case _ => false + } + } + + def startServiceOnPort[T]( + startPort: Int, + startService: Int => (T, Int), + conf: CssConf, + serviceName: String = ""): (T, Int) = { + + require(startPort == 0 || (1024 <= startPort && startPort < 65536), + "startPort should be between 1024 and 65535 (inclusive), or 0 for a random free port.") + + val serviceString = if (serviceName.isEmpty) "" else s" '$serviceName'" + val maxRetries = portMaxRetries(conf) + for (offset <- 0 to maxRetries) { + // Do not increment port if startPort is 0, which is treated as a special port + val tryPort = if (startPort == 0) { + startPort + } else { + userPort(startPort, offset) + } + try { + val (service, port) = startService(tryPort) + logInfo(s"Successfully started service$serviceString on port $port.") + return (service, port) + } catch { + case e: Exception if isBindCollision(e) => + if (offset >= maxRetries) { + val exceptionMessage = if (startPort == 0) { + s"${e.getMessage}: Service$serviceString failed after " + + s"$maxRetries retries (on a random free port)! " + + s"Consider explicitly setting the appropriate binding address for " + + s"the service$serviceString (for example spark.driver.bindAddress " + + s"for SparkDriver) to the correct binding address." + } else { + s"${e.getMessage}: Service$serviceString failed after " + + s"$maxRetries retries (starting from $startPort)! Consider explicitly setting " + + s"the appropriate port for the service$serviceString (for example spark.ui.port " + + s"for SparkUI) to an available port or increasing spark.port.maxRetries." + } + val exception = new BindException(exceptionMessage) + // restore original stack trace + exception.setStackTrace(e.getStackTrace) + throw exception + } + if (startPort == 0) { + // As startPort 0 is for a random free port, it is most possibly binding address is + // not correct. + logWarning(s"Service$serviceString could not bind on a random free port. " + + "You may check whether configuring an appropriate binding address.") + } else { + logWarning(s"Service$serviceString could not bind on port $tryPort. " + + s"Attempting port ${tryPort + 1}.") + } + } + } + // Should never happen + throw new CssException(s"Failed to start service$serviceString on port $startPort") + } + + private val MAX_DEFAULT_NETTY_THREADS = 64 + + private def defaultNumThreads(numUsableCores: Int): Int = { + val availableCores = + if (numUsableCores > 0) numUsableCores else Runtime.getRuntime.availableProcessors() + availableCores + } + + def fromCssConf(_conf: CssConf, module: String, numUsableCores: Int = 0): TransportConf = { + val conf = _conf.clone + + // Specify thread configuration based on our JVM's allocation of cores (rather than necessarily + // assuming we have all the machine's cores). + // NB: Only set if serverThreads/clientThreads not already set. + val numThreads = defaultNumThreads(numUsableCores) + conf.setIfMissing(s"css.$module.io.serverThreads", numThreads.toString) + conf.setIfMissing(s"css.$module.io.clientThreads", numThreads.toString) + + new TransportConf(module, new ConfigProvider { + override def get(name: String): String = conf.get(name) + + override def get(name: String, defaultValue: String): String = conf.get(name, defaultValue) + + override def getAll(): java.lang.Iterable[java.util.Map.Entry[String, String]] = { + conf.getAll.toMap.asJava.entrySet() + } + }) + } + + val isWindows = SystemUtils.IS_OS_WINDOWS + + val isMac = SystemUtils.IS_OS_MAC_OSX + + /** + * Get the local host's IP address in dotted-quad format (e.g. 1.2.3.4). + * Note, this is typically not used from within core spark. + */ + private lazy val localIpAddress: InetAddress = findLocalInetAddress() + + private def findLocalInetAddress(): InetAddress = { + val defaultIpOverride = System.getenv("SPARK_LOCAL_IP") + if (defaultIpOverride != null) { + InetAddress.getByName(defaultIpOverride) + } else { + val address = InetAddress.getLocalHost + if (address.isLoopbackAddress) { + // Address resolves to something like 127.0.1.1, which happens on Debian; try to find + // a better address using the local network interfaces + // getNetworkInterfaces returns ifs in reverse order compared to ifconfig output order + // on unix-like system. On windows, it returns in index order. + // It's more proper to pick ip address following system output order. + val activeNetworkIFs = NetworkInterface.getNetworkInterfaces.asScala.toSeq + val reOrderedNetworkIFs = if (isWindows) activeNetworkIFs else activeNetworkIFs.reverse + + for (ni <- reOrderedNetworkIFs) { + val addresses = ni.getInetAddresses.asScala + .filterNot(addr => addr.isLinkLocalAddress || addr.isLoopbackAddress).toSeq + if (addresses.nonEmpty) { + val addr = addresses.find(_.isInstanceOf[Inet4Address]).getOrElse(addresses.head) + // because of Inet6Address.toHostName may add interface at the end if it knows about it + val strippedAddress = InetAddress.getByAddress(addr.getAddress) + // We've found an address that looks reasonable! + logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" + + " a loopback address: " + address.getHostAddress + "; using " + + strippedAddress.getHostAddress + " instead (on interface " + ni.getName + ")") + logWarning("Set SPARK_LOCAL_IP if you need to bind to another address") + return strippedAddress + } + } + logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" + + " a loopback address: " + address.getHostAddress + ", but we couldn't find any" + + " external IP address!") + logWarning("Set SPARK_LOCAL_IP if you need to bind to another address") + } + address + } + } + + + private var customHostname: Option[String] = sys.env.get("SPARK_LOCAL_HOSTNAME") + + /** + * Allow setting a custom host name because when we run on Mesos we need to use the same + * hostname it reports to the master. + */ + def setCustomHostname(hostname: String) { + // DEBUG code + Utils.checkHost(hostname) + customHostname = Some(hostname) + } + + /** + * Get the local machine's FQDN. + */ + def localCanonicalHostName(): String = { + customHostname.getOrElse(localIpAddress.getCanonicalHostName) + } + + /** + * Get the local machine's hostname. + */ + def localHostName(): String = { + customHostname.getOrElse(localIpAddress.getHostAddress) + } + + /** + * Get the local machine's URI. + */ + def localHostNameForURI(): String = { + customHostname.getOrElse(InetAddresses.toUriString(localIpAddress)) + } + + def checkHost(host: String) { + assert(host != null && host.indexOf(':') == -1, s"Expected hostname (not IP) but got $host") + } + + def checkHostPort(hostPort: String) { + assert(hostPort != null && hostPort.indexOf(':') != -1, + s"Expected host and port but got $hostPort") + } + + def getDefaultPropertiesFile(env: Map[String, String] = sys.env): String = { + env.get("CSS_CONF_DIR") + .orElse(env.get("CSS_HOME").map { t => s"$t${File.separator}conf" }) + .map { t => new File(s"$t${File.separator}css-defaults.conf") } + .filter(_.isFile) + .map(_.getAbsolutePath) + .orNull + } + + def trimExceptCRLF(str: String): String = { + val nonSpaceOrNaturalLineDelimiter: Char => Boolean = { ch => + ch > ' ' || ch == '\r' || ch == '\n' + } + + val firstPos = str.indexWhere(nonSpaceOrNaturalLineDelimiter) + val lastPos = str.lastIndexWhere(nonSpaceOrNaturalLineDelimiter) + if (firstPos >= 0 && lastPos >= 0) { + str.substring(firstPos, lastPos + 1) + } else { + "" + } + } + + def getPropertiesFromFile(filename: String): Map[String, String] = { + val file = new File(filename) + require(file.exists(), s"Properties file $file does not exist") + require(file.isFile(), s"Properties file $file is not a normal file") + + val inReader = new InputStreamReader(new FileInputStream(file), StandardCharsets.UTF_8) + try { + val properties = new Properties() + properties.load(inReader) + properties.stringPropertyNames().asScala + .map { k => (k, trimExceptCRLF(properties.getProperty(k))) } + .toMap + + } catch { + case e: IOException => + throw new CssException(s"Failed when loading CSS properties from $filename", e) + } finally { + inReader.close() + } + } + + def loadDefaultCssProperties(conf: CssConf, filePath: String = null): String = { + val path = Option(filePath).getOrElse(getDefaultPropertiesFile()) + Option(path).foreach { confFile => + getPropertiesFromFile(confFile).filter { case (k, v) => + k.startsWith("css.") + }.foreach { case (k, v) => + conf.setIfMissing(k, v) + sys.props.getOrElseUpdate(k, v) + } + } + path + } + + def tryLogNonFatalError(block: => Unit) { + try { + block + } catch { + case NonFatal(t) => + logError(s"Uncaught exception in thread ${Thread.currentThread().getName}", t) + } + } + + /** + * Execute the given block, logging and re-throwing any uncaught exception. + * This is particularly useful for wrapping code that runs in a thread, to ensure + * that exceptions are printed, and to avoid having to catch Throwable. + */ + def logUncaughtExceptions[T](f: => T): T = { + try { + f + } catch { + case ct: ControlThrowable => + throw ct + case t: Throwable => + logError(s"Uncaught exception in thread ${Thread.currentThread().getName}", t) + throw t + } + } + + def getShuffleKey(applicationId: String, shuffleId: Int): String = { + s"$applicationId-$shuffleId" + } + + def getPartitionGroupKey(applicationId: String, shuffleId: Int, groupId: Int): String = { + s"$applicationId-$shuffleId-$groupId" + } + + def getEpochKeyWithShuffleKey(shuffleKey: String, reduceId: Int, epochId: Int): String = { + s"$shuffleKey-$reduceId-$epochId" + } + + def getMapperKey(shuffleId: Int, mapId: Int, attemptId: Int): String = { + s"$shuffleId-$mapId-$attemptId" + } + + def toPartitionInfo(reducerId: Int, epochId: Int): PartitionInfo = { + new PartitionInfo(reducerId, epochId) + } + +} diff --git a/common/src/main/scala/org/apache/spark/shuffle/css/ShuffleReadMetricsAdapter.scala b/common/src/main/scala/org/apache/spark/shuffle/css/ShuffleReadMetricsAdapter.scala new file mode 100644 index 0000000..96a9c5b --- /dev/null +++ b/common/src/main/scala/org/apache/spark/shuffle/css/ShuffleReadMetricsAdapter.scala @@ -0,0 +1,26 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.shuffle.css + +trait ShuffleReadMetricsAdapter { + def incRemoteBytesRead(v: Long): Unit + def incFetchWaitTime(v: Long): Unit + def incRecordsRead(v: Long): Unit +} diff --git a/common/src/main/scala/org/apache/spark/shuffle/css/ShuffleWriteMetricsAdapter.scala b/common/src/main/scala/org/apache/spark/shuffle/css/ShuffleWriteMetricsAdapter.scala new file mode 100644 index 0000000..f40af2d --- /dev/null +++ b/common/src/main/scala/org/apache/spark/shuffle/css/ShuffleWriteMetricsAdapter.scala @@ -0,0 +1,26 @@ +/* + * Copyright 2022 Bytedance Inc. + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.shuffle.css + +trait ShuffleWriteMetricsAdapter { + def incBytesWritten(v: Long): Unit + def incRecordsWritten(v: Long): Unit + def incWriteTime(v: Long): Unit +} diff --git a/common/src/test/resources/log4j.properties b/common/src/test/resources/log4j.properties new file mode 100644 index 0000000..3c1122b --- /dev/null +++ b/common/src/test/resources/log4j.properties @@ -0,0 +1,29 @@ +# +# Copyright 2022 Bytedance Inc. +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=DEBUG, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Silence verbose logs from 3rd-party libraries. +log4j.logger.io.netty=INFO diff --git a/common/src/test/scala/com/bytedance/css/common.rpc/RpcAddressSuite.scala b/common/src/test/scala/com/bytedance/css/common.rpc/RpcAddressSuite.scala new file mode 100644 index 0000000..f1d618b --- /dev/null +++ b/common/src/test/scala/com/bytedance/css/common.rpc/RpcAddressSuite.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc + +import com.bytedance.css.common.exception.CssException +import org.scalatest.FunSuite + +class RpcAddressSuite extends FunSuite { + + test("hostPort") { + val address = RpcAddress("1.2.3.4", 1234) + assert(address.host == "1.2.3.4") + assert(address.port == 1234) + assert(address.hostPort == "1.2.3.4:1234") + } + + test("fromCssURL") { + val address = RpcAddress.fromCssURL("css://1.2.3.4:1234") + assert(address.host == "1.2.3.4") + assert(address.port == 1234) + } + + test("fromCssURL: a typo url") { + val e = intercept[CssException] { + RpcAddress.fromCssURL("css://1.2. 3.4:1234") + } + assert("Invalid master URL: css://1.2. 3.4:1234" === e.getMessage) + } + + test("fromCssURL: invalid scheme") { + val e = intercept[CssException] { + RpcAddress.fromCssURL("invalid://1.2.3.4:1234") + } + assert("Invalid master URL: invalid://1.2.3.4:1234" === e.getMessage) + } + + test("toCssURL") { + val address = RpcAddress("1.2.3.4", 1234) + assert(address.toCssURL == "css://1.2.3.4:1234") + } +} diff --git a/common/src/test/scala/com/bytedance/css/common.rpc/RpcEnvSuite.scala b/common/src/test/scala/com/bytedance/css/common.rpc/RpcEnvSuite.scala new file mode 100644 index 0000000..21c22a4 --- /dev/null +++ b/common/src/test/scala/com/bytedance/css/common.rpc/RpcEnvSuite.scala @@ -0,0 +1,852 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc + +import java.io.NotSerializableException +import java.util.concurrent.{ConcurrentLinkedQueue, CountDownLatch, TimeUnit} + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.concurrent.Await +import scala.concurrent.duration._ + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.exception.CssException +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.util.ThreadUtils +import org.mockito.Matchers.any +import org.mockito.Mockito.{mock, never, verify} +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite +import org.scalatest.concurrent.Eventually.eventually +import org.scalatest.concurrent.Waiters.{interval, timeout} + +/** + * Common tests for an RpcEnv implementation. + */ +abstract class RpcEnvSuite extends FunSuite with Logging with BeforeAndAfterAll { + + var env: RpcEnv = _ + + override def beforeAll(): Unit = { + super.beforeAll() + val conf = new CssConf() + env = createRpcEnv(conf, "local", 0) + } + + override def afterAll(): Unit = { + try { + if (env != null) { + env.shutdown() + } + } finally { + super.afterAll() + } + } + + def createRpcEnv(conf: CssConf, name: String, port: Int, clientMode: Boolean = false): RpcEnv + + test("send a message locally") { + @volatile var message: String = null + val rpcEndpointRef = env.setupEndpoint("send-locally", new RpcEndpoint { + override val rpcEnv = env + + override def receive = { + case msg: String => message = msg + } + }) + rpcEndpointRef.send("hello") + eventually(timeout(5.seconds), interval(10.milliseconds)) { + assert("hello" === message) + } + } + + test("send a message remotely") { + @volatile var message: String = null + // Set up a RpcEndpoint using env + env.setupEndpoint("send-remotely", new RpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case msg: String => message = msg + } + }) + + val anotherEnv = createRpcEnv(new CssConf(), "remote", 0, clientMode = true) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "send-remotely") + try { + rpcEndpointRef.send("hello") + eventually(timeout(5.seconds), interval(10.milliseconds)) { + assert("hello" === message) + } + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("send a RpcEndpointRef") { + val endpoint = new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext) = { + case "Hello" => context.reply(self) + case "Echo" => context.reply("Echo") + } + } + val rpcEndpointRef = env.setupEndpoint("send-ref", endpoint) + val newRpcEndpointRef = rpcEndpointRef.askSync[RpcEndpointRef]("Hello") + val reply = newRpcEndpointRef.askSync[String]("Echo") + assert("Echo" === reply) + } + + test("ask a message locally") { + val rpcEndpointRef = env.setupEndpoint("ask-locally", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => + context.reply(msg) + } + }) + val reply = rpcEndpointRef.askSync[String]("hello") + assert("hello" === reply) + } + + test("ask a message remotely") { + env.setupEndpoint("ask-remotely", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => + context.reply(msg) + } + }) + + val anotherEnv = createRpcEnv(new CssConf(), "remote", 0, clientMode = true) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "ask-remotely") + try { + val reply = rpcEndpointRef.askSync[String]("hello") + assert("hello" === reply) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("ask a message timeout") { + env.setupEndpoint("ask-timeout", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => + Thread.sleep(100) + context.reply(msg) + } + }) + + val conf = new CssConf() + val shortProp = "css.rpc.short.timeout" + conf.set("css.rpc.retry.wait", "0") + conf.set("css.rpc.numRetries", "1") + val anotherEnv = createRpcEnv(conf, "remote", 0, clientMode = true) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "ask-timeout") + try { + val e = intercept[RpcTimeoutException] { + rpcEndpointRef.askSync[String]("hello", new RpcTimeout(1.millisecond, shortProp)) + } + // The CssException cause should be a RpcTimeoutException with message indicating the + // controlling timeout property + assert(e.isInstanceOf[RpcTimeoutException]) + assert(e.getMessage.contains(shortProp)) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("onStart and onStop") { + val stopLatch = new CountDownLatch(1) + val calledMethods = mutable.ArrayBuffer[String]() + + val endpoint = new RpcEndpoint { + override val rpcEnv = env + + override def onStart(): Unit = { + calledMethods += "start" + } + + override def receive: PartialFunction[Any, Unit] = { + case msg: String => + } + + override def onStop(): Unit = { + calledMethods += "stop" + stopLatch.countDown() + } + } + val rpcEndpointRef = env.setupEndpoint("start-stop-test", endpoint) + env.stop(rpcEndpointRef) + stopLatch.await(10, TimeUnit.SECONDS) + assert(List("start", "stop") === calledMethods) + } + + test("onError: error in onStart") { + @volatile var e: Throwable = null + env.setupEndpoint("onError-onStart", new RpcEndpoint { + override val rpcEnv = env + + override def onStart(): Unit = { + throw new RuntimeException("Oops!") + } + + override def receive: PartialFunction[Any, Unit] = { + case m => + } + + override def onError(cause: Throwable): Unit = { + e = cause + } + }) + + eventually(timeout(5.seconds), interval(10.milliseconds)) { + assert(e.getMessage === "Oops!") + } + } + + test("onError: error in onStop") { + @volatile var e: Throwable = null + val endpointRef = env.setupEndpoint("onError-onStop", new RpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case m => + } + + override def onError(cause: Throwable): Unit = { + e = cause + } + + override def onStop(): Unit = { + throw new RuntimeException("Oops!") + } + }) + + env.stop(endpointRef) + + eventually(timeout(5.seconds), interval(10.milliseconds)) { + assert(e.getMessage === "Oops!") + } + } + + test("onError: error in receive") { + @volatile var e: Throwable = null + val endpointRef = env.setupEndpoint("onError-receive", new RpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case m => throw new RuntimeException("Oops!") + } + + override def onError(cause: Throwable): Unit = { + e = cause + } + }) + + endpointRef.send("Foo") + + eventually(timeout(5.seconds), interval(10.milliseconds)) { + assert(e.getMessage === "Oops!") + } + } + + test("self: call in onStart") { + @volatile var callSelfSuccessfully = false + + env.setupEndpoint("self-onStart", new RpcEndpoint { + override val rpcEnv = env + + override def onStart(): Unit = { + self + callSelfSuccessfully = true + } + + override def receive: PartialFunction[Any, Unit] = { + case m => + } + }) + + eventually(timeout(5.seconds), interval(10.milliseconds)) { + // Calling `self` in `onStart` is fine + assert(callSelfSuccessfully) + } + } + + test("self: call in receive") { + @volatile var callSelfSuccessfully = false + + val endpointRef = env.setupEndpoint("self-receive", new RpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case m => + self + callSelfSuccessfully = true + } + }) + + endpointRef.send("Foo") + + eventually(timeout(5.seconds), interval(10.milliseconds)) { + // Calling `self` in `receive` is fine + assert(callSelfSuccessfully) + } + } + + test("self: call in onStop") { + @volatile var selfOption: Option[RpcEndpointRef] = null + + val endpointRef = env.setupEndpoint("self-onStop", new RpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case m => + } + + override def onStop(): Unit = { + selfOption = Option(self) + } + }) + + env.stop(endpointRef) + + eventually(timeout(5.seconds), interval(10.milliseconds)) { + // Calling `self` in `onStop` will return null, so selfOption will be None + assert(selfOption.isEmpty) + } + } + + test("call receive in sequence") { + // If a RpcEnv implementation breaks the `receive` contract, hope this test can expose it + for (i <- 0 until 100) { + @volatile var result = 0 + val endpointRef = env.setupEndpoint(s"receive-in-sequence-$i", new ThreadSafeRpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case m => result += 1 + } + + }) + + (0 until 10) foreach { _ => + new Thread { + override def run(): Unit = { + (0 until 100) foreach { _ => + endpointRef.send("Hello") + } + } + }.start() + } + + eventually(timeout(5.seconds), interval(5.milliseconds)) { + assert(result == 1000) + } + + env.stop(endpointRef) + } + } + + test("stop(RpcEndpointRef) reentrant") { + @volatile var onStopCount = 0 + val endpointRef = env.setupEndpoint("stop-reentrant", new RpcEndpoint { + override val rpcEnv = env + + override def receive: PartialFunction[Any, Unit] = { + case m => + } + + override def onStop(): Unit = { + onStopCount += 1 + } + }) + + env.stop(endpointRef) + env.stop(endpointRef) + + eventually(timeout(5.seconds), interval(5.milliseconds)) { + // Calling stop twice should only trigger onStop once. + assert(onStopCount == 1) + } + } + + test("sendWithReply") { + val endpointRef = env.setupEndpoint("sendWithReply", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case m => context.reply("ack") + } + }) + + val f = endpointRef.ask[String]("Hi") + val ack = ThreadUtils.awaitResult(f, 5.seconds) + assert("ack" === ack) + + env.stop(endpointRef) + } + + test("sendWithReply: remotely") { + env.setupEndpoint("sendWithReply-remotely", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case m => context.reply("ack") + } + }) + + val anotherEnv = createRpcEnv(new CssConf(), "remote", 0, clientMode = true) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "sendWithReply-remotely") + try { + val f = rpcEndpointRef.ask[String]("hello") + val ack = ThreadUtils.awaitResult(f, 5.seconds) + assert("ack" === ack) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("sendWithReply: error") { + val endpointRef = env.setupEndpoint("sendWithReply-error", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case m => context.sendFailure(new CssException("Oops")) + } + }) + + val f = endpointRef.ask[String]("Hi") + val e = intercept[CssException] { + ThreadUtils.awaitResult(f, 5.seconds) + } + assert("Oops" === e.getCause.getMessage) + + env.stop(endpointRef) + } + + test("sendWithReply: remotely error") { + env.setupEndpoint("sendWithReply-remotely-error", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => context.sendFailure(new CssException("Oops")) + } + }) + + val anotherEnv = createRpcEnv(new CssConf(), "remote", 0, clientMode = true) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "sendWithReply-remotely-error") + try { + val f = rpcEndpointRef.ask[String]("hello") + val e = intercept[CssException] { + ThreadUtils.awaitResult(f, 5.seconds) + } + assert("Oops" === e.getCause.getMessage) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + /** + * Setup an [[RpcEndpoint]] to collect all network events. + * + * @return the [[RpcEndpointRef]] and a `ConcurrentLinkedQueue` that contains network events. + */ + private def setupNetworkEndpoint( + _env: RpcEnv, + name: String): (RpcEndpointRef, ConcurrentLinkedQueue[(Any, Any)]) = { + val events = new ConcurrentLinkedQueue[(Any, Any)] + val ref = _env.setupEndpoint("network-events-non-client", new ThreadSafeRpcEndpoint { + override val rpcEnv = _env + + override def receive: PartialFunction[Any, Unit] = { + case "hello" => + case m => events.add("receive" -> m) + } + + override def onConnected(remoteAddress: RpcAddress): Unit = { + events.add("onConnected" -> remoteAddress) + } + + override def onDisconnected(remoteAddress: RpcAddress): Unit = { + events.add("onDisconnected" -> remoteAddress) + } + + override def onNetworkError(cause: Throwable, remoteAddress: RpcAddress): Unit = { + events.add("onNetworkError" -> remoteAddress) + } + + }) + (ref, events) + } + + test("network events in sever RpcEnv when another RpcEnv is in server mode") { + val serverEnv1 = createRpcEnv(new CssConf(), "server1", 0, clientMode = false) + val serverEnv2 = createRpcEnv(new CssConf(), "server2", 0, clientMode = false) + val (_, events) = setupNetworkEndpoint(serverEnv1, "network-events") + val (serverRef2, _) = setupNetworkEndpoint(serverEnv2, "network-events") + try { + val serverRefInServer2 = serverEnv1.setupEndpointRef(serverRef2.address, serverRef2.name) + // Send a message to set up the connection + serverRefInServer2.send("hello") + + eventually(timeout(5.seconds), interval(5.milliseconds)) { + assert(events.contains(("onConnected", serverEnv2.address))) + } + + serverEnv2.shutdown() + serverEnv2.awaitTermination() + + eventually(timeout(5.seconds), interval(5.milliseconds)) { + assert(events.contains(("onConnected", serverEnv2.address))) + assert(events.contains(("onDisconnected", serverEnv2.address))) + } + } finally { + serverEnv1.shutdown() + serverEnv2.shutdown() + serverEnv1.awaitTermination() + serverEnv2.awaitTermination() + } + } + + test("network events in sever RpcEnv when another RpcEnv is in client mode") { + val serverEnv = createRpcEnv(new CssConf(), "server", 0, clientMode = false) + val (serverRef, events) = setupNetworkEndpoint(serverEnv, "network-events") + val clientEnv = createRpcEnv(new CssConf(), "client", 0, clientMode = true) + try { + val serverRefInClient = clientEnv.setupEndpointRef(serverRef.address, serverRef.name) + // Send a message to set up the connection + serverRefInClient.send("hello") + + eventually(timeout(5.seconds), interval(5.milliseconds)) { + // We don't know the exact client address but at least we can verify the message type + assert(events.asScala.map(_._1).exists(_ == "onConnected")) + } + + clientEnv.shutdown() + clientEnv.awaitTermination() + + eventually(timeout(5.seconds), interval(5.milliseconds)) { + // We don't know the exact client address but at least we can verify the message type + assert(events.asScala.map(_._1).exists(_ == "onConnected")) + assert(events.asScala.map(_._1).exists(_ == "onDisconnected")) + } + } finally { + clientEnv.shutdown() + serverEnv.shutdown() + clientEnv.awaitTermination() + serverEnv.awaitTermination() + } + } + + test("network events in client RpcEnv when another RpcEnv is in server mode") { + val clientEnv = createRpcEnv(new CssConf(), "client", 0, clientMode = true) + val serverEnv = createRpcEnv(new CssConf(), "server", 0, clientMode = false) + val (_, events) = setupNetworkEndpoint(clientEnv, "network-events") + val (serverRef, _) = setupNetworkEndpoint(serverEnv, "network-events") + try { + val serverRefInClient = clientEnv.setupEndpointRef(serverRef.address, serverRef.name) + // Send a message to set up the connection + serverRefInClient.send("hello") + + eventually(timeout(5.seconds), interval(5.milliseconds)) { + assert(events.contains(("onConnected", serverEnv.address))) + } + + serverEnv.shutdown() + serverEnv.awaitTermination() + + eventually(timeout(5.seconds), interval(5.milliseconds)) { + assert(events.contains(("onConnected", serverEnv.address))) + assert(events.contains(("onDisconnected", serverEnv.address))) + } + } finally { + clientEnv.shutdown() + serverEnv.shutdown() + clientEnv.awaitTermination() + serverEnv.awaitTermination() + } + } + + test("sendWithReply: unserializable error") { + env.setupEndpoint("sendWithReply-unserializable-error", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => context.sendFailure(new UnserializableException) + } + }) + + val anotherEnv = createRpcEnv(new CssConf(), "remote", 0, clientMode = true) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = + anotherEnv.setupEndpointRef(env.address, "sendWithReply-unserializable-error") + try { + val f = rpcEndpointRef.ask[String]("hello") + val e = intercept[CssException] { + ThreadUtils.awaitResult(f, 1.second) + } + assert(e.getCause.isInstanceOf[NotSerializableException]) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + test("port conflict") { + val anotherEnv = createRpcEnv(new CssConf(), "remote", env.address.port) + try { + assert(anotherEnv.address.port != env.address.port) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + private def testSend(conf: CssConf): Unit = { + val localEnv = createRpcEnv(conf, "authentication-local", 0) + val remoteEnv = createRpcEnv(conf, "authentication-remote", 0, clientMode = true) + + try { + @volatile var message: String = null + localEnv.setupEndpoint("send-authentication", new RpcEndpoint { + override val rpcEnv = localEnv + + override def receive: PartialFunction[Any, Unit] = { + case msg: String => message = msg + } + }) + val rpcEndpointRef = remoteEnv.setupEndpointRef(localEnv.address, "send-authentication") + rpcEndpointRef.send("hello") + eventually(timeout(5.seconds), interval(10.milliseconds)) { + assert("hello" === message) + } + } finally { + localEnv.shutdown() + localEnv.awaitTermination() + remoteEnv.shutdown() + remoteEnv.awaitTermination() + } + } + + private def testAsk(conf: CssConf): Unit = { + val localEnv = createRpcEnv(conf, "authentication-local", 0) + val remoteEnv = createRpcEnv(conf, "authentication-remote", 0, clientMode = true) + + try { + localEnv.setupEndpoint("ask-authentication", new RpcEndpoint { + override val rpcEnv = localEnv + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => + context.reply(msg) + } + }) + val rpcEndpointRef = remoteEnv.setupEndpointRef(localEnv.address, "ask-authentication") + val reply = rpcEndpointRef.askSync[String]("hello") + assert("hello" === reply) + } finally { + localEnv.shutdown() + localEnv.awaitTermination() + remoteEnv.shutdown() + remoteEnv.awaitTermination() + } + } + + test("send with authentication") { + testSend(new CssConf() + .set("NETWORK_AUTH_ENABLED", "true") + .set("AUTH_SECRET", "good")) + } + +// test("send with SASL encryption") { +// testSend(new CssConf() +// .set(NETWORK_AUTH_ENABLED, true) +// .set(AUTH_SECRET, "good") +// .set(SASL_ENCRYPTION_ENABLED, true)) +// } +// +// test("send with AES encryption") { +// testSend(new CssConf() +// .set(NETWORK_AUTH_ENABLED, true) +// .set(AUTH_SECRET, "good") +// .set(Network.NETWORK_CRYPTO_ENABLED, true) +// .set(Network.NETWORK_CRYPTO_SASL_FALLBACK, false)) +// } +// +// test("ask with authentication") { +// testAsk(new CssConf() +// .set(NETWORK_AUTH_ENABLED, true) +// .set(AUTH_SECRET, "good")) +// } +// +// test("ask with SASL encryption") { +// testAsk(new CssConf() +// .set(NETWORK_AUTH_ENABLED, true) +// .set(AUTH_SECRET, "good") +// .set(SASL_ENCRYPTION_ENABLED, true)) +// } +// +// test("ask with AES encryption") { +// testAsk(new CssConf() +// .set(NETWORK_AUTH_ENABLED, true) +// .set(AUTH_SECRET, "good") +// .set(Network.NETWORK_CRYPTO_ENABLED, true) +// .set(Network.NETWORK_CRYPTO_SASL_FALLBACK, false)) +// } + + test("construct RpcTimeout with conf property") { + val conf = new CssConf + + val testProp = "css.ask.test.timeout" + val testDurationSeconds = 30 + val secondaryProp = "css.ask.secondary.timeout" + + conf.set(testProp, s"${testDurationSeconds}s") + conf.set(secondaryProp, "100s") + + // Construct RpcTimeout with a single property + val rt1 = RpcTimeout(conf, testProp) + assert( testDurationSeconds === rt1.duration.toSeconds ) + + // Construct RpcTimeout with prioritized list of properties + val rt2 = RpcTimeout(conf, Seq("css.ask.invalid.timeout", testProp, secondaryProp), "1s") + assert( testDurationSeconds === rt2.duration.toSeconds ) + + // Construct RpcTimeout with default value, + val defaultProp = "css.ask.default.timeout" + val defaultDurationSeconds = 1 + val rt3 = RpcTimeout(conf, Seq(defaultProp), defaultDurationSeconds.toString + "s") + assert( defaultDurationSeconds === rt3.duration.toSeconds ) + assert( rt3.timeoutProp.contains(defaultProp) ) + + // Try to construct RpcTimeout with an unconfigured property + intercept[NoSuchElementException] { + RpcTimeout(conf, "css.ask.invalid.timeout") + } + } + + test("ask a message timeout on Future using RpcTimeout") { + case class NeverReply(msg: String) + + val rpcEndpointRef = env.setupEndpoint("ask-future", new RpcEndpoint { + override val rpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case msg: String => context.reply(msg) + case _: NeverReply => + } + }) + + val longTimeout = new RpcTimeout(1.second, "css.rpc.long.timeout") + val shortTimeout = new RpcTimeout(10.milliseconds, "css.rpc.short.timeout") + + // Ask with immediate response, should complete successfully + val fut1 = rpcEndpointRef.ask[String]("hello", longTimeout) + val reply1 = longTimeout.awaitResult(fut1) + assert("hello" === reply1) + + // Ask with a delayed response and wait for response immediately that should timeout + val fut2 = rpcEndpointRef.ask[String](NeverReply("doh"), shortTimeout) + val reply2 = + intercept[RpcTimeoutException] { + shortTimeout.awaitResult(fut2) + }.getMessage + + // RpcTimeout.awaitResult should have added the property to the TimeoutException message + assert(reply2.contains(shortTimeout.timeoutProp)) + + // Ask with delayed response and allow the Future to timeout before ThreadUtils.awaitResult + val fut3 = rpcEndpointRef.ask[String](NeverReply("goodbye"), shortTimeout) + + // scalastyle:off awaitresult + // Allow future to complete with failure using plain Await.result, this will return + // once the future is complete to verify addMessageIfTimeout was invoked + val reply3 = + intercept[RpcTimeoutException] { + Await.result(fut3, 2.seconds) + }.getMessage + // scalastyle:on awaitresult + + // When the future timed out, the recover callback should have used + // RpcTimeout.addMessageIfTimeout to add the property to the TimeoutException message + assert(reply3.contains(shortTimeout.timeoutProp)) + + // Use RpcTimeout.awaitResult to process Future, since it has already failed with + // RpcTimeoutException, the same RpcTimeoutException should be thrown + val reply4 = + intercept[RpcTimeoutException] { + shortTimeout.awaitResult(fut3) + }.getMessage + + // Ensure description is not in message twice after addMessageIfTimeout and awaitResult + assert(shortTimeout.timeoutProp.r.findAllIn(reply4).length === 1) + } + + test("SPARK-14699: RpcEnv.shutdown should not fire onDisconnected events") { + env.setupEndpoint("SPARK-14699", new RpcEndpoint { + override val rpcEnv: RpcEnv = env + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case m => context.reply(m) + } + }) + + val anotherEnv = createRpcEnv(new CssConf(), "remote", 0) + val endpoint = mock(classOf[RpcEndpoint]) + anotherEnv.setupEndpoint("SPARK-14699", endpoint) + + val ref = anotherEnv.setupEndpointRef(env.address, "SPARK-14699") + // Make sure the connect is set up + assert(ref.askSync[String]("hello") === "hello") + anotherEnv.shutdown() + anotherEnv.awaitTermination() + + env.stop(ref) + + verify(endpoint).onStop() + verify(endpoint, never()).onDisconnected(any()) + verify(endpoint, never()).onNetworkError(any(), any()) + } +} + +case class Register(ref: RpcEndpointRef) + +class UnserializableClass + +class UnserializableException extends Exception { + private val unserializableField = new UnserializableClass +} diff --git a/common/src/test/scala/com/bytedance/css/common.rpc/TestRpcEndpoint.scala b/common/src/test/scala/com/bytedance/css/common.rpc/TestRpcEndpoint.scala new file mode 100644 index 0000000..6435bb9 --- /dev/null +++ b/common/src/test/scala/com/bytedance/css/common.rpc/TestRpcEndpoint.scala @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc + +import scala.collection.mutable.ArrayBuffer + +import org.scalactic.TripleEquals + +class TestRpcEndpoint extends ThreadSafeRpcEndpoint with TripleEquals { + + override val rpcEnv: RpcEnv = null + + @volatile private var receiveMessages = ArrayBuffer[Any]() + + @volatile private var receiveAndReplyMessages = ArrayBuffer[Any]() + + @volatile private var onConnectedMessages = ArrayBuffer[RpcAddress]() + + @volatile private var onDisconnectedMessages = ArrayBuffer[RpcAddress]() + + @volatile private var onNetworkErrorMessages = ArrayBuffer[(Throwable, RpcAddress)]() + + @volatile private var started = false + + @volatile private var stopped = false + + override def receive: PartialFunction[Any, Unit] = { + case message: Any => receiveMessages += message + } + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case message: Any => receiveAndReplyMessages += message + } + + override def onConnected(remoteAddress: RpcAddress): Unit = { + onConnectedMessages += remoteAddress + } + + /** + * Invoked when some network error happens in the connection between the current node and + * `remoteAddress`. + */ + override def onNetworkError(cause: Throwable, remoteAddress: RpcAddress): Unit = { + onNetworkErrorMessages += cause -> remoteAddress + } + + override def onDisconnected(remoteAddress: RpcAddress): Unit = { + onDisconnectedMessages += remoteAddress + } + + def numReceiveMessages: Int = receiveMessages.size + + override def onStart(): Unit = { + started = true + } + + override def onStop(): Unit = { + stopped = true + } + + def verifyStarted(): Unit = { + assert(started, "RpcEndpoint is not started") + } + + def verifyStopped(): Unit = { + assert(stopped, "RpcEndpoint is not stopped") + } + + def verifyReceiveMessages(expected: Seq[Any]): Unit = { + assert(receiveMessages === expected) + } + + def verifySingleReceiveMessage(message: Any): Unit = { + verifyReceiveMessages(List(message)) + } + + def verifyReceiveAndReplyMessages(expected: Seq[Any]): Unit = { + assert(receiveAndReplyMessages === expected) + } + + def verifySingleReceiveAndReplyMessage(message: Any): Unit = { + verifyReceiveAndReplyMessages(List(message)) + } + + def verifySingleOnConnectedMessage(remoteAddress: RpcAddress): Unit = { + verifyOnConnectedMessages(List(remoteAddress)) + } + + def verifyOnConnectedMessages(expected: Seq[RpcAddress]): Unit = { + assert(onConnectedMessages === expected) + } + + def verifySingleOnDisconnectedMessage(remoteAddress: RpcAddress): Unit = { + verifyOnDisconnectedMessages(List(remoteAddress)) + } + + def verifyOnDisconnectedMessages(expected: Seq[RpcAddress]): Unit = { + assert(onDisconnectedMessages === expected) + } + + def verifySingleOnNetworkErrorMessage(cause: Throwable, remoteAddress: RpcAddress): Unit = { + verifyOnNetworkErrorMessages(List(cause -> remoteAddress)) + } + + def verifyOnNetworkErrorMessages(expected: Seq[(Throwable, RpcAddress)]): Unit = { + assert(onNetworkErrorMessages === expected) + } +} diff --git a/common/src/test/scala/com/bytedance/css/common.rpc/netty/InboxSuite.scala b/common/src/test/scala/com/bytedance/css/common.rpc/netty/InboxSuite.scala new file mode 100644 index 0000000..16ee16a --- /dev/null +++ b/common/src/test/scala/com/bytedance/css/common.rpc/netty/InboxSuite.scala @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc.netty + +import java.util.concurrent.{CountDownLatch, TimeUnit} +import java.util.concurrent.atomic.AtomicInteger + +import com.bytedance.css.common.internal.Logging +import com.bytedance.css.common.rpc.{RpcAddress, TestRpcEndpoint} +import org.mockito.Mockito._ +import org.scalatest.FunSuite + +class InboxSuite extends FunSuite with Logging { + + test("post") { + val endpoint = new TestRpcEndpoint + val endpointRef = mock(classOf[NettyRpcEndpointRef]) + when(endpointRef.name).thenReturn("hello") + + val dispatcher = mock(classOf[Dispatcher]) + + val inbox = new Inbox(endpointRef, endpoint) + val message = OneWayMessage(null, "hi") + inbox.post(message) + inbox.process(dispatcher) + assert(inbox.isEmpty) + + endpoint.verifySingleReceiveMessage("hi") + + inbox.stop() + inbox.process(dispatcher) + assert(inbox.isEmpty) + endpoint.verifyStarted() + endpoint.verifyStopped() + } + + test("post: with reply") { + val endpoint = new TestRpcEndpoint + val endpointRef = mock(classOf[NettyRpcEndpointRef]) + val dispatcher = mock(classOf[Dispatcher]) + + val inbox = new Inbox(endpointRef, endpoint) + val message = RpcMessage(null, "hi", null) + inbox.post(message) + inbox.process(dispatcher) + assert(inbox.isEmpty) + + endpoint.verifySingleReceiveAndReplyMessage("hi") + } + + test("post: multiple threads") { + val endpoint = new TestRpcEndpoint + val endpointRef = mock(classOf[NettyRpcEndpointRef]) + when(endpointRef.name).thenReturn("hello") + + val dispatcher = mock(classOf[Dispatcher]) + + val numDroppedMessages = new AtomicInteger(0) + val inbox = new Inbox(endpointRef, endpoint) { + override def onDrop(message: InboxMessage): Unit = { + numDroppedMessages.incrementAndGet() + } + } + + val exitLatch = new CountDownLatch(10) + + for (_ <- 0 until 10) { + new Thread { + override def run(): Unit = { + for (_ <- 0 until 100) { + val message = OneWayMessage(null, "hi") + inbox.post(message) + } + exitLatch.countDown() + } + }.start() + } + // Try to process some messages + inbox.process(dispatcher) + inbox.stop() + // After `stop` is called, further messages will be dropped. However, while `stop` is called, + // some messages may be post to Inbox, so process them here. + inbox.process(dispatcher) + assert(inbox.isEmpty) + + exitLatch.await(30, TimeUnit.SECONDS) + + assert(1000 === endpoint.numReceiveMessages + numDroppedMessages.get) + endpoint.verifyStarted() + endpoint.verifyStopped() + } + + test("post: Associated") { + val endpoint = new TestRpcEndpoint + val endpointRef = mock(classOf[NettyRpcEndpointRef]) + val dispatcher = mock(classOf[Dispatcher]) + + val remoteAddress = RpcAddress("localhost", 11111) + + val inbox = new Inbox(endpointRef, endpoint) + inbox.post(RemoteProcessConnected(remoteAddress)) + inbox.process(dispatcher) + + endpoint.verifySingleOnConnectedMessage(remoteAddress) + } + + test("post: Disassociated") { + val endpoint = new TestRpcEndpoint + val endpointRef = mock(classOf[NettyRpcEndpointRef]) + val dispatcher = mock(classOf[Dispatcher]) + + val remoteAddress = RpcAddress("localhost", 11111) + + val inbox = new Inbox(endpointRef, endpoint) + inbox.post(RemoteProcessDisconnected(remoteAddress)) + inbox.process(dispatcher) + + endpoint.verifySingleOnDisconnectedMessage(remoteAddress) + } + + test("post: AssociationError") { + val endpoint = new TestRpcEndpoint + val endpointRef = mock(classOf[NettyRpcEndpointRef]) + val dispatcher = mock(classOf[Dispatcher]) + + val remoteAddress = RpcAddress("localhost", 11111) + val cause = new RuntimeException("Oops") + + val inbox = new Inbox(endpointRef, endpoint) + inbox.post(RemoteProcessConnectionError(cause, remoteAddress)) + inbox.process(dispatcher) + + endpoint.verifySingleOnNetworkErrorMessage(cause, remoteAddress) + } +} diff --git a/common/src/test/scala/com/bytedance/css/common.rpc/netty/NettyRpcAddressSuite.scala b/common/src/test/scala/com/bytedance/css/common.rpc/netty/NettyRpcAddressSuite.scala new file mode 100644 index 0000000..823ba8e --- /dev/null +++ b/common/src/test/scala/com/bytedance/css/common.rpc/netty/NettyRpcAddressSuite.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc.netty + +import com.bytedance.css.common.rpc.RpcEndpointAddress +import org.scalatest.FunSuite + +class NettyRpcAddressSuite extends FunSuite { + + test("toString") { + val addr = new RpcEndpointAddress("localhost", 12345, "test") + assert(addr.toString === "css://test@localhost:12345") + } + + test("toString for client mode") { + val addr = RpcEndpointAddress(null, "test") + assert(addr.toString === "css-client://test") + } + +} diff --git a/common/src/test/scala/com/bytedance/css/common.rpc/netty/NettyRpcEnvSuite.scala b/common/src/test/scala/com/bytedance/css/common.rpc/netty/NettyRpcEnvSuite.scala new file mode 100644 index 0000000..cb14daf --- /dev/null +++ b/common/src/test/scala/com/bytedance/css/common.rpc/netty/NettyRpcEnvSuite.scala @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc.netty + +import java.util.concurrent.ExecutionException + +import scala.concurrent.duration._ + +import com.bytedance.css.common.CssConf +import com.bytedance.css.common.exception.CssException +import com.bytedance.css.common.rpc.{Register, RpcAddress, RpcCallContext, RpcEndpoint, RpcEndpointAddress, RpcEndpointNotFoundException, RpcEndpointRef, RpcEnv, RpcEnvConfig, RpcEnvSuite, RpcTimeout, RpcTimeoutException} +import com.bytedance.css.common.util.ThreadUtils +import com.bytedance.css.network.client.TransportClient +import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} +import org.scalatestplus.mockito.MockitoSugar + +class NettyRpcEnvSuite extends RpcEnvSuite with MockitoSugar with TimeLimits { + + private implicit val signaler: Signaler = ThreadSignaler + + override def createRpcEnv( + conf: CssConf, + name: String, + port: Int, + clientMode: Boolean = false): RpcEnv = { + val config = RpcEnvConfig(conf, "test", "localhost", "localhost", port, 0, clientMode) + new NettyRpcEnvFactory().create(config) + } + + test("non-existent endpoint") { + val uri = RpcEndpointAddress(env.address, "nonexist-endpoint").toString + val e = intercept[CssException] { + env.setupEndpointRef(env.address, "nonexist-endpoint") + } + assert(e.getCause.isInstanceOf[RpcEndpointNotFoundException]) + assert(e.getCause.getMessage.contains(uri)) + } + + test("advertise address different from bind address") { + val cssConf = new CssConf() + val config = RpcEnvConfig(cssConf, "test", "localhost", "example.com", 0, 0, false) + val env = new NettyRpcEnvFactory().create(config) + try { + assert(env.address.hostPort.startsWith("example.com:")) + } finally { + env.shutdown() + } + } + + test("RequestMessage serialization") { + def assertRequestMessageEquals(expected: RequestMessage, actual: RequestMessage): Unit = { + assert(expected.senderAddress === actual.senderAddress) + assert(expected.receiver === actual.receiver) + assert(expected.content === actual.content) + } + + val nettyEnv = env.asInstanceOf[NettyRpcEnv] + val client = mock[TransportClient] + val senderAddress = RpcAddress("localhost", 12345) + val receiverAddress = RpcEndpointAddress("localhost", 54321, "test") + val receiver = new NettyRpcEndpointRef(nettyEnv.conf, receiverAddress, nettyEnv) + + val msg = new RequestMessage(senderAddress, receiver, "foo") + assertRequestMessageEquals( + msg, + RequestMessage(nettyEnv, client, msg.serialize(nettyEnv))) + + val msg2 = new RequestMessage(null, receiver, "foo") + assertRequestMessageEquals( + msg2, + RequestMessage(nettyEnv, client, msg2.serialize(nettyEnv))) + + val msg3 = new RequestMessage(senderAddress, receiver, null) + assertRequestMessageEquals( + msg3, + RequestMessage(nettyEnv, client, msg3.serialize(nettyEnv))) + } + + test("StackOverflowError should be sent back and Dispatcher should survive") { + val numUsableCores = 2 + val conf = new CssConf + val config = RpcEnvConfig( + conf, + "test", + "localhost", + "localhost", + 0, + numUsableCores, + clientMode = false) + val anotherEnv = new NettyRpcEnvFactory().create(config) + anotherEnv.setupEndpoint("StackOverflowError", new RpcEndpoint { + override val rpcEnv = anotherEnv + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + // scalastyle:off throwerror + case msg: String => throw new StackOverflowError + // scalastyle:on throwerror + case num: Int => context.reply(num) + } + }) + + val rpcEndpointRef = env.setupEndpointRef(anotherEnv.address, "StackOverflowError") + try { + // Send `numUsableCores` messages to trigger `numUsableCores` `StackOverflowError`s + for (_ <- 0 until numUsableCores) { + val e = intercept[CssException] { + rpcEndpointRef.askSync[String]("hello") + } + // The root cause `e.getCause.getCause` because it is boxed by Scala Promise. + assert(e.getCause.isInstanceOf[ExecutionException]) + assert(e.getCause.getCause.isInstanceOf[StackOverflowError]) + } + failAfter(10.seconds) { + assert(rpcEndpointRef.askSync[Int](100) === 100) + } + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } + + + test("SPARK-31233: ask rpcEndpointRef in client mode timeout") { + var remoteRef: RpcEndpointRef = null + env.setupEndpoint("ask-remotely-server", new RpcEndpoint { + override val rpcEnv = env + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case Register(ref) => + remoteRef = ref + context.reply("okay") + case msg: String => + context.reply(msg) + } + }) + val conf = new CssConf() + val anotherEnv = createRpcEnv(conf, "remote", 0, clientMode = true) + // Use anotherEnv to find out the RpcEndpointRef + val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "ask-remotely-server") + // com.bytedance.css.common.Register a rpcEndpointRef in anotherEnv + val anotherRef = anotherEnv.setupEndpoint("receiver", new RpcEndpoint { + override val rpcEnv = anotherEnv + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + case _ => + Thread.sleep(1200) + context.reply("okay") + } + }) + try { + val reply = rpcEndpointRef.askSync[String](Register(anotherRef)) + assert("okay" === reply) + val timeout = "1s" + val answer = remoteRef.ask[String]("msg", + RpcTimeout(conf, Seq("css.rpc.askTimeout", "css.network.timeout"), timeout)) + val thrown = intercept[RpcTimeoutException] { + ThreadUtils.awaitResult(answer, Duration(1300, MILLISECONDS)) + } + val remoteAddr = remoteRef.asInstanceOf[NettyRpcEndpointRef].client.getChannel.remoteAddress + assert(thrown.getMessage.contains(remoteAddr.toString)) + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } +} diff --git a/common/src/test/scala/com/bytedance/css/common.rpc/netty/NettyRpcHandlerSuite.scala b/common/src/test/scala/com/bytedance/css/common.rpc/netty/NettyRpcHandlerSuite.scala new file mode 100644 index 0000000..e657fdc --- /dev/null +++ b/common/src/test/scala/com/bytedance/css/common.rpc/netty/NettyRpcHandlerSuite.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + * This file may have been modified by Bytedance Inc. + */ + +package com.bytedance.css.common.rpc.netty + +import java.net.InetSocketAddress +import java.nio.ByteBuffer + +import com.bytedance.css.common.rpc.RpcAddress +import com.bytedance.css.network.client.{TransportClient, TransportResponseHandler} +import com.bytedance.css.network.server.StreamManager +import io.netty.channel.Channel +import org.mockito.Matchers.any +import org.mockito.Mockito.{mock, times, verify, when} +import org.scalatest.FunSuite + +class NettyRpcHandlerSuite extends FunSuite { + + val env = mock(classOf[NettyRpcEnv]) + val sm = mock(classOf[StreamManager]) + when(env.deserialize(any(classOf[TransportClient]), any(classOf[ByteBuffer]))(any())) + .thenReturn(new RequestMessage(RpcAddress("localhost", 12345), null, null)) + + test("receive") { + val dispatcher = mock(classOf[Dispatcher]) + val nettyRpcHandler = new NettyRpcHandler(dispatcher, env, sm) + + val channel = mock(classOf[Channel]) + val client = new TransportClient(channel, mock(classOf[TransportResponseHandler])) + when(channel.remoteAddress()).thenReturn(new InetSocketAddress("localhost", 40000)) + nettyRpcHandler.channelActive(client) + + verify(dispatcher, times(1)).postToAll(RemoteProcessConnected(RpcAddress("localhost", 40000))) + } + + test("connectionTerminated") { + val dispatcher = mock(classOf[Dispatcher]) + val nettyRpcHandler = new NettyRpcHandler(dispatcher, env, sm) + + val channel = mock(classOf[Channel]) + val client = new TransportClient(channel, mock(classOf[TransportResponseHandler])) + when(channel.remoteAddress()).thenReturn(new InetSocketAddress("localhost", 40000)) + nettyRpcHandler.channelActive(client) + + when(channel.remoteAddress()).thenReturn(new InetSocketAddress("localhost", 40000)) + nettyRpcHandler.channelInactive(client) + + verify(dispatcher, times(1)).postToAll(RemoteProcessConnected(RpcAddress("localhost", 40000))) + verify(dispatcher, times(1)).postToAll( + RemoteProcessDisconnected(RpcAddress("localhost", 40000))) + } + +} diff --git a/conf/css-defaults.conf b/conf/css-defaults.conf new file mode 100644 index 0000000..1f7d063 --- /dev/null +++ b/conf/css-defaults.conf @@ -0,0 +1,31 @@ +# +# Copyright 2022 Bytedance Inc. +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Default system properties included when running css master or worker. +# This is useful for setting default environmental settings. + + +# Example: +#css.commit.threads=128 +#css.flush.timeout=360s +#css.network.timeout=600s +#css.disk.dir.num.min=5 +#css.extMeta.expire.interval=600s +#css.zookeeper.address=localhost:2181 +#css.worker.registry.type=zookeeper \ No newline at end of file diff --git a/conf/css-metrics.properties.template b/conf/css-metrics.properties.template new file mode 100644 index 0000000..077269a --- /dev/null +++ b/conf/css-metrics.properties.template @@ -0,0 +1,66 @@ +# +# Copyright 2022 Bytedance Inc. +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +*.sink.bytedance.class=com.bytedance.css.common.metrics.sink.BytedanceSink + +## Examples +# Enable JmxSink for all instances by class name +#*.sink.jmx.class=com.bytedance.css.common.metrics.sink.JmxSink + +# Enable ConsoleSink for all instances by class name +#*.sink.console.class=com.bytedance.css.common.metrics.sink.ConsoleSink + +# Enable StatsdSink for all instances by class name +#*.sink.statsd.class=com.bytedance.css.common.metrics.sink.StatsdSink +#*.sink.statsd.prefix=spark + +# Polling period for the ConsoleSink +#*.sink.console.period=10 +# Unit of the polling period for the ConsoleSink +#*.sink.console.unit=seconds + +# Polling period for the ConsoleSink specific for the master instance +#master.sink.console.period=15 +# Unit of the polling period for the ConsoleSink specific for the master +# instance +#master.sink.console.unit=seconds + +# Enable CsvSink for all instances by class name +#*.sink.csv.class=com.bytedance.css.common.metrics.sink.CsvSink + +# Polling period for the CsvSink +#*.sink.csv.period=1 +# Unit of the polling period for the CsvSink +#*.sink.csv.unit=minutes + +# Polling directory for CsvSink +#*.sink.csv.directory=/tmp/ + +# Polling period for the CsvSink specific for the worker instance +#worker.sink.csv.period=10 +# Unit of the polling period for the CsvSink specific for the worker instance +#worker.sink.csv.unit=minutes + +# Enable Slf4jSink for all instances by class name +#*.sink.slf4j.class=com.bytedance.css.common.metrics.sink.Slf4jSink + +# Polling period for the Slf4JSink +#*.sink.slf4j.period=1 +# Unit of the polling period for the Slf4jSink +#*.sink.slf4j.unit=minutes diff --git a/conf/log4j-default.properties b/conf/log4j-default.properties new file mode 100644 index 0000000..3bc9551 --- /dev/null +++ b/conf/log4j-default.properties @@ -0,0 +1,50 @@ +# +# Copyright 2022 Bytedance Inc. +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the console +log4j.rootCategory=INFO, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + +# Set the default spark-shell log level to WARN. When running the spark-shell, the +# log level for this class is used to overwrite the root logger's log level, so that +# the user can have different defaults for the shell and regular Spark apps. +log4j.logger.org.apache.spark.repl.Main=WARN + +# Settings to quiet third party logs that are too verbose +log4j.logger.org.spark_project.jetty=WARN +log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO +log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO +log4j.logger.org.apache.parquet=ERROR +log4j.logger.parquet=ERROR + +# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support +log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL +log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR + +log4j.rootLogger=INFO, file +log4j.appender.file=org.apache.log4j.RollingFileAppender +log4j.appender.file.file=${css.log.dir}/${css.log.filename} +log4j.appender.file.maxFileSize=524288000 +log4j.appender.file.maxBackupIndex=6 +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.conversionPattern=%d{ISO8601} %p [%t] %c: %m%n diff --git a/conf/workers.template b/conf/workers.template new file mode 100644 index 0000000..b08a00e --- /dev/null +++ b/conf/workers.template @@ -0,0 +1,22 @@ +# +# Copyright 2022 Bytedance Inc. +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# A CSS Worker will be started on each of the machines listed below. + +localhost \ No newline at end of file diff --git a/css-assembly_2/pom.xml b/css-assembly_2/pom.xml new file mode 100644 index 0000000..693811d --- /dev/null +++ b/css-assembly_2/pom.xml @@ -0,0 +1,123 @@ + + + + 4.0.0 + + com.bytedance.inf + css-root + 1.0.0 + ../pom.xml + + + css-assembly_2 + jar + Cloud Shuffle Service Assembly for Spark 2 + + + + com.bytedance.inf + css-service_${scala.binary.version} + ${project.version} + + + hadoop-client + org.apache.hadoop + + + curator-framework + org.apache.curator + + + spark-tags_${scala.binary.version} + org.apache.spark + + + + + com.bytedance.inf + shuffle-manager-2 + ${project.version} + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + 2.8 + + + package + + copy + + + + + com.bytedance.inf + css-api + ${project.version} + ${project.build.directory}/libs + + + com.netflix.concurrency-limits + concurrency-limits-core + ${netflix.concurrency.version} + ${project.build.directory}/libs + + + com.bytedance.inf + css-service_${scala.binary.version} + ${project.version} + ${project.build.directory}/libs + + + com.bytedance.inf + css-common_${scala.binary.version} + ${project.version} + ${project.build.directory}/libs + + + com.bytedance.inf + css-client_${scala.binary.version} + ${project.version} + shaded + ${project.build.directory}/libs + + + com.bytedance.inf + css-network-common + ${project.version} + ${project.build.directory}/libs + + + com.bytedance.inf + shuffle-manager-2 + ${project.version} + ${project.build.directory}/libs + + + + + + + + + diff --git a/css-assembly_3/pom.xml b/css-assembly_3/pom.xml new file mode 100644 index 0000000..bb84223 --- /dev/null +++ b/css-assembly_3/pom.xml @@ -0,0 +1,123 @@ + + + + 4.0.0 + + com.bytedance.inf + css-root + 1.0.0 + ../pom.xml + + + css-assembly_3 + jar + Cloud Shuffle Service Assembly for Spark + + + + com.bytedance.inf + css-service_${scala.binary.version} + ${project.version} + + + hadoop-client + org.apache.hadoop + + + curator-framework + org.apache.curator + + + spark-tags_2.12 + org.apache.spark + + + + + com.bytedance.inf + shuffle-manager-3 + ${project.version} + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + 2.8 + + + package + + copy + + + + + com.bytedance.inf + css-api + ${project.version} + ${project.build.directory}/libs + + + com.netflix.concurrency-limits + concurrency-limits-core + ${netflix.concurrency.version} + ${project.build.directory}/libs + + + com.bytedance.inf + css-service_${scala.binary.version} + ${project.version} + ${project.build.directory}/libs + + + com.bytedance.inf + css-common_${scala.binary.version} + ${project.version} + ${project.build.directory}/libs + + + com.bytedance.inf + css-client_${scala.binary.version} + ${project.version} + shaded + ${project.build.directory}/libs + + + com.bytedance.inf + css-network-common + ${project.version} + ${project.build.directory}/libs + + + com.bytedance.inf + shuffle-manager-3 + ${project.version} + ${project.build.directory}/libs + + + + + + + + + diff --git a/dev/checkstyle-suppressions.xml b/dev/checkstyle-suppressions.xml new file mode 100644 index 0000000..804a178 --- /dev/null +++ b/dev/checkstyle-suppressions.xml @@ -0,0 +1,51 @@ + + + + + + + + + + + + + + + + + + diff --git a/dev/checkstyle.xml b/dev/checkstyle.xml new file mode 100644 index 0000000..0fcb9db --- /dev/null +++ b/dev/checkstyle.xml @@ -0,0 +1,193 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/docs/img/css-arch.png b/docs/img/css-arch.png new file mode 100644 index 0000000000000000000000000000000000000000..732e9caca9a12d5951daa325b9292d7d4132a5be GIT binary patch literal 188640 zcma&O2RPOL`#;VxBT-f%l#IwoHiv^EnOR41%&d$f+p$xLGK#V?vbXGcP>CY@*fX+@ zEr$^P&qKYxpLf5{=eqt~*U8B_ujjhg<9^)tvk)x}Whx3L3IYNGDwUhpwFwAFFa!j| zJmjR{H$<S%=zL*FGJF{s*#&uf^91@t zPhK?5U1GwVhhdS#XKp<|A^#~nl%f!Ey*M@RVrU^rW(CLj$w_B z-j^!Q)xC$EJ-?7&+Fnubz6%WFjaF*r+Xxn|OS~p|D=>$~BjlR7Ieqz)DtHsT|2#qT z#b{-&%geVdlAcp7lAWcgU8{b3n}%7DMW7=rM1SF$@QrUQPru!l&kEU)4549$)fuo; zs%j0{ztRYY?`+#gJxY`s>Y_8wK9sv^*LkTg~vx}(3NpC5zYuwUW-@LVMsIHU$>_Xfi^+aYEBJLbJXEJSsED9DfJ%Xo_On?V zL6p20O}TrY+k5Gmn-Y_d%by{T$@HF+9XHRNJ^j%R!b9w@aEYMgt=9#jG>8@f&g4m1 zYaZ=qmb3h4V_y&@$-f~>g$dNfP=7r`?WIbp`BrI@devyqQ)t|1kZ$_WST1`%(@b8y z>=p;x=4T8`*`+6^6+>P$Ii*4!ACXNRc#71Kew=->H(4L#n zNNBp;bxEc&#RDfje&PBe5=SCQ=sJxIL5ZT(n?Pnl?opy$vv2ZUrQDbjy$5``AASYy zA|yx*+X$kbDy6MuvCimq_;-<+aF|-m5q< z7(w8TR8G`|NvGD%xCU*fP|MStQBa(JdcpjO0Y~^#=BXg!i)J@P&P1eNT)4?_QR(|< zwR^JKq4^48>?4u9-xBIo5 zw>n)4u7ouw`IY&)(Kh_3Gw4U-u#xT<8){M^x95Rz78U*L4x_O)a`!_4Kz8%=gdEd;u)#7XHO0PrTa({34k1o~zgiQEb54EDiz^nFWS(F4UB&XS@xu&D) zLu6G=QW~x|C>`b=kOi5Mz9)qQoeA0}6+bC^PMGTBNtcswrv7uXLLb6do;ROW7ag{D zZO*ofT@gAot25*LcLgS93=A#zo_s^JJym|-e+?w7NZN_9D?@-gc zS!J7bZtO~z+P#YArAI?-YEZ6cacx(uldRkmm=l!a+uAGRD&x2B4BowThwjc&rKhP% z@n-SYG9!y2`ymHQE79TmnR?}Y2)mN z>PYv}!==nYyhM#Jf zx$b5zNiU%%4RyAXP?lzt9FS~aG84V%(A0W*PRYk@-Svyx&^PB!y%x7&N8CK(y{qm{ns(+y?lmF11xte;6+ zq@63p%+(3L7jN69^GE5W~ zxDaSaG!pbRs7n5od_)j`(9V15pybk}J45;CeE0={3q6q>7uW{(M& z#lSRUqu+6V^EJOYwbA6Q;;yohK6|hI`}!BXFKpThI{F!95$CVE_YbP(DCMMLc15)B zIuCc0B|R{k?hju)yDC}l`K$dz1#PB!l~A{h`AU;TlnD0&?(?^})3ef7RSv&~&FDLO z3rn11zEL=t+)u2PeZoui8bS|=*_s0!=SU)yj!z}Wh+90R-GqI<}*2-4M zF9(sd38j2la^jpG{z-R@QRBCZ!zGk`P$diBZKFmmg}*dc*`sT?Y?tM#nwBnY`PBYl zzN`u30jBcv@^{La2nq-#F9j-#OA{)h?j7%Wx6L(pK{ZoO@TA}`FH+ec@iec@ov&Wk z*xJq8hsBuV60g>Jh0nYpagmx;`Z@-U5Eir^N8S*)krz7Nt{FG+uJPXcQ4&oCr7QUY z{XDN3ChmT3uxT^*DYkjAcxIb(F~*1gTa3{ax6Ax|`*qQkn^T&Fa<2|C4Od+5EIJp> zB*1ClPvN6R=gOS7d6)Gb=OyK{qHEl#T-3kBd)RRoptWXiS6H66O!Pr`d%1e;qw;Wk z{r=+I?s<1qcUJn8N532#f2`TAX^tayjL zb9<`P0Yn=_{=Z%xw6zD_CQG2+Vab)W_w}4Uz5C+j-E?=9gGb`KLbPmo{qlTvT07MN zm9n4Pfg3!{%i_~jv_)HlPPm}NxPR)}bsoW*B$1RxLn&h`)u=ac_f2n%tM=qnIAd~b*>7rj(S0ch= zUdH$$KKb82tX_XLmiN)0i;|7;FFx#96U8p>AJiTAoq)HV|4H9ys(9@hp=T*2!ChVgS5L>Z z=Q>FwWW=|VOvq?{Y8rSMv(N|<+&9ho8D=d`=tLlvP#r0Kmf(}1kGT+RY%Dnb*1fnD z$6ErtGAB->_Vy;?31IgLCZClxG$3{S4`a>?4N4P;sNYVKr&oAn{iNV6g{w90QS~I* z?Ig{Hw)GZPo{MV)a$AQG8s&&g)9)sqK=u)8X`o`Iu1>%U-jfp$6EYEyfOmx8B}>Ts z&wC|89s;7H-$Mupf^7+ie}6{<{KWr>1TXwE$3KZ)J|Q3j|2hL+o+*$&-zLGN5dC>i z%mY3nkk?UEQ2{@7ESxPZ9bIgYuDk^W_%Tm7-Mr&MK)}F>e-Wx^Us?m>58CP)xEiRd zNm(Er1kCOs%`F8y9h~sv5Xg8+fwvBpu4e3>4)%^NQl8L@N8gYF@A1DDyvTm^6<0gx zMFVv$c15JKCA*k_kbuxdSqgS`b{Xe;R#Mv6m3}`C{sp~guI`gNACHPi3jgP$|MAh^kLtNtIx8X_z@)CS|0l5D5C7+fzaNwl z#P9t-&f@r>kA4eIT9!ga@E@tkQY0JSN(O>tvAwRT3w{DM!~cQMfxm#N;D3S_VRZ3? zXACO=0gOQ9y1cF@;nEmc6|LoP^Lp^99_gFP`A?~qsOY^v>NS)X;U;Ug3iIqd9u(M- zA-sp&mJtIhybPxFbhj^GJWsfQ_Ng}Md~}IBPOU>F?$Tj(p~cHMk6~no+1EtzPD49~ zd%|4AG%y0f|Nhq+efEu%mBo(=|NUe9b8iR&r5eb|*#ih5|Mf3o8g?z}fa|pXdjxXK z&iVhj0vH%L5Jhk){(nxu4om4+{U4L#A0i|s#}Ki16aSAbf|((1eq{gq49LL*tOOxk z|8;jqtD?#NA^pF$&mMrE07r0BME-vrGAsa=IPt$%M+_mS$tHuPp7`(Qi(dgT_SdQZ zeLgf`0w}>5_5Zy^cC9&&|G7GXK>P%Rkv0$i_aOrnAUZxDy12$%y*&Gco{S3fPZWmL z&KU)FN;5xq1bn>l?-_WM0*A}(4ACfDzTf_CfA(#TJi|)It4T^>;c+mul288gqv>FN zBnQ>OyIP;lj#(+6`Fga`TWSG9=q3T4te}4_ zJ30VX{TNG(_1|09uXT4=?0Tzs=O#^d z%qPp)OGj%9MVJnk*?#h)V3GUx1F#d6`3(oVoIdW08euTsM3T;9eIPFY_GKeTMfT)h z=fqBP8feA9pW{&A07ek6`abY4HM<19rTOr~mEYFQ^BfFjHaf)j*W5zD1WYx`6pn`r zlmQw&M;M3rYcO5I02q4MHUH`Fl?*YkYjsxej41wD#qo+iXM#z@M28rDPlCA#9{WdO z{@F#O1(=kDsMPEC+QDKeDlXx@4&h(hj3pt*eDraX5Ig=(s~T9?R{@EvXa5?EE)J~!1w!Z)Xh=XVQ-kherb}bgIsg zMf6{5(-HvN0?eeO=JC=(flok_3!Dl5bJoX$Mi+ynoip*#IDfpfcPD7FgO@{{Wd1cL z0i+m=R?H-${9B$6!5SE&lM^}qto``w>3HQJqO0}n%hjuJUC=fxHh)}TRNd2`e}}d< zcJAm)UT3lqwH1AN@J{SN5i*QPw4}yuan`nTiE9ANQO#hr_t(^-U{LDpN}KMBhVM+} z8~a6$wBvIYtkkMKUnPobXOwfV^##~QxarW9f1N=9OdlT#gwACC_NH2vJ|0Q1b zNEaXiktRw;0g-*7Bbh!jNwMwC%w)@)q6eZLw`+NUKdI_764TGG-52res~B;#%X;H{ ztMHVd-3WAh)Wu<_#Igd{Z#a@79|S>ew~J4dF`D>vmm^0i3J}g;IHNyM<|`{6i&7XU zu%?BWbq9-Ky^mMC{uL#>RLFy1L8MgqZj1e95_I8j(y`{vWy{-8jmsY8)EgGBpj&F$C=&NekX;hEt*ogKUhS4!;Rh4eviZP>18 z2zY%Wwt3*ymO{9boW7OmOMQDpyY!hO;$)rQTP7=}Y`MLq(yjTlh?Wya#=VkYfLr97}5M!^={qeypLi4LY#5BTz&eg?_pF{0}Kl+N>Mcb2u1K=?X3}*8i zO!7bQ@;ESbD1U`;X&23;9{VV3c_v#9htb*E?A5H*8Ma9a8_G~oEk_Kgg-N8KlNhIv3$_r1YRdnv`q)!nYmAg ziPyk+eVtp*dl>wfZGZNG+a{&ZcmbPOn4_62>pt@eem6}iEOd%R^5*do-jYu-awdb| zw0}CxumJM~fp*5(f)aK+TAdv8Y@_5s7k*A2VpnR()w&<{UtV(RS2`k?Wjx{s6|Vir)-Pw)FzzlXkqrHDlz7 zVKy8rQgn-mfT3%g^MirO+vOfg zUpxd_S9RllX}j3SpkTua+T6);4>!2q# zJM~PkLS1}JM2}Yd5Ex%+!~QZdlLp#XwlkYh+tmP!RV5*v0z@xDQS)UF7< zv{c%;MKwq3kwWt;`S4fPUia2Zb?k!QY@Bt?q}-5myUyz22(n>J7MP8$dOo7oeQQqc zaEJsK5+-7>)Np_k+r?Zn z5KZ71w4@i(qjK%7ao9eAEAYLE!-MULxg>9;h8db#DntU%1X*{6&SM;8aIC+=2b^=5 z)3B5};?|jF{apAf7+SaJ%G_h1SKl=z@y^S8rMhJqhE70M&%7=Z{nQt=gUu_dAfaS1 z03tDZI$G<2Tm-h>{zkIb@8>r%Zy*CiK)&&0RV5ssDay)Ls|1JHj!NYS)(YpDTLT}B=2DVy@u%c|IZ3ZK(T@EXJe>ddkgu@n zL#9R|){4(ZKf)0=nIw)(k{*aLMty<8CH&u$Fb--+FVoeYct(YxmY5{)UMg2FzIJB>*2Ft6eKde;#y_{;O!eONUl;n4sUay?Yx_XC{cZiXm z^yTbvpN~(d#_aaF#kAmpuDGOuN|v(vaODA)-OYtUkKGr_tmhCB+()`N!+)%cUWZ98 zTR<0;j&)J>)_>?CM~dmMm#P8}-oI1OdfbVqaa|vagUq^gNVbq<0pYs5FS?a|Nh69& zy=1Ylwu5Ae!X^0Rxha%>ONGv78JjM4e6*%?rGL$b(R?Gj^PRR9Xtj~_<1Oq1<68IS zDyxvdn%38&uD$BrA=X@d=3O9}dc|F~4-wU6*P01EpXYh?s(fVa7aK zF=-WEt(UW|d#Bsou}>)!GRX+4nDF!7rXoXI9fMHzI9T=fQ^eRWg|a{JuJ;Tb-YcFk zmh!kojLkq{C>Wr%T;MY#Y%xlA8+<-~4v)kjgxn%$P#hm)-aOs+^QM>$?VjF#~qYqlDH+aTNnv~-f ze!b>5oZUKa*fvgyDGx{a%WZZkswg@k6R@`nji<3gMTB>7Z$rer+Tz8LEAlC)aR{5c%{kg*gp_oA7~}MFr}3fl0Y09-`e2Dk zFu)P(+t9unzg-)5eqeUHb1IM5>?@Mc2|-L@Dm} zQZ0E6AUvu!1cxh7>fYNtRF85t_~Pq^M^Xi-+;Kiy4?-k zSS)IAmq#m@7Kznj0}fX_8lEBAw|9VZVjGJMRyA?$Fer!aJTsZ=iJaU1~U(S1=qfy&FNd{f_CXnMh2yHnT~X9#VNJ&wC;v|R)V_;_NM z>K)wrJe{(+&tRLKN;XF&0_%|I$aKx>IUiq|JCnT&&}FAQ8&i!wu7PyrzIhFCTbRb< z*7!}Cit8ba+1?{sHU+HHQgS$h0rp&qnoj9enTbF}G!B>D!@t^aSid;LU+M}ivXHw} zwb`TE&i6Qt-Q=y`M7=M9dS%e}lJ`M}^lDX!*xk$@xRvGNKvZOa$zf5-JEKED+aB!I zQeP@l@2iCWEV%sH9n-PLexrQk@MnwMeSM=80qa-gX1}c^@mN6?{Adlr@o?cOta@~C z<3Y8EL>%OS;Mr|fQ9&_@0hp<1B4z{`gK&R96xZi!m>+>h##+OWLl8%dvh&{Q$roT> zR634Hb*bdd)+pRp2a%<48EB~Oz#cGywDL9!Z$&u(fD2cu#8R9uF+CC)`;H@k*EX?B zh-qo&>lC6^a_Jj9*Ubl{je>%BM>WKA7%NdOc}ZT8E`3|Zp5I8e-&T4Q?g|;TWbUC+ zRoe@_++&XF&=2(A?X)92sMc~Wi)lqr4Uc`GJM69`pxZ>2_vx<~p17YmzK``SQps?7 z=o6bNwKW`GPdJG~MJ&G+VB6O?M)B;C8X(>Ur&3~+!&w#%xG#;7+>^w^Pb)&-8+VX* zHzZ%Rh4Hgx;$u=!#0<)W+5Vk;85xTx#-*Tr3q;JoDF&?^hCZlCmDAfr2Vb!7n&C-vl#mXfEF4Y0QvKU!O}8` z^k#2__~~r#Gcp(PiQy%RY@=9|ZC*=xCas3V`*4|!&rMeKMHY6>Ew7R5hAIz`rs`%;cW;m(S7&}-9_#M%tr)tgnlE>qyAR( z&;5o7Y%2iSW97HMM5}T6daKPi=-!`gqSbWG+gcnDT&X{Q&l~N%T|p&2+9isExcQP_ zkVZObH7sWj`~<+Zz`8T3TUdBJ_}nhd$68juk*A2v@uhKC4oGu_kJDUo-bs&Ds+ur! z8+ch@%>nldJoE=7Rtu~5*Yje5sH?_kx0ZaQS`OSmfD3(j!oIY9N=!Y4wdFk}9_u@1@n~PW?$U>dJqy?`X3{;{=@#!LmN-`Mq;_BXCA1Sx9#v5(!6>4F{Z|HFK z43mc^>#>#U0DLo3k&oo)a5v044i&es+5sNTDdXE_U}&a6P|A$^Xy}Q<3tE~4-#GDc z0LG3_T-6?g9|t^Nq8e$w)!;rnb^zx?Miupspa|0&Ge* zd(Uy|!5Y(1OfZ5JXWVr?tlHliQ4hLfE%oV?a}J@n=$ci{!XFqYs^eMko=^1CYvef* zo5=I7UGk3g6aJ3%J63xj=QHYD0(vi2sHWBhLyW+`yo4KjeOkUHUr%Zoc775gP5QWf z-1pZ)Rg|v`tUs&Db4tM5~$1e7vwo$1d z1{Vhm-{q|B%Fbc?Y4e(nA_$!0Ep7M)&_@?n|T#_5ZcrBJfpTYrNWJE;>P*z<+lk ztG47aMGXBt?{Du&=le_Ulv>3p+~xnl%%X#yrmwS0`10YG8pO>(aqexPn5Y9tZyZzF zL6X#*sA|8qL?z5L!2a!1T7>L<ONl&zRyGNL{`gE5BE6D$h zH-pAXJEYn~46M|-sO)pujm+{%4=@BNOynXDv~!|Bx;HA4^fYsjz27bm=Djxger2XH zkZAdPlxD@fZOO!7(w<)+doW`62Wf!*PNqIrV2yh=>@bwC@ZD95=BoY05fes)>usYV z4}PtJpSRPLs$DRC=*ZCmx*u#~tL62#YO}8-kiu|WI3qaJor<(}y_vbzMdRcux&qm- z?_S1OD|1SENeKDk6Y2N(oZWfv<#M>I*AlI7tf0%XcV0mS9xw~I7w5F>>u5q}X1|z$ zbS1)Y1#M7rF9hMU{wcEus-7Sw>T5IU%cGF8a(VrTWQz>J^Cble=wiHSnH`xHpl+!5 zIVpSsn8?+jqnnwfHBpdf1(oAc%h<;?tC>(CrXz}!?mEn@e1OuB^CqUPMk)&RwuSTy z)Adb&gk2M~&(V@pf}}6($v2m|4mZ&jG<#FLD0O4g0!!?U4;~2H#q&kOBlD65 zDe{5o2|T#Qdl+S#QrJf)#&;!2#;ar`lypRQ?YwkOaD(hqt)eZ!xyND~`GmF~`nL~w z$pW02nKVI>hy+OhZ{Q)w<;^I*i!LxjcKZXq)~T>D)&1INa}3a@Vo$i@89{~F+g?R7 z-P5zwps(>p@43nFcH&u)svo_LX;^xop2bnJG?IY)vY-7Lm+53u-fvNvRsmRo#2gPa zX-VHsuFP}NyErb0o94q~4qXjT`Pk57!=Qnvc*>9FfZ#%duPNcA4muPMyNvpzlUsL?A6!i7fi= z4OHf2Y9>3DT6cz5=;a7g-HOjws%E-;Q@<-;YIQt)tsX#nu?JJv87b`QA=3bycYs83 z%_bJa?=+7>>E*I01fa}{pZO%`eSeO6y{Jf_PLUk}qFa^Q6DsCeR9IecH5cyl0NDwO zqCQ%q_jQ?8?eWz$8@uc>541=8_ZLl8TCVk)W`h*%LC}-d_C5b6j$<{|tTQ1&8O@|v z?7lykX*fdeU>@~+k%(43w{OKE>%nyM%Ewc_1d;YV!St1*&Z?UqDR43a!Q3^kiwAB8 z*2+Hbt|T|-hTsgSl8$IuV&KL#IsTg$Q=!&En&YCv_Dld{XoTKWJFweEcR^=loXy)Z=H;&_3)Ed7dI4|^&R=}D(f^kMe)4)C519LnJ$IQED{*XMhy z4GomPda1T@$HxKd7qjS9M9pQ+UEuxoYRe!Y84Jzw1J+s@kR4w>k7I%_?*L{>a3$7N zhGI}}rKPADT~W{@zI)hFIga8(9Te|j#J^+gHqstz(T_s<6uGzi( zqWJDlY2U`QD;Q>P2rhu}sTKJAE(m~K#__ylV*Up{1I!3yGg^E~kI7O(@xr6~)wx-Q zTslCn;#fEwjtG!jnlQJIeF;YIYe>8FRiZ#}fiGTnGwBnr)6GN3*Pzhi_Ji;4j`~z!N_^uihyv{9alz%Cwa!}2ReCZ;qZSI+~v5CUD0|O4(Y-r4xVWlpLzklpeb)lw@oO0ptcKda!mpD>r=)8L-okN7anaV=XgT%_(}( z3)OQ;F6jEYfshD1%&Pa@ktdZ>I5m6mjcf!W>>-L1d9CFV>x%#vx;6qcH zHR!YLg!+Uufsc(z1T2x6?DcS}BPbDpC*R*u;+g?B-~QMP@Vt+VK^@a5=4O!l!s47x zrrJ7Gs1FK%TA17v*i<$(VB){*_tN1W@KB}dVndgPQX79zxJ^*`1c)4|@#h5KFUGT7 zYB(UcH-K9`0%iEpYL>j6i4GO5h2NOUzGgJ@HQ}pPmVvzSI>e0-xrV;q8|`ye5NqwLBzszTwdL{0wiL+#XIo0hv6^y|-YQErpEY^c5$vZz)B@Am+?N+q4k{kaWZRz@O=9n7T6^M+de&mtc!h)E05ckZ^h_=25`MjA9^Zcc zHPnX-_;Zl@f6ie`up5)3gNQun)s)+DG_o|p&?1PQ^J)oZ<^c6H7oIXWR{4L{M1i-e z=K8;_s?;R}&@T+jVY0wCOoZJxy)AfC@X7f67mZ9X&b^ioGRN}xzh#{Pwn%&9j4|Vp zXg)li1Sp#>5vXCiK-`KPlc}zppAO(ZQ;D8eAScF)Y;RxT1AvLgN#b{X;Lk{b55Xa4 zRTBeXt_imI;T@<%DS+x=(hld{dY@`$Q{4-@MGXho+Caeb+z8qP1%SN@{>;Y!LNG*~ zZuzS`r6qw!`CESv#`^0_d+b#Xrt~aB)u3m&`KrOE&i9D&x6a^4;bb8Ds~{Tbfgk1j z-=nPfhv?@Ue7m0(!6r98+mX09Qf1lpx=_64!zPG@yA}LfN2{u3ZaKn}c-M1W1de3G zkJs_{cu8PYZzQiVr57P;#k|s5t0}s~2Th3<2aD5?wc2+LK=N_l$Md|4+E{dV)SG&GY<RyFf&SljrulA+%2A~)e7j14(<1BIC{1rnt_w(Rs@ zw*eQ*uWGMxl-_~W|K1EczP2uM4@irx0*3gO1nC(Jf$fV^J)DjwY5B>yt@yh8M5>I+j=&i+E&HW4KAvFQ=OW&R zo)zBA_&m`;+1(3vq{nBDTM^Sqg4;YTxQiKRJ*DkSTLsMtgQkX*+YMBlIxqFzO!M)?je_zQIa0y5#{+k3gb?n%S)JjV z;SO{sf|=mKFWmqsOGLPj_C5UA`J$L|ydq2tYtCrwlz)eRKWI!WsDQ4*77!Ys2_%ni zA(h?t6NRq@H!eMX9ve1ZU|5L>GnnENIxZtuBbgbFIGg`40xj}Cohp>Yi*>6}Fev*D z2`U&2G7T5#m%30kh}vTAQu+PXla zk9cGD!%cvJcPcE+P=O#i#ryqcP5*EE$g z%2zd0(1`HbF<{_p#Z9|Es{BW(OE|g;5z_b`BzG}}jRMeeRR;sM^Mde(@^!-r)Z!2^ z_BK@#I}If8E^jsF8Zp_+1DepwTFDX*QYm|SU+rikH}Ob|9SxAf!fBh^{DJt&#zPXy z+;RcFe|L6}vqw@A`tqUQG^=oEQU5e_U3*VvXrHo)<*PJk33q@Ls^bqaqp|nUM_QZ0 z!s|QI_3@tcc_~*L0T1}JLohf|)4ZB_-sKX|R59@*L%c>%Q-k45?%bnSVShQqPlj3;;oABdqUY6M!x%7ZiVdG+c;4ZL7qexjdtggvr zDv{Ke0)Sl+My2g(&)C0hZJ7Uf3M9**US?QorQWgi>B(PllV*+h{y?Dvz%r#E0wvTJ z7zHKj)@rkZL;tVU9-pNhd_KRSLS-`0ZIpDZ7--vw9K&38PuTOmn zc{XkAG2iu`WPzeL4j4h;I|_V_SR~dbQOeE!Bb9g=zJQt^-1^}6nZlR|R~Bsp@aZ%O zXbtI%_FbM=8%Tiyy)Xd&?V-%-S2jqqQjc?cG-s; zZGvLHyRHeMij&LA^!FmdG)ePZ0R)gp)~UvZ%KLL16reE=8!0-0#gC09TjR-xecS3H)<4yD#WC z=-&B~lzsAHYYmSYyxk1n@y5Oc0U^Q-q&M#tyrwW{Zx8WBz%So2`fd#3-^dxVH=4td zaeuZ5XGsJiLK_b+1iF+AJ+Z<;KS-Kmd(Pr->Ki`c%T>}R}U5LtDiH>Dk0P}l=k z`?FU=u-9swpX0!%-IyJrWU#skWLtq{WVPz=$B?vtRsZ*$qheAJlk^&_ubjTBi> zX|su3@o=OFlyaS){O|+7=w?w)t%&JK5+G?K7*S|0e7DZ)M=nbasSdtuKi1%($3Pl; z6*!^+L&RYPmj=EujK6ZUN0;KsQOgbR+9e!~zEJ6KdlHZW0qD=VCUx?haC*tim2HXa zhfG+)1#lJ}B)3Gh;0SrDp0wmLba#SCIZnD@^+N^s!%&Is48)Ywgqud0aSzl@Jitwz9F;|?KCyX4W?z=vfr zx;t!$BjOyNfLeScG*Fuo2C>Gr)VK?e%H2Q=?W-k0Zq`5pTFfDvm1U-l7oo5e5u&rk|`gE`%rjP)hLuPU9N7pBNh#+RX}l3kDn2;M-@aSsb|g zYv%_-h4Jx%U4Q=iSnmBaK1`0|Dj=~a54UYH{W9Ua9p?uJXNtSK|kMhbi zOq(hT{Nq;GAAu%}s^`q&9Uxv}_PKzInJ?%Os3w0&j@jS516q1_Xz0b)^N zBWT@OL}hUAiQgs)#~zmByS^N+k=&_A;qHTOrB3Zr@xxh~Y$HHsl+N}`bjP}-0K_c# zNNNx4`yIaZz3xo)Net-zvEJJnIU(lv0rQ!;{PK(B95IiThXKk;*N8uGbk(@IEcWHO zq|Z8VFAirYrpK6hkNEA4OUgB#=4jDhNz3w9`{mufL#Vp~voxXuP-FJ%M1A{Iuz+Msz)}^PFEBZ&Ca>(;!EO|2z)|ayV#K^od7wf&PyBF z83KbQ4=+&?5^r9Zp6^aW#3tE~J_Yr)(TX9fgw#h&vOZN|=dN)r6yxJDu0pfsPdKK_ z0n56Gc483&F$tetu`fnmP~Va0Ug^Ds zxD|=-0u-iwK0cvnxQ6Lo5xmH_aBd2epl4@kv-e7h0gehafE^0t!cis2#WGOf-&oFA zE|r>hzK{)ozGbriLG^-SUz5tT=@Mvs>j%#M|4eKiXq$_PV2#P$Q)dHN0M%hlQRnWO zX{T%3;5@@SV%DOQ_tlydk|D5`T)A4NUl4D(FY@DDHlM`p$U=E6z9q(60Dwbhc@)_1 z^ziBImIrH~P5ahO7_PgI4)KERkku2H4;H?y_HzhvUE6IElp{W*Mp6|ijbD4=|9@Swe2Daxb^0gY0?ow_l+PwErW z@?Xe^SQ=hr4PALA1ike>>I~gYhFrD>O-b%U8QkAz?Mvyh4?(xCz)g73$p#fsXZqB| zERbFKsZYQDBkfS5Nf5uwKm+*9%oVy0Yrd52Q$IliuoJosJy7lQ_0#&?hUXB_dk}qh z0RE-+Yp*UMt4!sDWN-mOW`VxIEX8jQ(K(nFQO`QIh_97{ZY;k|QB4{iQ9|FnsW|JT zkfeTrT64d$tii~lbGMjJN_RFIR7hv?b0iOBE`K|uwl0tMm z02YmWHHBP1)GC!e*%A>Io|{<8FGZGX$|1zHVm>qz5|K8EPtij-FRium6%F>mP0mBp zjL}jf4F_Jnn{N8ax*S^g5;y|6iO+mkZvoyX4c%>6et%{}Y)oO*U;D>?W#{0@g~F3< zpm|dYV&b#@IPAQ1g;btZoklSk7IaqOc_Qfa4=a*m;IqLAmRKdSjhoajyPkppfuG2@ z8*u&2GmrU`A88)Qwr3eT>8&KkXUz8={eqSzF-n;??m!e$;0K56=Sip`fa5F}c)@)_^ z5MvGPiyv3k=F5uN^*>*CF5zu$K2k4d{L;`@wsD(EWw4ZJz`PO}kJ*!8efJX2RX;fpro^Ppu_b%YsxOu&QtL^vo-w=hv&S@Ks|rA_k_d+m`oJE1Xa44HAwoRGCNnP=)K%Ru23dn7 z8}!-YwmnT^W3D=&t9al3{X>#^dx}nP9Cqfzrj#2N48MgR{#gkW46lbD9{Xyh?nnr& zxADUpyLFrk?4iRCUvuUdn_d|R+|7vD|2SN3I5AwdUgVW0EMV;5`pV-n=6Z@TkJlEs zA@hl=k$Kg;)|^EUZ=?i}9DuyCD$KyOH@X-+$R2nG((bFn9Fb8l--Vu600m73CAYQn285 zQL9t;`4;&-QLSWI-?w4RaxGjMzxv=6fJE;vwbq39XZY@I)jRw^N5Y^JQ%gW8LiGZ~ z$3LNFBpi?JAs0lBqWd3Mz(L@z-KZ5YAA`e`=D^*JnLH!p=7Z_k>!>urAM_4@dMXn~ z0UPukVYmUgWx!Pe4q5c2Rzpg;~ze@oqMinAV+``u?{;{^xOJ%&m$!$-$wTjgR zR@&{k+F}TpLXbeJ7NEdXy2&t)+s0z2j#=oybg+S_5G=L1!YB#v&yg2im0DG|$kSuAG)1OFAvwfn zh;ZT)RV}aFZbt%88?T#Jy#tDP?~GsU&L>Lk_cKH9SU_<1c=hHA0PU)0Elh?FgvCJS ziark{M2@vcm=4JN!r69kVrAgO;*8_V)2<;n3Vn-7{2!2YQWEE0jy#*43_7>#4e$3a zRgTX?IWiDT;)Sat`jpR>S3_B`wt|@a)`&Q=m@q>vYVP_98)wk) zfvaH(iM}A;yGj8J@@x@P%kP_=&S0=Fh*-So<120pq8!JAT>{HzybB(Gc|BATzgQS|zUErgIF&WguJweWmfG0xlg8Hi9VgCKpD0`0bn=*S#Jw&S%{9EWEQix z$!!}gFcg=bdde|c@k>k?R4=^2aLXWaxh)DBylWh;M6Om8gPRM(7lK*L!37)K$n{oB zYMHfw_GGyUK!}(F>JruJp2Gn8wivg%mD#F4frji@hj;HubK-^N6oFg!6ap`5W4yxT zfD=x<_2js#h@37NoX=FP6$@xixP-rjxAGF@c$yRo`moAX1<@Elot=gj0RlJ<;*8XO z3I#Wc@YfZL*q7V4S-CZn?D75n>ajxa6D;wh`71uSlF2|JVq|N=-@kUFm2a$M{$VRO zxVHJj?be_F3*d0=Ben8C_f?yQ;*oh!1vT^s&1afM(B;LTQ&OKnAJfV(^yg#{{_`zWHR!q z3LBdSA*a;q=bZ_LKvciV5IaauvSCxqt3GcsS872O;&V&kAlY$@wgjjPoi8BH{S2mZ zls9=jyQ1~~G4>u%O>N)yupm_~MG#RsqM#t4B1Iq+QBaYhRHaD~L`pz<=pX_jpdun2 zM0%HALXloWZ-InfLMMa(p?o`f?|bie-~SuqjiIp&O>*|xd#$ zeT3}@Uv~nx1scP$J|(tFQC)Q`XbB*R_9o=7K$<+`I&T;Jq-!hrOx1G2n~rPw%KJdB zZUeMp@iSLZgU$BWhWdtWQ$ikznA^%-rV! z7NMHP8GJ`qBC%ya#c6l}dvE?l&z(hfP4^h0TgqkBgtFYJ@B95SATX}r+=`Gr$fTJy z(+QO|EPWky1Z*-TME~1)eVb`mu`&yV8zWn$U?%8CMXN$k0gr)OvabXgu z(I~qo>Ic{!X_2yr{fh!jsq~zKvqoH!nXLe%KFxlO_z?t?0lj8oxej919<}H3{^B~D zJ}8T^);hB<&-97$J)VK&w6IwKeEpliX8`R=&a^2amLxiy>qE=`Bg$Vl4)`o^S%=*?!cviXoxdw`QX4s@#^C=NOxmPV&q-K3keg|DZ)bBRwKyx+!=%V$dnFeIz3Y z1nZ8<9S!`v36UB3y^#O?@=*t13ZXtI?N4T^1wjp;?RI{m?XehikcZbnx6aqnDnI-H z@Sg_iwgs*gBLjv!m6)7zTcr5r?w|%z6X6o&U%B)>)IdI|kAhTum(+RaweEvlZsRlq zEeYA79|iGN+}h+_KyUter7uxHqYi8cRY)kj!2rblrPpmMrLaM34c(#e0+0=yTm@Vm zLEw*jJ?0TvOTtZ~ilN1W&@=g)H+nwuctm)}X(oif3U!QnI1zT&PD6DRYG57K z7<647hx^}nj!koX=P_Fu6z5Gk)xm76W1=_vPC0_*1exS?9?w16% zudk)RV#kq6x(Vp`Z*Nyy)KKj4R-GCxJ&i@V0 zd_*dnvOsDV=mupOo*a2Vfn%$i0J6|z5C>8Fs-H`LYEV-SsabaH{gh)Q_){GYBeAj^ zdwE@C2o@Nt-X49oz1a8Aq||B(D3dV>1q=no`m(@j3kkUMIR5~&1v&k(Wfn-TLf z=24UZ@Vf5;DozQRX&yqsroG`PG8ZQNc>5mD3@`*vQ9K+n9TL=i#uzxpdb?C1fa2ls z8CgMbjg66lB${Ke2|Y$m8xS7G4;CMSFYG$6?__|qDOT;5Hw;M2W+6;eIvQ@+;~3TB zwH%FPL1wd8sQg+0I4`X;N-4XJ2#9X;gEV8UVj~imO$Y}R=^gO-wIOY6`pS2DQuA+; zS>lryE#s#Mdrh7=3#RRYR41m z*cmf`7p*DE>iVfi7kVrl-ZSRMT7EG^~x$%&8nyk)N#&-@A4|B58$0s#DHjgI%l!-0b4lv2K2~MI}R##md$ z0M{%U!+6>`aDtN4EqUeHF|ayU^vbz-7>%+crDdKab5@6=9my922-|>osPZ+iC2IXW zKuj<&bjH1l4)w>Hr+Chk9g*UIaRjlu5C=_PGg*(y4iP$A@(Yc<5KTks|-kXq^r1?IW>>7Uv0p%i~<_im^XePqrP`eioDk4iH0EBWf= z4O=r4aoHjHPX$>=HMvMLOG3SCu8G&^ia2uwP_Pc2%8VThFc#|^GBa@QBBmVZP=#`x z{L!dcL}`&(YZX1URsdw^vdYj)H1Oye=DC$(Nu2}NFlmVq*GBwc#GZ9V-zWrhpKo(z zXs>xOWa`F+GygxfEpgt5Wry))zk>uu}khs&bR|$pYHoK?HnE99Yc_6 zh<+QOOZWY;%ooM=i7t`In58oPMIw?lTI;KBQ$BDVU{z(rx1Wf$(=qJQs7DI_>F#f?H zF^_jM}vNheA@p1am#&)F)ZbK_j};W za`S9v)~)CzDqbJ6nR_f|Fy74O_XcUP`qVS$9-=?%IjCwLBOekii#V)X#Yr#8=J=VlR zkCM58%{_662$lsoQd@x+Ma&A}PTP}B&jpsSXffGnCHsDXS5BU+jU<>KF(GS;+$YCW z@l}ZfcWwero7sp8K3*P@ps^m!l(~qKM0)g2H66IBp0rr4QXWm-`deW0--%A%gj`Ff z(g+R{CJbH(1NjFKYJhxxWN_K9>K7L+LpCQG7;iqUWF@X^MFU#q*VsOGQF+BzP@b6O ztUOrfMB9?zDYHAg&HC~F?iPRxJ?oW=oE}RlK?d}1h6L3mj=VA!tU zj}Z`$&&-I`XTd&MPYswyV)AmzJwYkC3Iv>|dEttzk ziqm?&9|z*K+=|WW!?c7Q1@?b5I{zKE-s;PzeHG=1zBwRzNrV1snec;z<>5?8+ZXj> zvvKp*8c)}TR?>*ROjT61>&9L{h-2G`;|BiB&TsrNX0WA z2tBMeRipI!eAB|%BwVhjX$jwV?%)Ez;LG&Sav^BFYTy*nD=Q(rW|l2)C@)<~XwmVx6h7@9*(N%#hQ34?YR;Seu0*Ujy&UCxLIyydoGjH~$b{{H zx*BeMH#zZY)X4}LW?!OUos)VXN3a$4Z+Li4iF+hP3Lx#3*gV$W{l5m8KPNd4KNIfN zd67f%h`GPzU73^@?Q?gcD@eW^K&|{;*s6RqW;*YukbrY?>}vM@Bp?YT??m0)2dXF4 zfKS=wyC4;<0ut+e8DOqd3{;(Q_PWfB>%B`lz3fZ#33{KNE~T1EBh)Kr#W`P>p^n;^U|Sc5o1o;!TCCbETIDX zG~x|JJ^tK!PE#0#iMBnbw7bPE$lOOoQ<`( z`KYZk1RPHg(Xy0fA3=r*EJm0k4lKftA-|2X{}DStJLVOr^Lsk#0K0u?c(p~|Fav1F zj|m&D(Q3h3`SL5Oo>Myd_bYQ0k>0@r3$(=KF2T)0!Vz2nVhF1EEX zB#keWVayDgf3(LHbi9CT{KMD&cRT`(!z*Cm+Q1JpZj0ILfbkLFqi_N(I!62MJ&XO- z6wYePD5fXDct8wlWs&Fp1b-pzV$3Qp@7z`zMk_wb?eZjf(c7;|Jo&}5(gA58V!|}E?%u|`d<1*yR^X8Y z$}Kev2;QwRoXj6{cHVc)Eu9W%k~a*~Qc{MDzD$zL!dJjftmOxq9&+8i`fB6=c+Ffw z#a`IhKwNNTM3sx7JD1mmX;rCmP}L3g$Y39b9+mlhjW$g-%B&MaNDn9zTeoKhynEvB ztn2d!j!Ap`Yn&b^=P_q8Ki?yYh^nhLU}QksvvNK^=h`uHmWoOhxMZ&|2w*#i}sa@6R_@Tz6W3-<_|2a z9rdrP2v{U19RjpcL{KV#V9I*N%ZfMt*9bnq#P}IF1MUE^gZKMG7`>>Z8x#a0O)a}s zPVGjyE+#+Nw}PH3GhPj2%Lbsd{ZCl_UKLD6+kOuR<>vYrh?g4QtREyBU2xrfK1d4Q zC{?ob;9k25+o6_?1II@o)Ye5@%IdHo0UtgJlIHJ-hL@SdM1N4pv zC*`LO8-_=a;ICF<_vyb;!EcI@6!!@D%XaFnm&kM55cdml z~ko-l}l}SibYt z8W$o+`<(xC zMCI2huLzYvy#ZU*YE142b1>i!XZ-gWF(;l80dPjZq@<_rWW^J+$!A_?@)PdAX^_Cjtpj*39{u`=CBTy4>_-ljgOax(tqRq ztueeR<@elrST8|aFmHXESqHkzm;*2|L|mBC>-xUF@Y|RBVn1FI3tM z2zgYA{DdUz_pa2|JB^zUiyQrE(fzBV<ww?LC z@9v!E)*QOErkqWn7mjQ_{lIPwNfkLLN-9E zSaaQG#Cwut`}bf3Krd3=P7sUS7U8Kp&kj^jzt2)GIHv_A+#1Gs6%d@?-zVCFw@ylxE%H$ z!{cpp)r;q7Xqa!G6VO2D&?7~Uor=AV_sgFQ#o1N0gOPml=CI1%?%L#^kVtY^a60nWU^cX02>?&WNa?@}PTM?QpuKdd^7Z6F0h#yuaC*t89M8-g^z+2Zy4r zwQI=Ia#{VI_LWMeOx-su9`rb$69RE-!w)rFLn*68()6 zZZ%auxz-`5wrylFU-Vm(DQemjTrpzep^(g3CiwZ#jcPEm`o`ZgT#1pPU29to3TEHz zg1HP898ix#>V3IWyMG3($}{?jaGeOzXj8TU!6Kj4r?_WebLIa>D0z<=($KABiSi0b zpc*n0b2AbkZU5zzpnrc3no$LaUmL1H&Gse)Vkqpgw@4cHy*%70|BB`Bsm#9}026GK zV*w-5@NNn=y@&)Cx}RtMo&nibSwC2CY&oD5LDaU4fdVKJd*m(+Lh0Rh7In#`+3`LF z{d5~w)V@zSXyzDCya~h*O@csp-nC<;nlJ1&q2}%<|CQXLC*CTR_e4INRjdP4Dd)lJ zSC7Ap7FyeIjgOv($Aaw4JCBZfp=KmV9Np0R^z1h==WH*otSsp*LXr3~VPh@2ExFHD z%BheJ{7;LcveJZIH2QVM#XXni?fG3a7**DnL|c3Zj6E&@^BB7?K^~9aqhQ6S)-2cL+OTwN0G$w#_SB-WAu?;qG0jLn{3`g$F}Z^N+Ao35;MuG)hRSsy${4%!q zGQilTy^by2r>8&8@dACjT_4NC_Fbu&;9T2iTAq*eAm(~r%|N$;8Fp4!-jiWl_tA}9 z??}F7-DF^BAVx8F#yy_xlVL3HhJu!gq1X`i=DYwU&`RuJUh)vVmWgZC=*pP&m|d!K z@Ol-I<8CR}p5wM2<^T%a@xP;QDf1QMyI$_!DM2ntn|er1~0!EZy4 zU=zFIWyb8Pr#$0WH7OpO;Ig9x;e5iqx`jR%X5A`o=_>B-$W|!483jAAS^=#!mm?HqoUnVs&zqso6Qx92)jUvkUZ(o5$0S&ITzT&&uSBkuHkEy&@^v@JSh=AxN+F!Kngyul8So;#U|y*IgWIVcDo%872(S zN{ojS*4ZZC!HSJNFxT>Rt4Rr`FZp>OQ1HAbvn|4NNgT^r+YKrrk(`n{5JQfYDCvuOnT0x+4n4f>4a2BuyL9>^<0H<1ZXzOjOdW))JD+7n4qsw^!ZdlXu8~V3+;k+Z}VHCsyXK8kOXA0?n7d zV?L0joi&!Bg9vOQs@F?&xx5Ak9cyN?b8M^OyYGcnJh%H5PYQj|9tGAOle6j-jMt6g z1S8~n$TwOOU5)xti$>L!>y=0&?>!UUmic@MR0%Ft+&MI{YSzW$D1<+EW4#)AM*{u% zXUT_-hKV=pcgiffS*z#PJ#rCr(r$6PURomi1Ne!?uOj*umgt@GFN^NscekGv8^px! zJ1&mHp(m}e_c4-uYNFo9W5S1%CU(11J`l46-qK*~&9j+{iZNbuj{siK{TXA6hvQL> zC+~p%a6##}*VAu;CfDF+Q!Q-$nRRcn-&)m5ekIz(oE~JVtNWjq=R2R$<3=ZI4>0`V z2XR~CW-aj@NtI=<`aX~H$tWjZC5yfN3;`UN%==cv{QVkJX7ve2^1zD+$dlavRUAo^ z3`x|zm`cfS#U};*RjSHsR`Mfm9lm{ruoEV+TF<0bn2L~X6*E4+&xL+}TwT3=1An~8 zVLkr+enj`;%Ac2VIPH|SJ)b? zSHU}{TEid(7HtH7%tc0)pe4NUDUPB#5ho#s_lz1s9goPZ6~#fz6xF1zF=j(8eCqZt zxn7q_Z(7b*5h+5T-T7}lz)ZXJzpNWFh_Ef@?yvURBWxglaMhM)eM&asWBuaXnc$)( zm5KR$KXkP*+W7rWi`9~W*6YR6wz!a_mT>2JE0=LsIi&D*`X$5U+DrYXjnf=-3XE*m zq@BxbFF}sY_=QFs)DP`GsB%j5JTJ4hgj{l-7hAIVNurnWfeV@{chV*0vSy*b_A``l z-ap<#EoO$|y|z`N`AYM8X>Pio1lZsGv8W}R zQA-W4A}Sn{sx6Iy+5}iPCUrxA{w$*dNO1)&cU`^ZH{gQq{;EWgUcK!{X*-)&PUpE( zoUZ0La_b^uX;fV66o0o$vI4f~nhln}SyRf%lf7tA1yht>`m|hCW9* ztkt>-TRM`bP1to5a zr{j8(L|eO`PP&s%#A<08ovx;n+tb-?gUW8)g z?33AoheR}tYhL&X24QD2p9s&@UY@z2pRYuynEg&ONy##9mr&(w*zExUvmeShoOj1u z>K$5FUOF92HM6WWtQ0t!c%LXzjtecuuRXf&M*9>uh5w2Tmd^znI#+2r(PJZeImM{L zT4U8?_-KdFRw2Mt$7LN^dxG)~F_*Kw#KrEqG(O}dpVQFjY`hhBo9e*F*rQv?N4zU6 z={vN0nRAqH`{ew=DcHsd0WM~Rl9}hb{i>>?-s@mEZdYZ_%|f}w)T*~Ig-}>-R&Mu+ zz9!g8c&qy1%oY5UTRJ2@pHv*)W{i&UT_cvl=Y>KczNZq&|F zXv$%7i&IZFzPy=zrk4!uBchG`S^t&>azJ9mVG1Ug3wEiY(6sozN$vX13*RQ@IJS zT_+R2j_ypInh$icTlWY@F_!K*F)W$Zg<7Ran``Cy83*s;@I`^x#b=?zww@lDIm!f8 z%j0&Ol>F3sjQTw zhz8m|ROqquEKVVWB7bc(uev+tP9qFjogny$334`(dkXZ<)&nA2?~`HNn}iz-y#Rtr zhgMu;!1GN?GuwL%9DGt}$4wVeTvQidHp)_{XX%M!G0dfpclO!g3tM}$m#pxx^<~V( zizS<7G(8|-kRad>@%)pW@P~f2{zsYx75$UdHT%n)aQeM!%H>u+l`(;%j@b zTgPrLCPx2U>$T1=i!gG(vTdR$SABTQa}D{@(GN|zIW|F$yvxl3LI9|4_TZ**K=w`R zS9_%PLEjq8@o*?;-1V1-*yvu~-(tw*cQvtdpt^ch=f#A;)9u-w5p-LJf)93oQG93I z+TdzFEv7pUC9}Iy7eq5+=ApumOu37e=1r;jWe&v@EvMT}+TtfVcPQZLk^Zw1iJtu~ zcQ)mvk7BJ;;|RiUj4OB;m!KRM50?Ncvd1Q|y{UtPA>l3)zu(#12(gDP^* zR0-zLkEL}DmWO?7uHXp=C8GItDIJuTT@78bypEGj?u9RBwiOxvm~2uNc;fhEu8|_@ z?UQR<8*HTDJz6mCZu?$%%Ol0`iqq5gi9*Am*=;}!%u1a@KDnCfefTxbFiVo7TGn9F zvwCeZDBD)b<+fL$^j7u8V@M57A+I(ejj((k4C{8y{s-#eX|ud8;eSl>a>=-Um_R#k z=riv3x<0ym*+=wem><2pOmNyyYne=X(SQ$o{;?95B?~^Y9dl8jIH{k zkIUttr^}*I?wU&lsIm4I)eyTI)9slijjK@0gk{($8sAf$zxw_p^;jQn}S09(9(Hs$F9elkub1>P6WKUZ|h6 zno$_7P$~M;CYn+S1{Ct>+|u2sXb#t0 z_aW(4KmNp5{M&~!Mbu1PLeWO1kS?CDqU>~amhUCkzbz?p`)c@hi)6BR6cn@f&rsi* zNp2M7xZBc0ST&=QJ)*}M6=x_znkeE$YmOh;Mnd;P_mN^F-@Fyyee-KgdQrSb?i_q5 z2fV0H@0Y5Rk)HjS%t@t*iMgDa5njRPGu30Q6i0woHiz>lD#x37k7cNjVQcUUab1QZ zhL+ewiR0m4jb`U0t=nD**3fUa^iD!<)=a#^>8um5NwybQJ{WfEm;QXVl|B+sj(RPQ z9j3Z-{Ihc<%srRBjOuP>gK&e?O+695HVTn_6BL`ZN)2UKtKl1~F{}*drF|fKz9B}{ z=Q5cZF+^S5g8qdx%vdchKPuh}lUR7jD^$kPzehWCjnoQ!X;d2-?ymCl-RIM(vcbMo za{<0&xEE{O3Xlu@3!jtK7$31+@{3OL>^E@lXcyPnBQHmP>bT~4GU-Z@h!#=z>L*B) zPF04JOYvwZau<{-{AJ43L^ZBat-tU<_Mi%hgdE#yaiXu{iPqls{rw8Dcz+BhI;=j@1c~bE$VrY zC6W0av0qi(a~mPM^KmPPkzlc9Dq4aM7nkXBjyP&8c|DL`p*Icv0N2BlK@0Xr1TQ*l zDlANn7#Cc1DQ3L>B3yE<`gq;dqOCo{-gW4Ub@msm-AP3G=w23z&B`s54z(4b9{;K% z@0$Mx1A@Jb{=6OHH5N%zP0#*i2!V0E$9O&xdF2{Iyjy)EUyVJL2n=H)^fbz^J@|nwKh~>+M-tC$gqFJz%=^Y5H zMsm0=>g+&mysI>S8FeFV&(S+!ZD6mZe@vu&YjQ^Hx4VSwLo8DMy=2BV;Viwb@QtN6Txq`Y&E+4X|?y z{R z#yo5?W-&4;kYD)=#xO&h;!yYs*4kqoYwX5(a!iY!tw4L2r5h%Zi}yweoSV`3wFQncha%ge1`=ntD+!AY5&ZIkxn zV_Q6;THAW40cGemznm%Y!KMGnS3cj6{uHFGb=_OM7@vtkxo(ns%Zn$Ppbs@(ULM&1 zm5JE+nv-ive((r4o2&w@lLX10H29QtcK~$Id_G!3oQ3Fi)&A?69Uqd3=+cQ6QPg ziAGL2yKd^}KS$u@Acw&(A1J_`k!8E_9Z_k?;Tu~TCYDft*=D9Y^`|biQs~<}DY(j_ zsK{2nw-vPyi}94!7-?o&|9MxA9A|ut(Oswr71^5aS3XOnF7FR>v#|Q|F*%W9XgnVq z%;8#ShI=M4@v5<$9HO7QuGt)UMQP?2~aqOOvtp=J$4VZIR5m~j#1799rbuzZ=#s23_XtEGWRuURu0e6r(gd7*stwout=2KD z<-0eWhlfU4K%@RZBRY9~gZ|(;>7|d9;A9Av`QNS8YSP?Lp~m1ozEVETYrl>1h*7sd z2)n(eyyHo=+=@d^s(s+G;1$LJ}x)l?Mqveq;mv7Nu!XlkJhBW>6PPw+X(?y;>CTwcZZI}6`KyG2|^8}uce<74LX<^w2M7muB5a1=YyLAobR zH_b@PBkFQ}{I4rasw*X>GJv!i_A({3TJz{=ICLjRyjo<1Ko0 zH(krov?ZYtjfqr0O=@J&`uEsFJ!g0h?M_q1+PWg)iAx+m@*laY;m%`@5rmC6Q+2m@ z;iu^(FU#AsT-spB2sQiRNXsO1y8=}#acKSRaKJ;8?z`AM{3IpqEoryVVwd&dZsD>s z2w06Fg;MiMcCV2yZQ!AsC>{OC2=-UKSLGDVfW^=*Gh}eOFT7N@kb>qeKK?f5k#*-M za+`B9J9=|))$$`>l0=r{lwnyyi=%hFX|dh}N6+bF4cV(I9wa8S3=*7+9|EnVotIxP z_}m(?sc06l4Q4+Me|k3ap1gk*8Sc@MB09vEs)Cza0e;Du8pFIbNOf~L`Gej(m7o7d-l&-% z!dBu!vv6}x-%w4%(aXambtKdnyH$5$+k?CJ#;wa|UFbT6L*18*;}DN^6aEUxvPc;b zCpSwI#>p;hCLX#ID>x5dM<_?^<)d;27SXV{qxVd6$($Ghm*WSCcG;${_azeq)npo8 zkk;;kacFys3@epec!m|`(8efaVa2$IsO57Cxc3%KT$tBAPRx9!mCpsR(Hnaoi4>6G zc1DLyyA6y_m;G9z7c!fAR- z;)8=5)+RK8aBf(6+V#m(s#f|*m$z|j9W#xZU5*BJP0ZA ztpPk8hKu0_HlREl|8^^9Ro}W$w?g4+5XWity*E5ad%HyrhYIEioO2U1VQ)5{hb-J?u$slt^AxUzn`_lt(@7oMvogpuD~ z&-^bMXVVj&jFSd=jrz|d-6ubL7X5VGkUf$%DuChsr8aoFiR-4@cGT_ z{8aH?Qw&(BdIhK3FX}+u^g9Z40+|HXtSP3wD! z;O#|v$S-asnMNcxJ%3xX^ZDoz^uNK_4h=$PFWFBLY4%U?kkvr3UuT(wn{kq}m?{mw zad~vJw`vjLW5*zFo>S{3IDTaVzV*(1y-&GIMKfLR(>KbX`-t+j5yWZ@UiJ#t7g6K1 zFFmAG*JkPGZ%tQOcY8^oZRpr#Cw3-JPU$xzQE}G1aEVc%=hClUl3IbUR+x3?pzhGg zY~5p1j3;o#-2lz!c5@KT*TLgxpk zKhYta+%h+65V8v|5Jso1Pn05HHA23QyB>oCY+vPf^hX|xsV)A|f>$C_^uOx|YoD`T z=lwLuKoY;D;3c}@eDP#=Tg7MVBnxqgo7A0n;7tV|XcPozprRQES9touDIQFfE+$Fn zy3;^~w`9U@JZpKI=l1DF4N3-w^5{k%JB2lv>NT4Rs}cI@j=J@eSDq5yZ}>H;i%G9n zTEZXRkolRo-Lz6?m( zFclG*&Z4uku+&7F2-cSDZlfSBJ8$XL%wOnZdJ&o9ii>ORMUH~I^0VGQB+D|^>(H{- zCFiqqR)R5-me*IsakYfUtzsj$qbJ0Q7^J^4&1L#ac@rAT(c-)i1uomljXvc#V=mYQ z7i3tZaZ`MtG6#xp6B;q$9&{1-(_FScv_t9^$)5RFC4H4uz5*%}flBkvhZ^c%E%Qd+ z1y&ZxOPB^4PX#_xF>5^SE>q&;j)`>Xvw!Q@K;dD&HBZMpOB=no`9e{| z&#m#CD3RmD6hj81_5P5^Ge#S{HRNq*Tup0{-Zgm3Ryqee&F05b(87ksR z^BFk>eUd*liHXk^rCXL+19nKtwG1`2m$*wRs|(9Ph-4P51O_N{Imwh{ns3y*pHd@> zvndX{D0{$qc&|MSDl|m`;T$weg){8fHh*Te&Y(JcK#c9qhJS$@eTQdrTMm7=^`L63 zWRFBojjq`K!t*aBzL!S0E@v)QxjWhOP)VednRZbAL2N!W{8m;HuaPmUlP`APR)fAM1o zd&np5cE7=zeO`C_SG1=rG!a5qll~zuj%DDmt7sMzXjo)Su}2^2`q+EAG;D2^4SEYA zK3QV^kRAJ^4x_1O&?y%pEbOaS931jqaw~YPDr+&1;%l&kRHH{ThskT5wb4G|miLZR zE!1z=6*!zB_|3+JanX}k@E)}qQ!SnzX};ZPcbd1|piZf_zm6jFtq7;Q>{j6$*wJ1yDL z1s82wy>jXO?XMMH^S6_<$mQBTOpO>XS_>{#9(9{i=oCdq@h6#+drO^q;qas>61p;` ztcyN5RXlc%o?~;9geGc+R<>p<@S{*+7PlusB6yCWGopI#DYjGA>(F>;|3f)~fsuX| zORRU$zNv8E@@ah@iF6j=*mXuEEITsl4*%1p1ZnFUS{)4e9!6hSH8} zsS#r~=LS3B&AR!*Yb<_G!I4+d5E&9FQ75}sPvX}tE?3c0P`m0bKj$8qoVYl=zdU+5 z^Q7r+600l;Rp+-FQ$LNjT7J>Y5|8J+7u*FE9t2V8+cjRk5Nb0?Sd3d~62mQ`Uv&!E zev5olK&n`ibQmoc#{wTIKWeR)lJej*s-|bPF`f^vzQM|0=T>@=^JS;|&B;z%UOL$} z?jNCOm0SZOx9mx=1*rhts%xXXA~M6`lXlXM6^lZYe)V+@{_MIM$_Z|#r~0HfN>c0# zKVnz`DW7zgAmc=^UD%jyi79|LHy^U`ow(IMnW&BDR?)x@^e<+D%T&G+ZF7ej)4rXH z8)pzC^sbeaW`q>|9DL3@f1&1}OO%2>mgi@g6&l5TyepB|D$!EEAf*3Q&;mc}TGT>~ zk*|TuY&MVL=2?vO5RNh>`DK$3yWC178}@mrm097G>+c$cSX3W-Zrv9bCs0Ql+L?v& zwzzG;cYm5y*0J9$=DtVMn7&o6l9%)5XE`SgXNj&pB?aL0^=+-zQhrw{7lw`MZ~h8z z{FSB&%)Ni95e7;v0))yoxoVGO?B?!>|HV08aX&`~S1W=7P)@y~^UTptmiNm`8=kjdWjo1MO>So_&`Xs$=FxgAjmW8n>(&|=y!qOxd4_}# z3C54kWNemf053aFQ)uUGlS4OF=IE?C^LGj(#h}q-RkD6Q5GQSaHDIEFvMln=(Y;eE z_Ag{oGj`R{Urxw)jw-yX^+{Mv#W{`CYrH>HF1E0#oas+{{R?Td2%Dbb-ZY)Im)WUn z)7N4`4d@;oB|g26Sq(V40m>u_@#a9I3f7vN=Pd$g9sjFRV51}dwdG(`aYyFWE_CX& z+9bRjv3Y~=r&2g;L2EQ~OuN!|h;~H8@Pd9=8GiW9wekiEDU%2i3YswVoX||TP;N5H zrkT0>ChF359k4S)rw1KSq}gU`qdx1#t@`{VC(y*hI(*1k#Wc}PXcG^OHwxuOp|Ct4lTNbF=>Rxa>@2=SvEmPj3-pZ$N#DD%^u#l=( zG1r8nG+aD_OhCx)YwPUL%}TJelR`enAU}Q9VEuCHZiM_dr3V!WyC@{P=lC-2JSTo6 zkX&gxkan6tZ~g5pDfiEy4W-k1AWTG`?W=mdOf(+1 zpp|YjR6MTX<-wAugK4H|=M^SAQ=~wC6m6ct z`|o4(q9Nndp<$+deKWM*5BC)#3UAI_L{Ye3&SZ(~M>|jv%WTe!lgwPOKUm3PZQ_;k zLPoABKpcK#XqDg4{8;BWw`+AiP zQ@NRG68Jr;19d9AJ68*O5*a@b4!^q-oaOXcHDKBLLG;tToc&Scze`dcQUGYckC)&d ziiGkFh+NLJd|#dJy_)KZ2J_>jCScjKO;Wi<@4{5VHEw{=MM)Ftvo&TZKKStDG<`{E zOf;!|VLYb`0abr}(j@>?w9( z*qZ&<#v`M>&xpnD;?}LKaviENJEO-~(bn)B7MEu4i&JuL1C2R}p&ydGW>wQ$)8oEg z4>6O)=z5(A%{CTawyUNC&jbs;=`*yc`PWR7nct8qaHjukGEEarS1$)>1~(khV45pQ zY4?x{letnE#3eqGxUY=V;WxUO&9KN}di=7L%hIA|)87zJ0VS0;Fb1=sc6pi~i-be& zj!jjFuS~`TU7^R3QhPn{2Ss0tKxSM_%XNZmi0lt3a9%%O?lpHWD5L$Hd$Y7^E+vt5 z{GQ-f;G9y^=KH;x8na-X7gPPNfENx{%oBONT5N2e3lS11NM5jGcH&`IOk4T$6 zEi0ccMqOtDxO??<_{?rpkQDVbRpPQJ31pru$_;b_Q>}V23T7RX2&uZ<7p)Zpb{ZpD zEVnnT{POEZa5`#C7O&bIC1?xzKZzoScVDX8jNqhV`+A8lbfxy^(8AA5WnqgPE84ZO zEU`Z{qlHtNR|?Kwzme1(W5$9t0CQp3lq`mZbjwkF3KxX-53Kb$ow@(|C*3+%yC}S0 zyhKm73Yu(Xp-I06Y8}h%0vSYv=^0wZ5ZIlhwNi&pZJnFeZuHjx4w>ElxPuHM5(}W7 zms^GuHQfyYb1HrsdKP!XVjRrWp_eBOGp>=+`pQR!qZIOoPtgrx&+n}muQ?odgyN&; z(?5z#zf#elZ&j(hFHZ)x1giu{W&5vuh%b~Jfq$}?6}U*Ym}EL0uc`Vf8nq8>uCZgQ zEGyCLJmZ3(S19Gzj+c9xUok`H*m3Q2w=apFP1-X|T}%`G0}XY5(*tMlOHz@y0`dLt zCA)wCh)~vl1YV?XzfJAEv_P8@#oBUH=XqDDNxWb9EtNl?7TzP~w4{Lr&N`&`aBA!G zg?A1aabJbbTHK8T_hp#Ou5R|MU^>=S0y_gwSj{Uz4e0OiZN!n`YRY?0!qxp@jj#AQ z(*9h4?C;N%CVnvSGXVp|@I1?eJ(Udiu20SV&Le6}n1)7BwA4N4x2hK;*6$8v&+Fn2 zW-a2l3@p)|B1dUG=tIT*CUP%@BW{@7H_=5=(H9Y#zI}E=`eh|1fPag8aHPd!j1kQ$ zQTB&jkvdTOeXa6j7{-suM$thzb(b@%FBPDfNzMYeKy?=YG~RQAeK)64h;#o3vb6;_ z3ZQTwW49CTT%TE?=tDT!Q;7WLf&bfWEw0OA4cRM#?kCS<9eXNU9w#0qG0$!B=ghq|nKU+O|olb5|=={!wOA8t7 zCxjoKCo~DX!p`I`aMAW7lx|?U;xN-Nq6iv(4xdqTETjM_5Yog~T+SSs2Tv3Hy+`gP zGlESLwPcP@Aj7TdtwjI%>t9SClZ6%F<^`WLgj3i#6py&$yfwBj*_@KTk}2VfWag<|5H&$g7u5jmFo<#d0<6z{Ei*R9TOy5}6k3Spd00cn5c-2KOvw&~- z`ZHL@Ms$eHo&%xj9$=`q{0c;{cZQICXgQNGUmQugRpd>PTMsHS_az zr0?aqnnVC)7W4j&j?l+96x5h&r|I+{Yk@Ukq9pXjV}Gvvb-XVbXt9c+nw?=FixFAU z#Peh3E8RKC-cx@2jHZ!kMugI>14Lx+%@wYQm_Ww`y**%)m58(X8Dh9vxvS56c+qq% z{34Jg`wQt94%ze;)Sa!HVs93a%z1Q2#Ub~LT+Rl_iTA6vH$Z2nOBwwKqgDXSW$0gp z%*_khNxfve%rxxF77P(w_YplI!<*R-S9R&8JJ!;a98Z%auzoPJZ$QXC`S;UH#D7vvnVZxV?G&qT1EMJ@vmoF6(kp(Lnt5|Izm5;aKn6`*?|v z@Sv#3lt`wOIYSSXDMQN4BV-B*DYHybGL%GSAu`WH86s2W%p_wnPnqZ6dOvi|+57X^ zyYoBe`}t?@>)O|~FVFimtk+uiy4QW%RMNeO#*q1!9Kl|>d)Kh2N42)17#9v>vm&9Q zB9yzclrP~Q%O;7E)stdeZbC(^gYhLGWf2Oa2Zb?uHdei;RNE+2>6{$B^Wf{CvuHm8 zF}wNj7asmNoNc@9xUkg{|Kr1HS+n&|EB#a>W?C+M<{kCyJHw@h z;Ec%2*VC0_++RYAJD==TF@q{?(JG%IuH`l>mi`e8_f+KbcHDb7dnd^DAG!n2*S9g` zAELxdW42qPR}@_ghO`aGJ9T6g{hMQs-Hip&T0_tlB)enWd!>f-$l@hNdmZ{}lF2mF z_jils7N3%zfed!PH{neNccmbqDH5E52 zxP)DA%Vf}{b!Ek`b@y#B-7u-pfT|xi_#XG6iT7Nm5AzLA{50emSTx!i=FYy?NF)2k zVP(cJP>zqAqu9YeZyk4a8C5bUL^Y%ub)!&ZNY(FJ!B)J$<79@2UZfSC(3*WS8D5$O zifC7Mv`D`kgu?#mRz-imP8STW&50ax%$ub%c?qz0Jg|3qV)vbNo<~PsbErza;%2Yq zwss~ecruhHqwi3McG+8Q|C>n(|F|pPC{d*oy&)2tQj8cGKRW_auM2~lr`SYp`|@^p z<7ZqOXRa$st$%p*;Ro^)x9M-1Q97~0`s^dPcd72qK86=L2gy5hOy4D2rqM|Hx{TED zq-+&!YZK|jU3FIpyhu(MoF!7pMZ^DtQtX9y@Qc{Vji#|8lYI5FSD{jMd8>ILI?USyG0XWZPn(@JPM6}w^q6%OI^^Xad>NcDH* z-$T4Ipci{P-YYKN9ZHV&Rs>SJgsa4K>40GJevgC~9hb`2O*w+1F{T`${tJUkvl^?6 z#3yn>2cAn6yqsor9xUL_UwwYL=p~JIzbaE=wP|&n47@c@)7yGNO3fuTp1jR8^~!>!=^5PYdG)3!sn^Y~It6su>2(53doee6R^>V+UhYg9XF+N^n>==KL^qJ{vSg zLP0A;%qQyb7B#F3b(ZkGfB)G0H$M}^AaxO4v`cbdL9udSqkpGwapv1a#ciJR-#}{W zN6|v#g^M!nX-+-e@18#G?cf0j?lNc0xsvkg>*n~z9Ex(dt@WV)WLNn@K<#C zyeQj_jV`W)_vj(~4!Z6|m(z(mOo^ZXPi6?35(Z(2l6ywq6hUDN2wp6FJV2^uS#0L| zvINKkwCui8UL`6}*!i>UOog_+Z4RP(!0d{pWm4yH2O4tB@j&W;YTIDa8vy76?Yc?p_Z_UEUM=Tj0+SQ7q-b%Zyefdm}Vs#nvDsB$U}joT=V zw;(j`=?H3w7eLS?<@I@kryxJ+w+8xCsrZ|4Q7o20T&xwGfW$G5XDHi78zW3XrP2@s zP&?adI#_e_-ETNZZKjx@^~z;#{h@iy&v29GJ8qJUgAfu{J%_7$ha}18m8`IDGHv?F zz_ajrv%a`H`MJ(U?obw$6r)D%^2=iSU(1nO{Bc8C#a>E7$Q$-#J^l_Eq>Zj0aRmYl zcpuGuZ21+eJ`qj8fT)-6JLj#X&aVO!bWV%U*Ot;<%snj#zK-N=Z*5R}_k%x8*f`jW zHXP*ME*SQ^0CGTgV!A;2&h|_<90BF5^Xf;`zg5b!Jg$fFVW43{Gpbz>0heTlov5)4vx^c>1 zSPvAm^k+&o?q=7XP41eJL>Ei?(;9AXuD!MBFG2Flw=mTG;7glndU1#F_UiD#H|rjR z6xTqA=E7{=ZYZ?;O(^_;f+_pGFqY5^Va(mD5A1$PUW)l%%b|MEYbgnpb0Uv3`B`u$njoU$kS3u0V$0T7sxn5~E zyq^0_c-1~MLGTSEn@a5yvcUFu(g2BPP zn050vF)P9deR8zGc3zKq2G1h4O#hpwlZ*p^I&&Bn2L^#;B|*EWSH+4?<(Hg>R3okj z(0m{3!3=CML5iMgdDYE60Fxk$Jf;Ekdl|{%UPdB|K3OhUR4TT!X}vKE3ITb83Lgbz zCBog-VKUw*4=g3@<(_qoXx&Z${hg%}|5J(X`ziS=#p4BB%;p5V5`_Km0|&rKymL)X zKEN_XRog~6Vs^C}?048D*q=QBX*kP=hOFr>osZcSLJpe0+Im{5O~m^3#^#&cx2so% zRkWV&ro2@7&%#Ofhxgxax_J_*Ay7ja#R zwjMY066lZKsjIVTSP1?P)5ZoR#p=r_%-E4HWTr;~(M2bG&6fbhoiCpkL>A6ISGB-W zD?U%aO=wc0D;Na3d+f(xeHUJlm6*0C1sxZCa?QG1w49dOA6Qx&&Jj(h(H~vLHA7ULLlId{!%sLr5+BYDl(D}u357a~ zh2uX=F+%gA{hrV=TWop}GAs|$;Xc@uI1>bB9wA_5ULm~pJ&@A!)ni)~HzDB$QRH3l z?Mqfo&(bEm)@M6ROl@rgx&^P=LG?{7WNMwI2889#s{7+Ey;n1CHtbHHphbV$_F1u> z_v!G1%tzCb)#^Tytzml4-`#Hf5bZZl+t+`rmCj*`d(wnfQc(;v3F|2ogmsHQlU|%U z#}@vnbxUVwJvf&th9lm`mOTpzjw3{x6{*w-W*nZP%3>>BY+$3n07w zHN!fC|NVpMJ9P$YI*|WFX8(sNzDbLl5VIeFwm6U#FI69+*cRH~%UK}cl;+%e5I}5sl_3BWdLSAOLcP-x?*ALq3fn~ zXJf#fv|bcuWvcRlU=Ac!20r)uL)JjT30X-Me7z%@!g@vf!cN5_OycLksiB*)I+ z2$pvaUM`0aZQd!wDtbYH;$zzGyG7#V`lnZ`qp<*#`>wHmweLt#Ke&UUaso~4Jg=(` z`$^Xa%Sfs{hlpvX!8O{o8xpz(n@(-jO`8dq(UJE*p44LKpBO`!TXtK$aw^~&>#{`W zGR`bZ{7m=faFcHu*t)!PBn6A2)7#+C)MNo{BEAcow48CX&!iI4i#bJ^pmUkD{7DQ3 zvA5ZH&s|#(PA};j<2QU&ijsZNv+{C!!O&A7%tUK~0zarB7A>_&Vh&9Vcy^{|o?$C$ zo-ed@Cm`kSn;on|zFkjaWCvRc*6SI{q-Bs~MwRz^hhJn~*S zVaG+fJ|k3PX2!A0%b+XS2mljX2ir+T5zI;zvP?EUr zUNmlhTg&7w>!AB#Ixl%>efJx>BusU_~5?T;mYP+!vF1>yBFp0yemn`wjD}kC{KHxMo zvzA^WHDkMaP%TnE2mNcJ7zq?B(iV;JQ4{ZFaT;{QWnKS4UE%VNWiyX_9}_=l2V;hY zkFEz^E0YcoF}b+}6r;kG--Q<`)M3tCNmv)1Pi{OTMPhaT573EJ`VCqn`;rRStNQJC ztNxr9(IggU*bCO6BGmL>%)pF|sPUTjv^^c48{q8Ix)$K$&(PR!OTZ1H^zL1wrm4BF zoZ6{LQAy^veDV-a$q1T!)6T~GQ(a%|3zgZzs%P!CKJeZC{22MVzX1U`r#!)pD`aD~ zwEwcxKDwKVkkNc=fJa?pcE2O{#V!&=^Ve>`6~7*;lPOJi0yMwv+^sP9WS=L>JDSVl zHdlkJ=8&wDJpO0@6CFD~-4X^#Y%ut>xt|tAN04ryD=xt zX4-*m28EyvyLb;67-xgZe~%(_WzhQ&dMY~iZ9j!aou#G{uZ)lv;qNdrs%3F4^hxa6=YSQA$ZpZAOndWZ#Oq;lgg>3@&Ht_D=28KraEnG>FzxMrHj88WD zTJ>d8qWfjC#ce1Gyc{q&&&*QQWax54F`hb<*T zy!$L1!#dh$CEnL|_k-oq@Are;O-MZJS{PpgC-YOh>M&kYl}qQrGus-Zcf;akgL|Bg zv}Nvd>@RVeb{j#|inp}REDqy)&!L7h3j{0d&B)taJSZ5S7o%`go}^%}7Pu;L&J*op zy+rw^t92>i1{N=pda<`N4!b+U;~nj1eRYiLJ4fz4LtbW}X8QQwEt;=-@FhiNP5W8^ z+e!ZtpR2o{B&qQ4eA0S~dUky@+Wm0&(E-a6$Qh_pSPyXB52v7Hi~>}AZlNjan$%jVBz@yHnrNv?7$#=!G#dnv(dLj!et&>@RVU1Md+kc}yL*-pu(xMt6NigpA3LLMgFy_s`wCw&NAoxwI#csd zF9eGR?3d_wNJ0kH3sjrsHsB>ig76b3Jz7j#-}(xI(C(Dmo2`u%e$WKw@Ni_v1_+>5 z>)|(>(-G7t&69eLP#}hA(@x-#mxbI;667E7^HNPn*zco+e6|M3AGgxZtybW#L-&K}1?vc#0*eWuNFsh@Ev3K4U%$5mL zo??t+OU+RVVC3^>u5rCSB6JG-lNi24x8wT%O$%#T{2wi>{}f8DBb-1rF~mWX0UhYJ zc^S!Ti=aFo3jU$5!JaYrau&cV8r+CGNj)Y zCNK)(JaS$xKw_}g+z|WoaNhji|GafPn<|-V(Fz%W5N3%Ih@R8KsWUERFlwTivGfp5`tAU#rH?iR*}Bgmo&uqHwdx=!+(t+qngHp z{H{n229a<_P#3o_ym&a)Tt@$rDBqy&)oXZRTA+>b>9LC|zdiV`$brwtHNeh7IBq5p zTAk4RbSXvuPCx0OSH%HGr1O`)kGyBb z>n36%<1lcj#h^-!abK`jI{yPA!am&Rx0G@bCflW3d*#Tjy>f(9boC>B5s`j;@K2dp zz>rNKVUYH>{_b^V2g<0|AQ0G7FfJ8)Hiw zCrZ?!Pk3P>IsMg=j8{NO&z)p-3v1h?;IF84ymH~8q)$G@N_|!~_ez;-n7(A(*u5ih z|L2D=;QX@SP!yL&PNH+lV!Kc%uTZ)!75$Zz#XHpErS#Vh|73h@A1+7nU10I&jaS&? zcKQGOxJk$l1|c`JdyZXAY7uZg&Tx{VTCxs<)ZC6N%Vsh*5x)exn4thV9yAFddqOuh zIey~xe;tAO+g@VE0hvl$lG%Zd-k}7XCM%BzXzyT34vm-3f-p7VKk?-PZX681zO)PZ zYyak;hs7&`-O%H1`ZR@Aa}jTgP#YFrS7 zT_S00N9}LwhyVJK;*UFm8K09%P+f0{mnHJ*hXxvTA0E+@JV5&1soD=l7m#tCi9={ zis1h=Af?{I-69?=LSPp64VQT0CAkQp_9A#g4(cE1!)O>q*wjNID+h)s6GtHh+c8Dj z8pxZ58_O{R7VUbq%WXEt zQ*A1_PUH_M{i$xUByH`-BJ}6?5c&uNp^sY02~`!seF6SLO<)~RMPEVW@m|=R%XX+) ze3s!+-eZO6T$b-ko9 %Rw0OyuX7!Ke}JFv2g5`b^G7yOW%EKS+GOj;qwM_&kc& zi@$y6qN5y4ffWL6yFC1~ay;!jl&E$h~WMVCakp0K-;GJNMFu5t5Q!LrzB z60UDx*i%Bwp%obA`18qYFa)<7j}M+&ftDn%B(2@;h#~}XV+p_KcC?k@)(mvGI_g&8 zz$O$zNUJU5OVLac+b#Mhx{)l7_i8STdfp-9uZ*RuL}5#B!kB>=6D)W*#-a5x`n zL+lt^kyf7<377TUmHu!MhrAI_=j=*K%_Bb%zX+8xE`y#_hiN;)9e;lANT$;*qt8(Y z<@s!dmkpef{(=0b{6*5PR_^$uc|-Q|E7HL2N@J-3u<6bh+a<^!s<}Y67uZLVl*>3S z$&@$pj7}_zbNoCj$}-Q!Og?S#r}9!m>?SL=Z}y9hz=QP7M!Dlv*GqS{EEb^A{d!1k zd``xz2H#)a_T)N0;e_*vb*YV~d zo}syokB=LMhWa6dP835@*%yX4Dy zj0~1WcY7}Zi$)SKF$Gp(-cevP4+N21ubmBQR2xUNz{N9Xd-3vV6dPJZfg?#@+Gihe z4-bp2F0M`ZD13lfaT*!(V+Mr*ZQr@$-m^}19QiMw*@;;HxOU-o4QeviuRn$95=&*^ z$^y@-Cs~`N=$zh8ozAwN=K|u==|wshdezK^V3J@SCx(W$GYHqF4KL>S$`_tbF9#mB z3{gQX%6-f6QUO&s*E(^vOcAkbJ}u$`k_|p^r%a2d!Tj``)B0lO7Fmo{D8B`Ui}Zmb z_@?qHyvuxh+cn@KP!$l)NzG$f)(^N#FFzV;nIk=y2fsUL;$JyWL`eU+PD1DLxI8bDOK=CRY-2$zdGr>5Qp^4{pE0Xf5O8*D?* zQQs|yIhr&W<9ALF3v|zb3lAS_**@(2a}$d%=R+Q6rkjxo+=6O1DAev5Z7UHuP>OL56t(Stzf@t950;D|aHqhmo%fG{I;P6(L ztBo;z$eO%VkMYYk^BV|as)Z7*<+_%La215lmAGnyin!D7*~`~#@LvKX{v#k z>cWwi(em~(rmy$T3!aVb+t_+|8SjcDmC;#dkQtp^Q6Q)q;q&l8Cz5!_oQYbl+ORvZ zSq^Pf4+`<0O)lHk*3bocB@ATU)0tiT3?HfIN4G57WJkA6W})jl>X>FyBoL zrQ3Lmq6p&Xa?JRh zAt$KcU>zFBVQ`F_?BhSNQTS9?e!{Wx4(fw93vantAo|Jt)56`D!|T^PU`5pOb^P1`LWI$vt74n@~LO9C@%5awR+} zty`#SYVrNT?Xu>et8k;NJk9o&yYu6G`nhGL0Aiv*z5Sig`1*u&+w%GkvIbwJ-CIzT zoV%++mQj~oIy&YEb7tB;FJhnn=j_v-^el%p2jJ1qnoM9X-Tb-q^p-FU@12)rH2XEd zX{P~_rfDM&4~b(r7a9dlSuGFt{?Fb!#xbkIDM;)uL_lHI)cxl(u+Q>la~ zA648gaN1aYY?<`sviFYmouT?4AE0GZ4|t)`r*JLjwUOQkjauySB$y!Jca!F%l|q`| z9o8vr(z1%_vUo$aNbeJ&icy0~@9MnGh1KnKFjO=<^9D(Pq<>y$64sylNa=R1W4LfO zV@h&wi7PpVe^}TjewZP>+lVO*rF3PV#!IoT2(Z{*C<~>s)r5^TIp|}f|LFt+qyuqc zVNR=l4i~5JgjYQZ3W3k0W(fk)kr_VH`rC?ko-YU;J-v!Q+iKE9Bz&z$({e3S)I|uO zw~aYB>r@S@w&C-a1qA9ZX1IRfLDOd;`lvyHl*$?}HrI^1Yir zhb7H+unNo1)iUVEQ$afOgQKVg;a^a+Axli&w#Bnps6yBr-Js{@#eW4{x_9i)aHFeC zGZE|t;n;1_QUvTjaImY1U>G;url`mF3L~5OTzv(TJ;IFV2an$PlqZhW$O3P=twUp4 zYt|N0;5(k4VCMF%Mc^%8n+uM>hc2 zcNaSA8eCiti@-Wc<=Q~Ml*`J8{^RVJ&>*ANJNa8t-K5xrZ$q>4n-Sr}OjqYmE+Z4J zn-2R(8wp0-&RL*1$R5YZ26Ix1{lHhFI#BLVh*tnio1@khWRhxR6qEip?H4%S@r^hM z5j(h6OQA(b#5zV4xU0+8d2Oj}!4@>PkvNw#v0B%gx5(bXC*o7gBqZ@4=;9Ze{=^l& zINKt$4+3h{)m`$>+FPU*lL9s21r;IjqI(Ol&MHE}ZJ%u8vuWf0u*Os#gFr*PrE_Z8 zCmPkwkoKwdFzrN89(Wr!%SV%SQuNK3SQJGilz~u(>P3Hqeh@Nm>skDw;A`h~`T_%AYE8lC5$%{iX$ZKX7D`rklbW-r-J|t`L=I zSB$#!YFQk`O(99}d`Q_*OYRb!J1z_rA=hV%o479@Zo0w+jDWiyGqrLw1FL5B3*+sFFs6=9q#~F@W3oB6Q!`u^AYzJ@Mq|!03wc;`Ha_OJI!ib% zoL!lvGP0&AV=#3@Dc^0m88_}_)1Ycax_Y4Nd@ z_|;6#x1_>zj|9Prz5(HOWvCF9*A8ldN`}88>tljb{9wVI4a?n=f2QU% z+qwmRHea_v)K(bx;lw{&EMC$lw! zG`;BFK=$dRg3A$vzkw~t;DD>AC+qu*mWn-FHaI*zZweBh(2 zA@CS%gqd15iDu7=li1ui&X!*S}@6R+(v8b83epx zCdFFgA*AUo{u7(zr2-y8=Q!u;`!)YonVUU;Bj-I-g~VwqmSk3<+;PEa#wkr1hsRj< zTk;E`(F}?rrP7O#^j?7E*S*sZR>}eM*#Fei_C|!yx{*B6Kl>h0{xTNR1X4OW?g96}u=>eurPLd;Bh(hw7D^*poJ9M4dXZ5?J7TNUybI-~Af+oAZ^ z)k4lz>#XxS9=7~a*o|EN0!aDA64AnTvX2aDA+!1gkqlE+oh)wA4QDC@0izdhCu-UD zpUAO$JrFcp@mV_l3P1OnJIlp9c+x89QzUz+e=W$;(4mQEUlHi(0=#@rBX6Uo=I!%c z7`}~3(xx(P#9*m4#?!&~Sz#)JLfMHvfW=`DAune(S>|GB0!B!%?CF#wxM!5>yp| zwTi$K5x9E?<@Icn*MSJYUu?gmNAWf?>rbNt_QCC>SSVY2H@RD##GahbN6NdP2vnPs zsb;*_OMmYp+exCbvPUMy&hixGW5DZssYF2^TiGM0rV&O3$FZSPK-1l&tOmPMtt^lj50FHVOk!f1tTh^m#2J=Ie%O#r-kTsi3!qJ%n zn&T;`Aw0~dIm)W9sR*CEtccOacV(Ps3Xc{vDhoE}9^u4TpIbg!8sDylyCiUU{?&J| zWxO4R$Do5+uPiNldnIzbH6aX)AwTxyW>{o*{b`V(=}qSZKT=J7l-xZ!E<)I&^SqpG z3xI-r?v1ga#?E1sn1Ffu{T)Fmkkx4()i{00ozI65(|o0>HnRkZ^@cd;vtfN*|?;47$MiI=Fu*y6sd%u zO8#LjY+Ep4!GG?7PjL$jIoM1FA$^GWA3-x>`2WN=47tk?#S7Q3CJ7OS!x4IJ`GC@N zIpAO-t{9*3aB>S?2l`Zqx;0WAh{4ct7&jsMWT|tX-RNsbv8rhahN8ti(ESoF5~Ul! z3YQffc;fv1iwvB5BpvYYaFz~!5go~My&WGpNd&&!#?8_81M`n4EZ*tKyYtjkHPfiw zq>vYydrvFsX!~Z86Hkz&F9Fbq)r})t8dnZc0K%4!oso`}Owre29q-)G-A+dLefRyqKNi0R|-pWVZVR7v8>b#8@f?Id#dc78P zTEC@g_>Wk0JdnIC)A%c}aE!pY%m{V%V<|)Lv7*)H%5=Hqcak!NC?gQR?ZynS~hLVQz3b(k79?wofUkzjmQ@{6Q$RZNhxjSLsN=IqS{ zt;Hc};~CbXmiZpxq874=_?N0e7(q~I(r3nuZ(9TBHzB4ecem^*%IO+pph;`>UdGsdRok!dj^T_K~sm1U&(a6P&&NHAqnk0?z<4Fs_8p3P(_I( z9<5qtQ%8s~x-MZz?fhdb)<2K5e=r2m2FUKvFkQ>Oz5W`4#a5zORA7udiB=%IiM8+9>-+(uL&A z)3axgVZJYWiQsi?!h86r4TPA~tWS3H&YaF) zG}JfWSh8_m*##keG^-l=$=kl&OJCAyD91?2)O%!;RE>biuSO>M;0HjJW^$g`M#;Bg zyz59!ztjW2j**4EL&Rm<(H#|j5-hvypo+I}cQLuEKKE(9+ModDg*Wn;Xe$t?$1q4) zh1Ca7Pw4WJ?x)AEPx%SFF)LvzrT-(m5;zr+B#UL;p>|n!5;6$u4t>Km{!|VTd|_;Q z7rd@xi!u*QHf0n^d*0M`20+0jXYJM*uy*s}gTJPr5>AZYGoQ*e)a9x;dZGI&>WmK5 zsP4I38H#p+%yO$i*)Oqs6}PJF78k%3G`2Z=(SzO*n%HLEF;=7}&NcPA>})lFGgSlN z6&9U%Nf_%*nm=95>e;{~tk8xy+3wGG$C3uyTVycfXTAd6=9AjEZKyxqLG8PsVvta} z9J-cu=5e~L==$^+u7(5A)N*gU#o{4yc4>D$*eExzun$SLVpOHZa;KcX6dVm^l*HmY z*jkxz;%-6bdOCa$69a+R`dBRAU7?P=0&t}@_f8YeA%(o>#mogH(^~ z8_+(!;$PfWLXtUj(K^(dt@QPcHP26N=_IVb-fI5CAlA(f$!9^70Oh*AR{#@W2eIM- z>N-b17kVob?iaO(>iR@v^H>X#KVyX%mg!8}T9*LH+zom3>%Y_KoVFO&DbU zxVXTecT>)x%*N%O_40aq4#92gcbqD!6ItoiA9l!x!s0B+v;G74Bq^a>cMn47%%OTQ zpgJvlI#Ko1V(ILljWhC^i19(qoDChNuXy#4XKF71)DUV>y4A5b(O&oCv(-$U^O(Kk z?pOx4=bkZ%F2v%P+y`+xZkn_Jd zS1%FHl@Hty7hteG#*D-0lN~Ea4#Egrn%&LR<)7}uGHaT=5CBLR z+ul(C;{W?!#D8Rv9!vax6+QFVA$2Ed%F~vkej{hf)0l?(Y$M|7S0V%$?Kf}*y!)Q-qfN3j;SWR9Joq|+|A7vsuZMvNJsOv(0B zcbLwdt{s&N;~P;elmR7*B*8mBZedrP5X*=Cn~pCuP#E9YZ{ptOS~c<5!T!{_DMj&L z!R=oslc`qDZE+yclXZ{Q+bWF&7%AX{Fr3g6(nA;*UI8TOi_i{V_4zTAjl0y#D=kNZ z!mgQa%Wv37$Uwcaf7GKXJTc$7C}cng6?swyMwYRIC4abHL$q&85fqQ!U~hGVV09Ad zFyj!HJdd_O@@rnQ`BV0Tf2myhVs3{<9W*1ERWn=co>fBLtjD;1kq~FMLr?Xa4*d;9 zg(T7TVZTe0?W+7D5635G_BocaJCxy{Dwjm~eU~4N9wjR^?tS{_4XhNo=7Qb8+Pw&H zA0oh5kYCq5F!W9y4eE=L@Dfj1EOLksJPOJK&{_&VvPUT;{SBoQnMyuSw^UH_YbqI8 ztA7lH{sA}spCiDhMFKvuIhDEL;iccN0PfM++^RNKECrv!H8>**QOo(=MiA0RQC*C5F7GP!e*u3ds(T;9hojGW~?3536<*fG6xU)2=H3L;|n z=iuJ{ynJy?%FQ+&p(xtNBOw%BA;S_ibkohAn`|orcW-Zq{*Ar){lvI9K5IK@Maf?O zB8ses)bqXdkm+@0@>X5%F-5}5C`5anJubnx_H~S-Q__9Y3xVcblCzju1t>lf3X5jlH6~`g5=T(1}9Lpp~oTI zizuJ`&m+gPFK~1DX$}?|2uiId|zJzvFHFTe1sp z#^}-BFdHB)ZHIl`(0f5;O!pwUeZ0$D89p+rZwPnh;6?#j*h^Jtn?c1s)i!=%Ge8Dg zUd9pZ@>m^z$Bg>7M!AojWIAkuEjiM$v#ghP>s(U7u}e7Vi%7YizNjF&7I)q5KZ_l| zJ)eY0?jG@=9K+5%f=yyV_KwHBWMU>50?&>db536b%0y(Pxm2*miVRnH$_g5Lqgbx& z60b)ke#7hew>Bneoc{W_jH17c{0Meq825TM^)d$ti&_SKx8Whxm|YZtqiSr}liJhx z_@TOW1FtEkiqP7T1mEk5rGt;DrzC>(C`Y#Y92XkoX8VfkoTeScdhRtxqw|{NBZa%y z<_GrezLf62p^yDr(dA|fdME%`AYPHR*nS=E-dXaFN6wOjRJ9mTj+SL~VB+IYW(yGL zt0wM$ z7@M5-{!Z+I3=3PFueu_3u+x4rb+r2WPau~^Amk$%XDY){qbMiC2j6T|Tfx41q3wo( zC#wcGI2Js$oeD~a`Cv)JP|VQp7z&cS(zj+4$J9- zbj%f{B`lD3X>ZllXrwkV7-w8v@<%nx}!`cZe*S@amU=-xoyG2f6|*9j*G%kQfx0|$WdugRc^ z?o1N%383qQNK#`@FU8)EQaXnGC|D$3A1baNohkxekNi9($$rE`cIY{KVir{h6jj!pp6ugru`ulW_c1YTa^qtbNfst z+TQ($55{B}%+DDEzy7>B#s~Z59x}fPAq~O5eeHSi1EaU~cb>Zj)O*Gu8aKVWwdGG} z#Pn2NWzb+$vCmYs|C(n`gk=U=;WHbRLf|Wk-tGnMWoSEpV%I8Y&SNgeI|*F#OyOe*iQUpkw<_Y;{C9YcJVzh1efOA|AN=m}l*r!}9O8!IQ`%IS|ejJFh|qVYMUbbEcd>)x}u zP#8~dYN&WZ(EvI`x6^$3rtecOj6f?{jm0;*T6nMFbFcx! z-47$R=JfVa_gkcoE_klJSjI6M-ixz+pm8IN{tHsNxrhg7=Iux^5*=`D4#Ech6Yyrn z3qJbDcFf0DTq;V;r9?gJw)Ei5fzNE|i#UE+bSJW-Np#^cZ#3wh^8p0B&DkTsH-OFd zzz~R?Tp(yyDw2y5{dS2!2gY+6fYxOa3X&W(O6J#FKqsYVs&F(C9M&qSbj$$c4;EK< zJ_gK*LSXP9sCGTH+{_pb>e@%#`3vY zB>(=VjkEu#JJRNVaYg_%W+#*i-EFh#rl8 z(!%5&Hz8aLf_^O9sZzZ$YG^^-8XxPwIuXRbmOi)UIBGFv~hdWDd(lv zm%Al)A;*>$iKpFCiaXDx;FfVa?iS9{RjsMxX6m#Oc>rRcG$)v;B!wtn1jgw^t+K`JehRZAx3T2B?ISBeZtk!EnxL!>XMn zD{6P=B>*dPuIx{$L}cB*XsG@w`w;~!ET*YmA8Z1OBns)ii001WztG%?TMDpxiCh3? zq60Bnr;JOnc`F+7))xo7#Z-hS>9&vo7q6$QUr>Z(KXKma#Zxuk+zg;XpG@aJ^|F>F z%2CgsAE2TM+~Y)c#n7_lGm{D9Gq1I}()-h8XQLnN^u+V;B{L@uhi8VI8A@ImZ}qB7 zdV8cTx%dQ0lZ|i=@E`H(VYUL3+b3{*w{|4~BKMT)W~#FOf-5EpTRBu?F(e=C;axjuF%&a}6mxk!ch~~srZJ$4RXppN}I?Sw+24>;9K3h5%{hxD1!Nf}xmtw&Brj&h1C} z{d0N8^AVJ;vXSAL{z9jJ6T(Dt()3%~TE^)spyo3RaSHI~7oKOP=I@#j;48ulUVa5D zH7dRT+VIK$ff!=ZAaZ*t6eCQO#=Z1(Ma7|CN+QtS8@OVW5BHTZ?H6YfFM7Ek;}zof zgpe5*0HW)H-;2XRLB`_PM|~g6IK+A<-T71=;`v01?+Cnzka)35aa-$4MBj4Sac!->G>BI8-r^%C9 zX+GbSr;4M~-lq{09U?gEb0mnlR%W(6f3a^v%&70;%ITGFVPOvfdwcum?5;7tmS*n% zENAE}D0snj`)6=zqI9`>!4KyH)Xw*GZUJ{}u|<~aU3!)6-B+#h7v%<5*G8k-X;`Ov z=1MkZZct8awugGa;do_bZLMtez2^BM$iAKV62>Ai`!zp(;(&EG=r1J-D%+l4a@icF zyF*R&VjIV$y*tlbQ<*>QlxD+1rwwq;36ZNhLDURDElQy;at8IPKQ}yq;FfJ4A(=yF z4$EryFmqveQzr+)JmUSU*0yF^NhR4|FsL?UI1ig*;)nV2$v61Q#P%)>bJ`*1K6 z0r;<5@x9gJEqXJa^WE)HD?cm3!Xl|2zc88-bXNJPvY0mMi?02Qd8jsPKbFqhruQSY z0uPj;bd$}*B3NDq+*-re2;Oz@t9re zB6&)=+BK#01JsZhXtnD0sf!N!HF22`OrY9EG%~2`wrCFA{_y?Kf_v13yYulL`ppM! z6HLr=P+aS2+QuQO6Yh3Ox(8WrGCTY2h06oIRJK4F@z&0*@Pgho1<@dfMou*RIYl{V zPCKO~s^yt)$;@Z^vkZtbO$#Agk$qY<1W`V;&tX=C*G^DDuMxdYR8-GNzQ$jvV?S{~ z!LKneZh6B&@(=rdP>f-alhO~pLiOEw=q72V zEQ7WX0j|vl4%basN&0~okA!Q&g)b?o$9EPt$$^PZMzUH`I)AOSi8 z|NfMZ7M+^y4>2+Png=Y$hY4@s6Q77j~b}>PPplL|jFRPJLTIKcN;KF?@K% z4E?IUQ*cAA7`i1vOWk8%VxMr-3>_fN>aO+mZ%A29K75>BLeW1#Is6p~$3ff5Yg6o> zw`ew}6xkYyoGaOT*$Jzo@oP|R-*VnOG}xNrQcLA^F;Gdm$+`|2DKG2tL0*z#h4%&C zvVY@U-AIQS!I2x+31Mn+ljs#n+hY(YB1<_D*W<*CuMbS-i-76W5a*15M z;^3&3dnd^L_(yB9EuaONJlr6MBYE$V3tCFRCtl=>&OR|C+#=S~ox0fLDDXuhJOGX( z4*Hw4Y4l-&NBa=ZK+^LUNl|$+{e{*>^{PS&(Pvyaj9;k7%hLL_F42FXJ0(-+Of%{OzEfjKybKk1V=gd*#7glHWyQ0l@vKIQ>6j-WQ6i`d-ojGJsOBtw0d+S zy+#9BpUX8ZimyyjZGplF#y4yVIWO7cCebgZY+sl3 zaiJr=zO1~T7xI*}H*!d0!lF?4QnR26wA-Szp2S~ffZIN^(m_TORzTKIsnQ(A)bHlhZE#jwT zZHGS1?MLA97o&C_dmc}BtErz_v7XBoPrgk%$M(76^NGP@i~Ekdy$|eID7hB0LA2Qt zwjn&l7k}0FgtS8B)~jqH{s-OVX_`zxpj^WU!w{$5&V*68<>Mo=$7Nj@x`VB<4itF1 z`v??LPE}b&U3DqrEG*v~7=5Pc$U2CuR27}dQo!m$MwmSIq}`RJ7sH-3CVJAWo1_6Z zRmR!Le=^B$Rv&nDlL6w}&FY4>X=j27cASOpv1VlvXXb+OzrDA)5mqjI>DBv%I=ba2 zke;XwO{(H(x*!7|wYBWs`$F-vg^eRGK1^Fb;ojub5+|KSbacT(PndfgNMuI0s<>!5 zOIG#X16ldWx0BKV$njxN`ghmlz`?`xGdh_z4?PbZRf=?kCfiG&W1mB06pd6l@$m2# zA@fRFGm6tnI=1C`)5E)2N2;NQ-K#g@i_W>&myXn{r;YTDeC{27UrN6cc+=iH!ump8 z80Se`xA(_w%ns~F?Q`WpLFDf~;K8CigoX;4`ei{Pm6qK;#*&sB>hY7wQEB*3wvS!c zkk5}UG3!3Bk*Vj&ngwVAN`bSk%zuBDiGi8f_og@9S0>E@6voyUC?~IS?|&R5>fkA=@UPgN~X(fe9^fl^kfZ+ z%*#-*M|;uQOEoy?t%TnooZ#=hcObL$f%BR(TsMtA5!|J=#J~G$iVGqYjdt<1*sr!f z`?HG#xh}wH##euXPoTM!MTDYkv1>-nvPE#1Pqw{_y{z&9M-M_Z?J!}(Ov~xS>|;ex zLARH@lh0F4$JCaxj4tRu3QrR4nu&b0x$QnvQ~O%Vi}HXlgC5tbWn3|bUMBukLc)ic z_YSX@KzP%>Xs_JkzwLGdz4b+sE*{|H%WM2bVxynSq0Tp+?#XQ{u+-<}{_tJ)bfVbi z+$SEr3UBg~&iJAVM>A*cotjhr%`oytL}iIn#RMlD*UwW9d(?4KaMeF@?LUH_3~waz z!ZWo<(1fehGLE;P=@Sz!`ya+WMjZR?0xHEWb)D6Aj933B~>*yKJlZ@<8Lahc=IjgDRqa_vSHBYk%ySLl&^RYv_Mx*Ulya;b4Ww zxS4)PbQ+$9v>&oh#AdJ(Hp(?4`T;S9L2 zJWLYQL;#2odhMTCXIvTuM-%lgBxf1ai7QeGFmtaHG2df1d$GZ@PDFX@bPbL(VYGnS zX&Uz#u9__-45{TknO9HaWbs|2?-Gigcil(u=+Q&m?`URH)C2!AocFJlg*HM{bV^Iu z@+rsd18gOkwH$tGD2_@C2D7W{tubllnZ46`Ewx|lV2u@f#rP@u(w*pwSyqo*Vk-A3 zcz3}oR@Owz|C!iXyj+`|#=RsR!f&S^CTKafbl|4qsC5z4_$IY4S!Iru!ZSqthMzic zbK1h?hR**-+gnFf-FEB3fFLQUARq!F9ZLa0Kw<$(r_z#2m(n0zq9~$BHxdi!mPSC4 zmR^)1N{4jEnak&S_IclZ_CDVk-x=p0{DU#@{@pR-n%BJMOohYbpNNA?>-{~FZs0TI zs-;S_W*h?kLZ#~D3*pSb-wp{zdey#qnBnw|kY2FZ6`a<$mLvJqN z91tt?>5eh&T5r+t{NQ>`c%;bmxLo??*-7Kb!)No%5vrf>q`NFIkq_Dn;*L{iK6B8WZuVhm5L)H!iHp&$JaB;m!s2se|@2tnGTUOtfcbb*G!4K8_0O(p$PT# z)<4HC^M7*eET|&AvQUTW8R{FJRz^mbx0gqw=x*3#-v9BEQ8ERM#Vjlw3oNzd;K2Pz zW*IAxEa>=>38lV{{-b!ylxIH-YFs!c0VrstYJ_f>w>9p+3wIhFCPfdxz6~A#c-#k) z4760^%&4|CRzoR2W==XK`d<<+PXbx*v+@TR6JCwbCTL4LBzBSvNLxkGCev)Z3!18P z;~>Pm_FTd*LHPjqbZ{kq+VU`7SY}N2$*Qe@jW`=vlX7R#9D(cVtH2MOpcxv;ZHjQ` zM~9nO>Wphg0|Q?s%=eG+CtqI({6Dz{chu)LC(10z#MfJ?1TmgsKj`XPKEni{g5Ej+ z7(sq{`kIEx`XRjNcYr{>N8B3SGXakgnh!i+9jd5OgNyYgM77FwK&e}R{58vF=>4Y- zDfL(B@-_Y^(wc%XLSKC7j`5T}C_cn#q?Er@K>k-83Wi2QKos#y4*#R)iaI5(6c(-~ zm^fO4?^jC|=eAv!hP8%8RK2s<^RPjz_#tJHJ2#5v{CSSft=1>P#+^}*Dr|;2A_;pX z&qTaX`SM$GJ|Z9MEhdVTZZ+arA@tq(py04ZMY(DQF53q$u%^~+>%9xxHVt}(S&g(K z&<&c0Q6Q;(FISVbNS#~_xA$;Pg5kHMq~!PLL)l&~3sX7MeM2R!|1O{*AFQ|5>3q_j zNg6m4C4C3nQa)2t@&R0s++LNRYH_?y@9*VKS8LUr+$0~|7Su%4!cY-jec2<0vH{R) zV<$iC9>KqLx+Tc(F2cj0$IQW?PqG#qRcpqNLpS%8c`bN6bysMY zeVaFGsfp$h1ca85GaV2^=-@A$^-J`RyQCsm{8pYfQ=b-OLVJN9UQ%#a6F}&1i{y8R z+a)e-r%Q(rjae2{-z!Sl)m}LGDWyI#AYZ>Mt3Ia|2U_D&22~C!KK@rl_GT2J`lV#| zX|o9T&5QdKF%kJqQUA@>Vxl3;)`EdEdQIXc37bZYw?z{iTn1lqjJM`|rzug>iQ&&< zC&YDi92?p&`id-mK}3th!j+ctKb9{2&><~)S~A5I5OQ}Xx_i|Stt09;tq)QD=&~mX za`LifZJ_93v*mTe-^7F8D2Samw;w^9($DDUOqUOdHwSUkK6p{ zhC5%PjMq~K9xdkPXnuNrLFs(k|LY^hVQQMLh8%sYyxk+W1G&Xo zS(}PeP-^5O8sfjx!-{u0S$)Q9&mR^a5_bO9=SXX0x|#&Z?>xdkRRHJP+c!(s=1&4A z)Q(_oL{`vS(f4@lDnIX7J#xCm8AKWwDFuK6IU9~$GytA%kbv+RO&9b-+^Xp5={Z<# zeV0zeDf4}eF&X)fw-8${OWKss^omdq?JMEs1r)skiD34wh&f|v9@;8?xQp1rOYg`Q zz6P8L#!ewGxm;ve8v}IS93uZzSBE!LN5R3hx;JA68tg=W2-DP#$9MLV4;tcZp>7@! zUk)Jg5H?*xgkgowC6^J)fGPMViy}Zy(x<@b{|Kvx#Pi{GZ0cT5sams7M2Z&Ckk%I6 zFT96R5BP;Vixi)rc}Wu}n)(BZuh4tD9iv(ae&ik~(FS{7mr@O5{i)sY$6ER)icbrS z6?O0W|5>M6vZ$8Nvu4(&UQyw8!oMPhrAh~teM2>;DuLK6W>hWiyvE}#np-%uYYxrB z#{OxmtAUX_vVI?F<<^`(^ESMFMB;Vn$}UyoR-_9pwY2)H#^%Vg5U^v11Qy6gjJAMpM3Q_oBJ8^vpPziO;7?v2i|v;Gpb zd#sP>u%FP?#;24~C&PX6EGVN<9Y^6#A%TG(68G9fQL>>bpd+i6$@L^+L)ar z*OkZ={^VVWOO#KrvB|&i<5*}f+)BTCsOShjuZzp@mE7frzT#59(0?zv_QK%kJ7}34 zFbkyLH0?`j0rXER|Mb;(4#-#?XoZ@SM4r3JFU0&OFvEG&9_(m6Ueu@w#7C4?o^~nS zV3GQWKQCHC(P5a+eCTiP=<{l_Hb5=8cV}@b1J<$wv1LoZj z^N0_u7b7jnBG96c9V8F$Z zq9S8+-OpH~{wxou{yG?}Dx2oD{`RPQltH)B%`S(BwO8BN?jERg#^&KnS@x(@24bG; zlE|#;G9t(4!#Ghxi=+vo#gK&|xVmZu{jC;1P*Ax`@G@R_lf ze2oh^c(?npXkr3?DQa@^JeL#~m!F?swocdJRw}|Ht1MM?gb@{8lzNp*4?H9Hx3OM7 z$XP;OIeewRd`JCzvFRC1GlqX=$qr}3HyjU0l3SZ=?#n@+O$J2kbRF|l7nzoV zRwmSw-of_EkOoTgdYi-hAjc*NEH1OX{V%Wh;?_;sRd(M6=06$F6K6v+1jKq2Y>(!@ zGS6@)hiuIxuuel!!OW6dd*LPBjSP%!sZe}%;VbnoIdo+Go2z+3To>Nv9z9Hg7_OD1L*^ARv78SATsd zomIV(c&3rbpv>Z2fl+;e+v-HF`ry~qeO|TS3@tdm*Uz(3Hx=Ph%5mpEaw&g6MvPS7 zCC?5wOmCdRF$#MgTF`jyZ#x&a+5GWFDfGe82|FZl5h5ogq}IxMc{B)r=&B0f-U2hn z-Z4EcO0al(nCe)(jZbtJKY*8bt~*f<2twh<_I+$S(pUj@tWC{I;nciui@SzD9L37Z zc3qX=N!e_9%FdwjFuG^1tD#>{o#qss+oL85kl@|dL~gaP3#r83-SaZ+E6zvv0TR;V z;NW07DgDUvWwV!utkRq`pqOw`+2QHVQ0tgTnO2g>)#C!|+bc7=?;_YW9ejPCoe6>% zf>qo9PdFa;*ri=GLO8iq@+t7ot<|PWbd`|?%3rz+>pnI;7ZHv<1gncsIthO zz#55SNuLvEdW)`P;`9OD*2uR-!)e$R5}WyvgV_&nppQFOyA!QqE#kX7@m-8(-gYq@ z-?-KH;M_`x&h~?HNnSn!!Rkk)0{D_Fv}*zzj}l5d`xYKgybS{Waf-V+GwMkj{}=Ig z4`!aggF1xVz~yca^1$NbylEiwyzH*U^X=JN8LfXBVRPElDfCGM;N8?M&k`9R4Ab_e0HP5SrZ+_PDJbyMo`3e(*eXg@E zE?%x0`oWs>cWD!=RAyY)!>)b}oIo#8)Ip;fEh2bkzv^LgRr$(@aV`Y3p&|iC>`{7h zZ$%J_v79n;f-TBxWh1@-i|dq5G?&!PeqKjopLu57k58NVY9{vb8@`dMwRDG{TxJ?C z9sLR5dG-(#|Gr86u7ysQ!2I;XGPiiVR?Rn^8iqWVV-Y3%(IWh?OJb0mj`|InL7=K8 z-GMOKN(W_*R^w|P6krvQ6o+5VETH{G*F2mXNZ_J%$E{`mRwzG&J~YcQ0z5;QMk?=yjyc#pP5p z7kn?Q&vIJmy6VG&@wtCe==-cPp(N>0k-4ZSXgjQ4CSfPDGIk@qCwWXW=xoKg@z?Ou zu7L0nb%MAhZFfrjTj)#5Q<>5;xp_>?_d%-K?|MFoLSRAvNRe?78^g;gMa%E=M_1B- z+0fA)u4TN?uBsyX1w3Hp<-a}PIK`l6vvi7%45NOfox(oh5zo>lS}!!s;VJcog(&ud zRi|h0|HpU7rT7sy(B<9#c>N{dbm{=*9OFkZ%MFIdMOS#f{bO$}>(?PL;Vw&hb~9mS z*EVshG3+km3 zY;GRIc!mK&7>unuCxi`SypeBvuVT(t^r?b+q-cl;oNd9G|LSbfDSrQ9gfAOljBt1K zr^n{q$+ANDLLFu-i9>lhc||Glw_o0QiJDocf1ulh2+e54*^8E%whFlr5#hFM%DnJk zA11N=fs58NAmK*LgCfq0nb5e)G68tx@`_BaQNDe*Y5IWDkzD>Fl^$wI^ks^cOD^?A zE|t=~L-~Xr{0b@Fg9VV5)@24Q|EovQLi#`tb-Cx}EP5ff&--fWlVZhau1~|BC0NKw z;^%0&Z1Tv6%0Xefz`#N7{OfzwjDy!1IxV|vls$S`*QuI*eG zTLc9q6-u{)>&BwTl^M1@{R>Bw0s}>OlH%v)IxQ|G{~q#D_}358_c+;~TL6su*-XQ7 zX@g5!D&Nbj5-3ka#x;Fy`=0fqrL9dU#cMHl3;0%_`jI@BhS`EmjTepx2j8$Nk!UYu zDMn|24K*`ffRY+hQGWmS93dw2sq56nb=@yT`cXrRug1tL-(;Fj2#nIgd>#_P{Z#U4 z;oy*;Y`z$jdZ>*pb4UW_$TB8^@@LztoOg1svLB3~b9!;#->{M@=)KiB_fB7OqVOfX z_3o@nxl&oz?97=2;%+-qu(O~Ho_lb8ER{NTh@`v=XODKlKlSy~fwzVcs;_=#CvBAg zRGdDamS6XH_sNx~P;H6MR*t94TmPC~^)=qo<=H5S5=!K`|_Pnp^xeV3>-Q5U$DdGXUstH#n^@hvM4|%sPWfLGl11 zy-Jpcev^nENxyL_$btdwoyu=ffCc63>kI z?SHfYy63NcgN*DtP2)(12kov(WBX4JBzaT)7=!DYbK5JWpmd+mW)gTpZiR~S_}GCp z+%Gsta2hBH6xC_+Ty~-xtoqExYOByijF>Il3Qh16t5meqsD~*84UYgdB(@mAS71Xq zPS!+E8xyU|vI98p?4w-9Af+z+F34Vk1x{Q9@qe*}{%J-p9`|Jo2|9jr(yy$O+9W>C z+tHCSs9fgg@nMm{zZA1;G3(GwzRby)Drv0pbK*T`}m zBiRBRe|b0y%*(G>t(pcxroXOie42#)nBa_ryp2(b+3m&Q0{UoP18war)=-!XoR7Aa zCzO{nc#m%?FRvT^v6qg$?7oTB9rkr0g2iXmd-d9%J3lxcs9bI%4#Zj=T15q=iuvBB zBUuqRZ$Un|Qs%5BElNN>_>7cMLfWxjJOQFG0Uc$Hi@5y?X_mbaf}R^ZEzHvePtMUB zvZ1+rpd?uCG^40r=W%_duUn{Bdm)-zM_U^s1GzE8W$34m0rHw_ljPOsY;Cp|he8z@ zLDFL^iO%)hQvYiRfk@w@Ep;&qUi`mV@F*XuQE1tjF8i3-Uy6-VJ~pjKdjSYen%WBx zRq&cd8x`o^FpvJ0sX%26@Te_4!D9QDU|XNMv>xMIM^JkYRh*rf(EwE>SXMMFi?0;z zc=6A&&i-HQ<0ub`@s>weO|o|ZcLdy0B;}euoUu!#EF3h*fLS;6@AgBXV2E=OD!Mve z4i1Hm4gFEq;UHfqe;8iEb8YyH9wIh2pmcI_f*?;JN9>zMkp#nbd#_(jFWw=3z&+f| zcf9-m_L#co-%e;RexZ0@>1)`2PWPu6_8&7P=5^^B7RF2k{a0q{VOg5QuWzP;7G3Ah zv{`f~XaL82DANPTSXWY*$gpvb*ou9Ry%Gd0Ng*Rlg!o6XGZvwyi-*A52EFq(*wa$Q zBK?Z@vm^C*c=pkS_}G7W5icaB7wVq6x(SNsyX`*SOJh^_0(}>hH(n#@)1UrQX_)m# z`(L|yHUHC;<;y5NzH5T-?m8Kao;l3znC)g2HhK9Q_hH?O!|dPlkDIUvri@ldMfa+I zwqMK#StD2e{+}#MsmW_+*!p)bH&| zt4n6+NtYCD0f@i}XP`b}kfj4b3=}YAO(CPNBF1$Uf4RmNp4JmJ1}upYbq!x+3&Q4z zJ<0b>M|$4S7k_rg`D16@ zw{DtXVeBmae`ROCXd{oq5o`lNWXA%Z$<4e-@TjHbnc`8a+87(qop1r;pTlnVf*j-L zo`(C?POVVI&V1U=C9aLM2cwsA$0K%sY})pvVf}nj$>LDnU7)3U7wx^~H@C8!^lD(F z!N(0?B7k=laqizQaL%vF_w>Vet-HnPq}Tpzt;L#s09|G(#u|V>|6i@aC>KidwD}Pf z?yg@CjWu+moqXKJP$Ebz#bk%OxAg2s%TxOIGxsp}y1r;jR(k#h2K7cyfAAG9r4%{86g{k1p|%W5U-Ung4VPzMZ{ZA9q_f26J64boFt!<=L?( z@+SN6B;Aib2wohsrCZcXDbN3EkfFv`iHQy!n$0B>8u2j9Aoud?l0 z1g${m;!qH=|3z}1;0P>(eNq?;sE!Z|tkKOXE7}5>iHq3v*go7Rt4zylNxOiv%7QPN z&v5Kf{xvox)YB5J7p`Nu`+>(0>xLCc5<=TwCd|XqcoXn>`DDfk{1?(>TajlTvhI56LnWVpPwIt z@(-tu>j%fPVhc^?3mUci%w}Zn_Pa1i2e|f}ThKQ6J zS7w)_)ke6^8SXeMt<|rl+<=i zryZkq)%ta`JAW;WX`WT3b~hjyen7#xQK89j%cFSulY=TKJe8xFt52xdl$mxKyOv8i z(SV+E#o=R+cMv=Z)?S!c^*)}NwFrd>q5e-$>*xt{GX+85NamA%l|zB=X=t=O0N|M~ z{kL|1VUZ7p1Xo-<%74t?i5F_&Qq0?ho2`~w_?eGd?ZqqwFug}mWo@HQ?LAvSU{UO zdir==;2p?(hM?_Rfp|bm6mCPk55qo@AUmp%_r*~TV^lF^MjBe^amf3~l{}sVRjP?D zK>#H1@G!3Ge&=$IAHe6KibJ_~pGHWYc(jOKh5Bm05?n5)^8}0k7E+_BEpTpqLw|V8 zGBncqypHOn@w3qCZzy_#o6K0(2M8;s`KNTcninW9>a$mA3B=tO^?cLsg60rGG@F54+4dTc7!^Sb_y&W7 zcU)ohEL{!oLC@*jFnF`+6@eM3%38CT$WduEr@%JaxC1;4RgAQgVp+u@ra=MS2AD+{ zo~v8MBT%L8qqR&zVk*Q9rGTU>Leo|@bPdVq@o)NK+GNbP9njB0aNTc>)+C9g^Nei} zp846-(ro++gm!g0%&!0=sv1;PM$n(Ad$i9kA(>}{% z72&Fq$1be-r5@d8-Wg3XGs2TDR2VYnVq5$0vRez|$r^qFaKxOJ)}R-UlXzbC&?IjU z`n96sC>rm3!0<{E$Kf}Gi#*L`x@`z*-Sea}dyS`ulKS;t6*ersk0T4A;}y~vH_ldL z!Fivoa4pOPFBF7|>BY~^ zGMz%4KpdOPohvCeWCxzK7U2j~SB1F?^FQ|jKVrsh7Kq`jmAF-OY^q5p*$s$YxWo7F znd@Plz?A5B5&N*4+^_FPL@TSuu9!X#q8{AGBl62_G8QZTegM(Maz~DRV&%OuZ3}~} zrEx=F7GGjnfYs)EdGE#wTa~@sjAc2Pnv>-`_-RIqE9EbL@B~y053?=_-^IY6PA2my z%sb>f*g{^@A~`qjjm%ztpV=*TPo#=XYp7w1H`VMU&%`$3U68*kIE#WZ#kW*Plo};f zq$E_EV-0S+_J$R2Ds2(&WXs>#S2IM`Uuzbr0PR|lO6^s)gm?R;jnB2e zVzvrj_weFwOZP004~{-RaP&K*FZs~b5IlOU@V+U7Lv6Ud(B#7lWxh4gvyH)2azRcI zYy0>2?=He1&naGl>e2qPJR5RGZMwao11S9REOHToyuI|JtR!j)9n%s9fd+&;ix}y# zR*7$VB$nac+&`f6E}O>41E~_~1<(4^^Q%;YqnZx^rCL0Cv+p@acr!;OF?(X&`m`) zR9Poor3E7lW?jJZ5MiXAADujx%8l>ldZkcJSJzTla9tROx_|V$K&<&=ZEe2pPcw3gr0*1jI$+=8^figvtN`@*y@HpGp(5Js@uz1Jy9nW--(M@aOI zY9=nV3E;$X-u=8G(Z#>v|BRhw<{8DWn-=?b?J@Nh-4h70x$pSxcN&NBqCEqL0t0ba zeK&s<*#-1?qR)lkQVecxZbp-zjx?S`1IHF+WT%`eVW`4Y>$z!R#%wnKHU0g?D&F6* zXAbM;VsVzKBoGL-f}bDYjBQ5qPN)4OVMR%^+&-?$A=r#h9gB}<-xTy?LNfrQ+xGzG zy7T#T`=~;z&Z@u0)pnbnWKoIdlyfSKiCh9_y-|r^ac7L+Xn9@nEjR0L5H5j*bdq;u6&};qyFWWnWJr78RRR$xj02 zi{W7LjCk$MeQ4f27g)W#1uF9C7-w9aZTitLkYKSZsPnTQc2iV&Dg7nV2cKVXu3ur{ zU|fWD^(RSMUU)MZp+&^L>u>CE?u-0Ze#MM>`VMd3mfVe%_2hQ7_L0GKk^Ei=1Mc>t zaw?nG$(YMyNrp81S2PHJ{Ldx1go)orNT=o^7-P>u%sfz!Pq279c*YFlFaLacyIv-* z*8B6A0T_2<@s}Bsj>|TK)6JN>EkE)p?8RZmoi4~`+%jdYl003%Dz+Syc;-;l;&C$X z$WPvMtb@@T;JnTqwkWqAg^!HZw_EVdt^yq{VG(f_Y6_kqdstqnmU#a}vy6BY=TtCDtm z8^%22&lp&d(eei$Y%j%bMBZ21Y3Y_|@!R`>KP`w)Jvb`op{lCd;UjS zuOC1c0-#V`dg{4T(nA%#T-W28y$y_rqgksR$S`}i%rfD*!l(J1 zQ%g!Xr@i2{hJnul^=4z0y-)j|VkR*fbs&L(_K}}p?@wTQ?>`5oXE993A58vFDr()Z z{>pS}EaO(irPL8ucxbr68!;IXFV^X}y(+*vi~7@B5Cf>N z+vq|dv*GRw^8N$R$DP>`me$iS6jgCy&a!fuYxa#2-}NLTb%?{f04*qVuGGz1`X~e6 zv*s87n-(K3WGc;Fs{g#Q5V)b3|6lmbq=547 zWhN92MNIewNW&ss2o|UBCgB&sM3M?``_onT#=ENqXwjq0C{<109>$nEy+hVIYP_wCYp<*FCZ{mJZWmRKy_W6q;gRTqt2dZFb_M;Km$(WKR$jtQ@#X9Dfy67Ea^F-WmvWiCptMDIBDD zJKjo*jqgGmqHim}ujPmazOs!9Nr!vHg853bN(scTqm|E7ma)3+M<%*&2!|m@^_N0p zAil!eO@D>+tF+lq$RHji==?;4tS6Tj#dY1`?E+Q6%i`~Y?JFl#FU99#GsL;D;X?h6opCCvbvZiH^kukbMO0IYwRG7F+93N5VjXpmNz`rc7l>`hkxnDt*7JnmDE6NZ-4P&2=cXz7qqGsEMW2GP<*tE|g7Y9N8au@Kpirz< z(^Qymxh0u*xEp6=2H5@TtJW~?2nl4gaPul|xZWo)s;eKklgbF_f3cH1z@r{;Y}mIj z<96J6$`@5wb762lDPD{*)u%Uyd+lM#CG-%6vO@Cohw~K8_0jW8DExsjfH%?rr@Bbm zgIM~77i}681G}V#yP(XfuK%P`3tjc9!`e77TErz%(b`xpM4Tj5U_|{>xY*Uy^6^Lt z!rNF!6UGD@4?OM(xQSqpt-*uALvHO%7W0k|PlLt(T&yNsS;jXuQ+Rd%no+9WtVF6K zZ1AgfnNc)23bAPorY&b{z~e-dIqw*s0KO1}PPi#i9I^v^j)MG-(Stbl-SfuNN$vo# zv^p4|lRF`*H>iP5-&~La2%RiuJ%SP)D%ehfPZD>#(BdrF<@|34>#wFT3iD3v56}TI z=o;6&jYbLYy+^DKWm>f#sGs#F^6;*jd595rFZX0+2u*#(-yy^d)9ZuTo;hp-4;z2m zc@~gi{APkUNHbUi1;_>3^}m|O8fJfISV5Nr@iiVfZ8rzfmil=RI^emQb|NUwl3wDd z-M&SQOg6?{t6mGoCiM2vdpyUo|Bf73WQ`Y~+Zi>XzVK=S7$XBT3gBqUi<5G3xmHJ2 zqTN@!J}k~Q4A)#U zOR@a%&pT%6;k>OPAYEoe_|uCy(XegpYLfRYQs6GV%T`I`(m7Y_wsK`BpvQN3luSg8 zyd!2!l`+%XM|^K8$VPgIKVgP6m0PX2W81V-J0;&0fQZFgXuNZ^Vb3so$|q^vf7NWs zOJW2jhEWoKwYyR4wQV!=JajzJ=T@1&apC=HH1j0U;_`S(n+4G8(@Ysb_@|W))eE4i=k|Pimqn}!z8Y1xxYUnzi$UcJfHBO z^e3SU-_8J>(d<<6C;w4JCckRJIdAZwERlk`eE0Wv<3Y+0irU{Mu{@YbUiq)oimiHL zCSd|P@49T#dH|kN<$a5QxXIeqr#@pji_7{UhEdQbBjH;{89&+)ziaBM*+(hHssHiH z4(L1=e@9Ql=aF)4X}<5%QClS9DXZ6zUGg2j_c{@#LOu-Iti zJ|3mt>vZ>_^W!K32_aWik6c3~iP+V;? z^iP_lq#Ya_6u)o35gvE|-(E3=VHrb49(u-YY51jng^jsxEXG$*T*!wcI9iZ+T(OqF zcPM-N*P?;kE`28^x=$x=of zP?Xj3Hae>?bSTo#h+It{4Hi6%muiidPfHN0k{>*(&%jgCkNxEG*t}Klm&j$Pg!XIh86ca0T{ z*TcYnPp7FO>@m0(@~`L-$|yhapX2rs%viWxu=>qp5K)oM7wWJpW?n+$O}{p$9F=OeZe!_Bx3D? zEaWGeE00E+HIi(5l z*5U6)N$oPQ;eB240uQW>)|wO0#VwKSK;EkgfS5=Dg79#L)&(7i@E#D9LsEkxLU^rD^t&5+BJlFHd)$!6xjEl+cE2Mk>*|(EJLdr~3OhZ=bPgw-xe4JD zO2+7bdi71(Q0VH0dw0wVIHVY!*Uy+yIYpdQ`!g7>JSPItyMqLF)16Qp<||l8aoLv? zQd-xf-0;Q60dM$weRKh+u!S!1@zoI{V_Pul^I!0Yt-q(slqBC9&aXSHa=4&mxhOp- z>k2PX#|)}r0Mqk|3vC&fd^ZQ`ei)uo1O=v=8iO!ag^Dj*1nbpO^*Q@no>#*c4i!-<#mpR5~1mtT*1Z>K+ES8JTu{HU1X=#tdHg=cfI#_!H1za5c}WpSpd z8Y-ls-9^A#C~UE#862G?uJ!4|4_ox2(Unve9PN%*3$b`q<9l`}evoDNnI8bRCq&pd5_4SWk zNM$@!xrW)w-9w_5bt$HsU;tBlqutCK@@Saj?00QN9E-sI%kife_pLub7B&p1&o4WHKDFacv>h&s#^)nz5v$JPOFA4t?lKleaEt|Ui zk~Y^96F;`p!1UEow??6)I())4XZ-6ON2Jk3QesZdlh{NG3oZ-qvIHM5p`x5g6s`Qz zdJ3;sYJB&t60;V1iyfd&evN9d$~MZYIIHAjxK+4`L#2TjIXT`L*%-P9VB0ydm9Ob% zFoW5PdmOHy=8`78u1FGG#oSbi=w&^c8SE6sf;jslCe-0$tsfVe$VqXk>4nvES>&2E zXvu6|!v>Q@(z;>$p1o8_r7`P%Prz1&6Ec+4wM$;n1LSl=s7{?uTfw-0w|48}m9?kT3kHv*lyt8~n6 z_DbVLKv} z@B9XS6HM0;93q9r@2Vxelr}V8$^Pkvjx%#MS=(@4jv8GI{Rm1`WTumT__jE<@2dXA zqZdaR{3b#v7c{{IKO+dP_lrdJSS)4mStTUb(u_kM`w6ufu6T+C=F*iES^ta7cWa)} zs%OF+<{!*vH1dwMdOuOLXr&{U7YIm7>VODDWu=IEs30-#S;CF^(4}4w$u1JSH2OTP z!65P4Q{Go(RvF>JHLgpV5UX5`FM+tyF~I@241z!kYw1CE6W2ajzapi+8+c`h;(S0c zS==it7~jTYB>lu?*MmKu%U1e8q(EQO+CCz}jtDlr0yZ7Nb;jbS9y1;U5n)p3L;Pop zc9ndKf!U+gPfE@mO=!FjZA5L6!DTR<1B-Gzr9p=q$TS5_kX~#XFMXi1un+jXy3D3-?p6*KO*La3=-{Q3y%A~6B z6e_*$pIWPp-)QEmL3@oRl_CaGj5ictf=`g8HSGF~u1JACktfxpM-zSj^+KR8y9P7k zv>dbxWGW6p1Vr!Sh;V*qCN8e)Zov)TBM&YPj<6NfTZ?2-lP0Y9;Lz=o$6-+pgc;tE zxcsI~bq;fU376(d+xf~;R1(yW-;20~-@YQbS8aP|jeC|R8rylD)|3!%b=m;sdB&@G zPcpVOXauQ}%VL+{h`aywh$kn>E@vT`2yh&5UETJZ3ysluFM=l~?z5&SiVm2;_UPMz zsN4D9LMWi74Q8v*yNg9|pMxo71aD|>Gf|!GdKz&l{dzd{K!eQFOC<%ml|0DBx_s@833Wy6=5Q7Y7?Ec}!FX`0a}+Rz8Dx-LiAbGooNG1s0Do`XvDn zb4iv_ci#GSZ!3@A0(vGkv&TR`=jCPbvEm>= zM(zT$n`Eg@{y#2~GVQg~JzMMobj=V&cJ^535Z_BWQ7-yuX}ChHAB!T5@15;hEj;`7 zY``*8Bz8LlcOy!nA}!$;AfYf~!O5x4 zeIJZnC@nFqJk#B?U3g>O1d{psD`9J#`!91%D{LYVh;DX`D!dM0)lz~ zBvE&F8L2;eNfUSg)IDnucPjUJ^7!d{XyR5>=FRijNmT$(S2lbA!(+lux6+^Nt`270 zCO+os1^4|Vlv0eyUn-j3{i zO6ADGmvcCy3QTto@cTvO%CApdEJ!VuqZ*zJ?nsFAjTm<1DCn+zs{%+-gIZ9DIlVQAVa zr|NQ-B~HAcNlL@4j(&xGBCquR7;Y>3jQKYj#Y)wFGmEt{0xkuiHJ0Zc{vrxg4{m_& zK!g^8dZ z<`wJJfDaC({1=)rb`h?tL^BNiwgP$T(4g)4z97sX5?z{Ec*TbWExrSzel@`z(cF1O zR9yE^OnJAU^6@0%yx7)jDYkQQBUZ+jAw~R5og76!r(v|cSlJ7#C0iE&1Jxj9cy&^@uXAV16F`H}Qj#%Dd9coqUe*?wc$@=1P8eGj)r>5ThX#XYf zew%sttVpI!f`^)?QuCKZoq|8}=qn%E)jW;bxKMG@EK*>gG1{MHuB8o`VPSPyJR`2_ zrY(M8jYuk5TNl-)Zi0<#M+H37@4!xZ$sjhY{)#PO2nndixus@s(+>6KL_|bZAYm4Y z11#MPpiUF?V zpuHw${RCnYJC;amc;~5%e!OWH45}3SwI>9HV`~I$tn!ox`GZnxsk^l?52tBHsc)N% z*wtMa2do1Vfd?;YQ=}_dEY|mA-?C+)_)CYGEgEt?TV5C4#QaC_-pgg!OM#`S;`37d zPMNxjI}mamn$dougqX!;&C2+M&MHkR&cEMRbWT79%_Tc`pJ@QWh!oTh? zaZ|t6&3^fhaeh0FVGe48q~P+IPpjUnknVWq2CG5uIQh1>l>y7o7^?nAY8ow|WYnNBPo|+t4;TbFcWPoyi{qYV2R1M~4j8S4|$!a~Vrqv@L>J zDgB!WZ@bzP&8fw_gS*`cC;C@UfBPlQmv;Vs2XT*}%`Ym=3#|$SnM&$&Exh+XM8ddq z)0_O^g**+RH(;Y3U-PQWR&GN)czceUcF zI9<4kIm$_VQjRJFgb4~^*>(&49g|AY8#7c6y*trUbE`yOEHF>BwiZX74!1N&ifxB1 z(;SUS<4kwywQ*`aO)>TdFW_zMr8Q??)`vg0{|>2_E<+`Wpgyg~_QYvq#-*Lrl3GKN zp?-Z<-fJeh*Nisk{2&=L1z=MBq1%mXGjWXbc}3ZMoH>`|8tZ@Y9A~4Q+9C=7vn47i zXT%so4UC%9lU)b%K5r?i2Um;tILvqx8mF+Z;n5gE+0jtU+xE{(*3yGL0HuNLm9KI2~=+y*?j<^RT33H|N1 zYsLWG4-BIjq!%t!OZ7S60I5T+8J{0hA7uvpno^-p&+PO8l4#N5g70<(#a7Mi*lZxN zSD(0FZ9h+55j6XZwGwALynYU4F5h^oe?;JmIfP`A27m5g;Wk9ia4^|$Fs|A*5`&gj4TA;2 zg$cj6*b>NGgUxU-+`)Fu%Jsj~h9O8B$b*g49VbOoL7Dz`uY|MM_WWb%j>G1->vyfb z)VIiYJjNzi6(FpPx z&&COD1z-`Raqc$G9NuLvvhnjbscl&d0JHOZe_#M&D|E?n3mhdmr%KevV75U4w z4uWZgUe~qRC03;c6QU}_jG#Nh$DlH;V|Hmh|lKu4wep4q)^9_f_ zIfaJfecXss))|IrpzKR+&J{6nw&76{F50zA_#P=I2>I{<#2S4wIacQtO!tyMbw9CY z^{#{>UTnifk$9U~?RRCp67iJ|lM0|ZndqBR5k^ELWP{`Tz~*})ZL+5Y4cFalmkuwd zv+UH1ZcDruOdB^+U&hHR#pG+u85HPW5S@)18!U1Z?w*uYYR@*~4m?nkK`2$)>_>3`V)j>Ot-V1XLA!P^$+K1%@tL>miA{QhJSOpHtL19QO z?QvW=`IleVn^m1L18kF-tt2S7nU29)&$9C3=VPiLdkw=aSTNf-S#j$xAMgvd@vA$x zPTyFv7;F4BhrcXy>LY{7B3wy#9VwxIuE34I_b*j+#&xFId~h(^9 z()^STYFy(P+r@YI1>abxlv-d z!i-4;JB(g?1?Z~H3`Ky?G3acc8WAYduIy$Un>?3>Hs}Qyx)=G7={oMV zRPoQW%M_U141~tPo|^5mlnjd_?Fnxoi%f{7upE#@E^6IXJI6GC=j5$ z$|zsU`@J;Ug8IBXzv)EokL!cC(u(d=X>-VO!}dRTNcB$|y9tF<iQ zj0TQgruzA+(QWMJMSWvFd%Def(Cg=$S-V>EqDQ1wEcx3Zb+irw^&4H1$X7S-U>0s) z>D9PZ<@5GYO4MBisxQKQ;RqjTC?h8Wp;3WFf?n|g*LmMNtXDR#w+;5S?()0*`l2zj zTPO2$Hi6tw(OIc1A;o)99LS>8lv|PqYd>^HA6}?rL5UUYHizVe73_#WC%EU}&p|4b zdFL&aI721il!Y153rXvZfA{!#+UGiwlC#lX%|<5$kCf2)6T_uZYBBujycU?biz3OigX7}{6nd*_oOf=P+B3lgrs z>*XX^lsMD&;}&aV`*UXKzC*2&!1K2IYxfO1 z-vLC6|7Z7!SB4uDExu9c_=q3Pp?ig z|CzoPNfj-12cq^!0$R^*OuGNRA^6CItin*%LbVumQslnM`(yUu=}C~wK=&&ugAOQy zIW9c!Jfz)WNG<-e$&%mcR1nq(!hO zpITF)Ho{***3mzpb}j?DEI$9{;~oHkQoDt&LgPxu@0WiXY8@`Wx((TL=jUO(3V_Yx zj{dIjeBv(8!n7*nX@4?qyeM>lL7rIbE_FEL#7mf1Abpm;)jwBq$WnwUH4eAkFo86+O^Y}Qr6-o|T5-#jfzj#u4%=6l3^(x@NJ>a5-6e=fN_U3>0@5K3BGMoN(!J=CZj>&8MT3BJ zch@r)d++DI&$-We-}lS=X@A&1T>m-e=y8o}+ze!Gz2cH1Dg?u(7FjGGd#)vLtF0%a z6-3TWW$qJ!kV~=|>=WCr=I;1~E&q)x9k6?ljv<4wJcGTrwOT2uXg28 zK`FWPJ-`?47;YL#E{n{$+M<%OHQ@^?|&q4qgTF2dbt$D4mqHfqE;&);B=dQkB z+s@{An6yC1ps-@SOd_8>rRVfW|&kAc9YyhZO-oc~+yG{V3mSUA?Prf8ENTAGEY|~EYzXUn(eou8_oFiJ*CqN4 z!zzlubDX1JDH+t-vwHFvXV`C-+sqREkx|5<5Xz$8G_`t%ZILvjm1y|g&R7L6VRE+H zCB{H)I_CN`F=RJ=xff%to%p6u3zIUDg^V8$wmp`J(`2oMAiOu9_)~p{eBw{@yJ>pI zjl52kk`aY0FeY1Ye2|*NTtB!VquzYE(`-plA9#LRe;Q{OL%UqgjSbjVe}f^CZ0Btv zh2q}z&);zLov-n;-e8C)e@P^EKFz?%O>t-99to&`yDy`+Xs{n>zKodh&Dsr;!V!}= zOn#n#xFlK2)@Lh?V!6h|APXTQsBRZN z$iKwPeg9En&>;$>0s{iTU=hq7#}PyI>T`~Ens-zZCTWt8w+NYUq)QqvH01BuWR5X_ zw#BW<^Eo=A1^!ZGH!m;Ge7*XQJduBGF0DU97ao)q%3s;jfiY^py^tzDCI@ ztrqRxTr#_#OyaEfan}F()oGJHeP37}9y#wkvgb~A(I2FV{^F=S+;nZDk+D}gev{8q zVS?<4fg^mScyV>ujfU(rzyl&qsN3PkI{fw!0EkwLNeA1b18z8XkbC-zv>Tq;bcd(y zeE-d9!IpJ>iSsL%p=xZ#=gU!LmCLSI-BPu8uAg66&#UVh2SsW;AhW09c`%M*vIqag0z)(7sMCfuqO3Wi-FLM06l7JhK0e z4FT$TUDo1rd^Dn%+t48NqweWr(7R73R6@yx-)Yd`vjNV9BUeTgz*a}@3jkv-sc*8? z!JX>xYb=2h76bfFpCoB2t-7)EEmlbs@oh6oFxJUEjFi6zOZNProhU7s1vXjtE$}V2N^XO~#Z{m*Jsr#8;}l(y zC>*DO&J51_mog7-j(0TOcWNWPEDP!kB_bF2)g`+3VxUxY?2fEv1&p3g3zrO!8z^Q1 z7Szb%T6Zz8qr|-3Ft^xm~rX6l>NIP zq!ZoL4(4^?G_CY<@rLs=pueyI_D>q=X@Z{Rk70ah5CNh(N*@jMQh2gH#sS@gq&C$* zwPFlG)M6&<3n-PcN#X)%LKY4DVk||CCPi$o?|-W{^3PYetaHlqY^*cN(|!R%x|-ou zf!&o#MP9xAqg+;JnlfJcSFO~DGahXtX|XK*HUnVczPkD;IZN4ZS0b(Fkg-4gZbT!V zT-4!l)bLu%y7_8YuZOax3AbK^+=8(H!PDO#Z@uUl`HkP*XAyrK8-cV8reWt2*NQG;nS^D zm}ItbdJ3Q)aCm6H`;?T^9KCFsJSFS9dB5d;?pL+-r>Gm49~RmIxd8M8=@N{6Td@mI zfr6+YHPSqoI^>)U;@A?^vo*&;k%{JW-Ai*%F9U5G&2TtQy#!*+ECFAYz-IbTDH)T_ z(5VFgK=hEnb_2gU&hR_n4nAnX2cjB-`7BjP;R%lNHUrD_q=07w5C4;(-(b$ZzbCd$ zxiS+jpKiG!I*HaIZBM)StXL8N4((+e$E~vdbzb7* zoMs$1*7URin!}30nPQejFq*2l{3;N6_xw-;SMi%WUf}BRp7wsLsoZ?C&C_w>zK{cAfv7t@j>ZBINj=-n#Cfz$T03uQXpH zy?D5s!bqVZdT==fWu$2CmAFfOh!ZVw!IQm;S4{6k7Ek~`2xTEM0xqLWi{)*aY4=|7 zx#d1SQ}(&y?sU17D>KQTZ@#JZ9l;!+9ltVuAL8U7QdR_pLDRu9y3S#fP9b?P2^pGw ztq=CGI(Yt!m1*{xU5g5`WqI9h3X+Y!!8alk@K|LTLl?b#^t{Dan2Y+#_+mX*>ZnsE zRgMW-LG~q>z&mHv;GqJ`j;x{gdNt0)JG5S>NeW6p?vy;17*Xqa$wqo|L3}i6L9pK9 z>;KVGwNO*PU^mZk5$l%t@HFL>((0qqL8td&4z0+IYbnioZzpW49=rPKrUa-cNbzKPd<>7{YVTQpD%%mj2_uQ;~LFnubdlnk0a@OuIx?)$}H#d#XQFOrT0 zvVuy!H7i)ZNBx&LVvk0?OAVN>GFgEk%ZiwY&uP*P_WZlQ$`K zX!=N_*{g<`ba)Ef-N*lWcK{$=u~Ka6&LZr%#_G62qG!C$$YwoT7cYDt1)XTBHu|TFilcBWg6@hTiGN6 z<_riPa$gA6-QMt;AcInm0+<$z$oAJXlzm$jKL&Qsi|Su~n|B2=!wO^uAo?1oEaI;a zO~8=CvlM{vNC$Nw!LQhxfC8tq3Uk9O&}S5Z6XL|o6|))z#0DI+ugWKtBpEghKQ^o| z66tOPZS+Q5s06K7+eq2mrtC?7XtF4iwo;N|5*hefl8vBe+hU$W6p2-QwLDQ*P`n|@Ct@%8ZZF7 zhK|L@B&x`5xY78-FXK(^B|Hpbf`sQSHr}A*?KUlOXCy8_0d)j0OcKfRT8%}1yd&~@ z_fRKp8>5>AR*@{rlX`nVIQ6*7-$30!05u2zJ5xVmlF)Rw7+1xc_r9%4`>F&MNc1-< zdduXnenqf@As@Gg5RU+IPE7JiIFz6}0K_N++GtL4FIqNywC;D|2E;^h8L)#D51gxC zteMZ~XieTV?o|kih0l3_;_UuvAQu3($snpew25r2l znXdg!!udN$Y93?*IM}_%iT?&Pf?zn&h8HT*;g68?fO%6z-Wj{+jAw<~m7l@d0aK_f zz!gCOJ?lp&eTF4pA~SJMkn6zF{sI6}jY+8})vY~m1j-vmHCbOQY?Rz?hCZ)DVUr5t zl=QwBNqr@k)0N@AI`9sJ^7OD;6&+yas?z0n=nT_StnM4;UWKUQO{}d%xf_cF7CoD+4W>;MAhfsyWI4iA8?A5vBLc#W!Q48i%n)VvP58P-9p#9KJR@Z0P1o%=!{ z@fvUQIo9=YiT9{2dVl%M;K(F9V65m*#=6?Vj$90WVb((Kt^X3l1YaJ5m;l#cIO0a) z>RIL&K!}e^r@z07=mke!2Byy5Nc`32Esmo?J_hR-UxU%_ZM|C1z^1~b-2)1u1PV~b z>rYw8!D5tJFQX7kG%!Vw2^Iw^C2*LT#^#m@{mNj14gzL6bNIHfF|hH=awI3rNN<7w z_=MjQ&{PtH0wi<5+)ooaUZ4>HCgQ)vtGR=r&*0taZoeW(WQs(^iueEUmAJ6vN~SaZMf&W3f4ky0Dr95^~nkb?zh7~O9w&%z%4UE z0nOz2cVMO-<^UkY6AgAaonKM!7l4UL7*>W_(7ooMqm?E zhADF&zr)HefMzoG25KGM|4E!`6Lo26NeQqpj~W1IVIDw(q_>ZXP8Ypx2HTrvb(z85Q8U%jyQ4C(GfT2R?JvYz* zhcDgG90fa93BbHPHyE@|1>kE!9m?ikO`aYC3jqYi1x2VEU~D?MZkI)~)aOhee1O%T z>m>W0CuP@J4&pnj`q+{1I=&e#c)2B4fT4TZnMw>d%yej)+mt`!EUev+beGAfx1I`1^oyDm4L}+|B0z9SW2D ze&Lkx9RQAFv@x8*7=S@b3P<1|BTWJmJNs&{5}FQrnAgF$NZ2wb0Zfa_6HGn71>0ho zyw``b)%p&N?dJJycN$Oi>Bf_pji~^rtCVVkbwG@kp}R=bBSN4;W7{F;3q-7(O0VJc5rxzwaEbw)kd^2k4FiBjGlmXGd)4v81pP20D0zQAm*is?MM#R{`Gf|_g<*L zX%&aTWLK_!xCj`vbOg@eVU!Dd7D~7hpfm znA(Sx*`^tu5o6C2*K=93!VS_t8hQ;Tv!?49s5k+}2MH0d!e)ai1__)dEkXOmLyp{+ zpY9PLi#MjF`E*23+VYv4M9mcG@B*NgoS4$!Pxa@?@Ib5rFpaY#_rvM+gXOE=P3MU0 zih%jJnrV67!fSvAi3YXPxv*}Sr)`kxF7e5k5P zvmOjxz3cx$C-DXVs^^KlN2g>h{{8!j!1vcbp#rAchZAw@l_8j`|Jar+=%W57)F|8k z=WT@xogHuNv`4LKPFG>y;X{sIxtfG%*sU^CUG@SKN0zFoLUgeerT*45_S;Ho?GpffMNjV^<&BQ!D2sauk$l0Bc4;)aAX8P8;q#FjN~vN z8A;stZn#qddK)as4d`kBB@88pQG8OebOZ}KW{{Vmf{+?F;9wtHox&sYk zmorO19002WNJcK)ns`t%0y$w2bkPTfH`4=Xc~Zu>lLhqkW*uylkq)`7Ty;1K!U zWO*P!nWQ6r1#E3H?M86hmPXF*(hx}u5w)QG`uS`!miUT>(zRXD)JQaj*w_t|Gvp1f zgz(p)XdyozDqIXiFbbKCj6wA4{BRX%5)2>a&tO3mewJn;=bO^QjIvQSl}!AskTv%& z>CDDl2H4MJL^+7TK`V&S%WGGMSaP;-H&6`)tS6qDz1N2-Q9GznN4)n2+;|fD4(`hT zan_m3|Nb%x>`Q?_2UI|rt4~czxQf&P8QGWX(KBNRNR%~W3y@-ITlkgX$^cxg#^ZW{ zg&|S)CFIt@NyBU!FXv_C<}MToXzs0gfRf9}aO4a)l_8EVkmccT+*W=Ztuz?`buGTS zMgGQ@Nej)XM#-mu7!@k8JeX>r6|-g{Y8|L_`hzjYhO&^UPoJ^N95(wH2FSGv5n~D2 z^^;ZxQ*Dcxb~-OSQ&~mhTHYYNA0{NiF0xZ`8y?>X$R<{fDfVn zPy!q=DU6lRss^lO1b`8FE&wXF1mVFiCtjRrd3~`t*=p6q;1e#{Xz7?`9f)dcnH&0bcnbiqbOjRPE zeSv!u5UCZkWC2J(&mKj`@D3Qh%R3{6&jAB!KKB#I+Nm1{-iDvS-lAcglA~6srUyoP?R}GcLU0G4lGj z{}>`San$3kW-qi?q$HHJOsJ^97~S&)(0^rZZSDDof|{h7g_>n)4nx@A#bT`kF5v82 zm#4cp8~Px_M7tTejt@{eU4-T+qyjza&Y{}GOX7L=tz0Z8}W&o8}a(hgXS1JQRIyM!gxtRi`KER#kkA;T#O~I-6&iQ}069eGZ4N*s%y}h~5 zu{RH2!f6T|cV_sP|73w6Aj&4}T)Vs*;~&vwW!NoW}(uf1N&nxV>Q1CAExCoFc z-_1BMo{sXWCx!tBlt`ZVD>k)YQ9FZE$pu`1aFg%>7wT-jPBohpfzoqF(YFd{=_y># z>9DYT;UYKrz|=(JSvNlz$K|+Hf7Tt`Vgkx$yuWw4o4`@X(y^aHnnfvtHGwPo0bJ3Q z#o+D5E^**_2}O4b5Ve5kccPXG9tSZVn~Xu4u624}JykDEAH zR{>>U`j>TUjto%ZZA@5J0@miw=N2GCvOtN^@+*wnKf?f%S(-x>{9RZA1U}$2fwAI% z7GVM3(gO~=h++oLYV&9|&S@Ga3`A$V()_2L&&8{zo$F6&l^(0(EIdN)<*3Q@-mIDJyS)y_xMuxq~jyb%WAScM*#ie?rAS1$*(SUE%W8u&GSHmMfb zUwQE#lH%HhVT1oIe#;sp$lnZaLD0YmNE27s{@vIG9ad6sRD{J`cy4DmP@PRajKry<*(WP3#&g?jD#cjY96iHA^d z;LC3zw1muyCjXtXOcfW}U_003c)48-H$Py;CEvIl29`vD%%9|df>&YV6d+Fg=2xw= zlo3Sh5Bv<4kf>oJ-eeL`K@XXiHsFixP<-JrF`VsNRTIuQ2`0>u;lj71GgZhZW>8YoFkPz$hgsNNlpU@f4^IY3Y3a%@@MeK~xMOqoe1;&$^=v@9U~ z)m+S#z?Tpvd@VV);FSOh?q z<+ajit6ss$}qXwP+cvH7{nFygc5t*eNdnA;6ua@ROCRw*1SD&3(k( z7|_R$)&ZJ1BV`DRjU-&5AK zen{{k=ZA|s6J5oxVoZPm03^MOfP)zKG2}l5CH_1Ourm8r+;ivjGg9v0cXn| zIx#RPV(Rl!isW=SI=U1}0IJ5$iD#!?{lBSNZWo5RyUKm9G;mCxk$J_fR!sar)>!H5k75`Mb=^Af&+%hu5=uJ~!&q zyMq0Szuc;X@O3Sogm4vQp0ppcpJ^vCy58h z5y1$_hI%;6m#Xq0Qs3yh%$!pvG=CkL99Hd>8sbMz7Vc+YU9+wWUI4v(prjfIl#{n9 zGl)SJ@Jg8|<7IK(T0=iInp`&#xwTT*T-dZK(&A04FaAq_9B2dFtwpK2>A>IYiyHz` zo9`g6N$V+YJkbUqAEo&u@L>4qO(66EL7hfg*o3c>aN(8;H+Zsq+T92*oxh&!_x}%1 z_7lYyp8ff`8yBefA|gWp57EWbg%3~@#Crq?9gqTd&GB4&V8mjjMb9s7)QrZ^?GH{h z-KkKs*bsq!aVr0ZeyQ|gNT4Q0l{pup$paF5{grx%C|rbqKK_#qiLy&Tr!N<}W`Tpj z^@f1O9kPIw94Eg9JdkRx!XqaI?_c<>64!tiRlQh0l;B=zPLP^sJ_8V!YqXa;H{d=* zXDi~uNp*$kTuK(_E?Bh6e*ta&V^F4O`QR!E$24lDHqR|#I2`>Tq-_fo(#?XJg$hr# zImsoUiFjo$g~y`Qu42hw6lT8-MR<4j}eY0W|4hxytsU6-^FL(5f7#oFK+(@p`tO z9x#IUMvL&l-9QtS92OIA;f5H;0)ywm1JS-EkPzk}M{b_58>!_2KW-&CGl`zQgB`B? z8R!uzqBv@jsJFhBS1P{)y`6|+iV#!R>egh%Xl)-oO#ICMj5h~JiVEl|f?swvKt2BQ z$lbtENIh_pI5|!MM8`g?kN~A>JIcVgfq?6tp;A3y z@)GC)aow+4Rro)7Cyo>aZAn+caTM_L>cP(&5~d=lw{ds5WV~*b)ET78C*)TkKsa#fd~Y~@7od{B$~ii za~Vk<%3x8FJqrHUJ@6qer~DwY)Dz0)0wx+{;8b>?JrB@TuRk6ph(rA+P+D!}=w!cU zfti4Zirpf#Q~`BD_q#*I3eg=nPPL}kfB=+H1J`3+1B^Ts3H%z*90&CU+)u1IocXDz zlIUcK>g*OIB)-#;kWruUQ&!T{I;b3SWWOU^B0g`aQJy2Z_u7U?LKt1lUJL%P2)wux zpw+Q37!Rij)o+{$J6;1br^b|!W(8t{56|H9_Mn@Pt=P@;2!R^5{Q9LiT!>?82Zx>E zAtn|AFv4ZW`u1pud4WD%D0$q!hjqkT&=7$KZ}_7NPJG}fqknf;YbrzfiVCyd7jQCp zAhOepbSF9yj|9K^!fa=1o`Ia4?L@*eNd?u362}JJQU1ZogNLT;8C3K5h`^NS`91%P z2l)^Nexr#(e?5t6o(cXNz>|8jz{~%Y46Wq4*U2x9;E#X;k*L*qqFeJ6(0l(loB}%? zV_R+A{|+cq#*+&PL6Bca%dmYPG2>~k16;4v@!;jG%4Yjis>mMHPah(CGTA+eAzRdc z=l$=AC1!u)i0uGh@B3525M*(H#;7Chg<7Fz)O_bmFKE2D3wT~QrhNuLVI~dsI9+?0 z`J^ofctZE56!u?qQ9rdM=`PY05kfc!;Fp#VoDO3}eT;!fL;1M(rHml5Z=ZC_Uq!va zXxxp2+stSpxcqWL!0v`vYDRg`X%jj$X>wRqn5wn~jW;0k3Sid{x&Q+TJdLIZ4I-x_ z2IpZIcRjDHe0zC1vas8nTfn5AA?vVwWW^iPWB~3l#)~Qo$xnr?aqWFgyq+1& z+D$Zgu#!cIx_&1bk|Ljwx|9Q{<_RF6#0by!#DIZ~G01Uzj{DLG!j-NM^by{8#>EFe zOB?yyCJR`*Z`kB~xT>mPok$)CT*X(HsJMWp2X#md(4!#QF!hCu7v*kyIFF2STPio^ zt3@N-Ln+!4GNih8z7c8^G3Q$Dj_tLV5j@$-jugkLq6~JiS!YJ~&@>96uiCteq7yy=>s)|ImTiR>65#CY=|JMQlRa zX-xjTUZ+6)>$j?v5cW61os4N_QRXfHjt4g3BimfO?zSuw^}_FT^(Y-SC z6vYJ;rcdsXK2gTqdxU7ygVKZ~E%5lx8NtgH62hLEfl4^;P0Mhb7LAJ_4+Y4D(6M4Y zSwPKhl}1c8BAzcOFC9UV4RWpfgeqZ#$6!|gBss=^4=%VAWVO*}s5N;{Bfjy+5Ebh` z_pH-btgf<6Jxg|(X#09P1OAy4GS9s}$-1p!S=P7{>Jn5^;lt+FyMxhm7~J*wL|App zL_Ki~iXSfr5AriM<6)wmb05t$^rTF7>v?|&J1c~`E=XzrocnfpyGwu3%=2!`H0)fz z#%*254mZ{`<6?Q?hjnb``@(jF`dmy3(Kr*8aDCX}7I9~BNksa85Jd9EZZEj}gE`I%d_5edv@keT^crO;)gxI{n ze1Z4XzI0mrMuYNpaQ%*{U!&N(u8{Lp#?wwD<~$&D&XAceIMI(V-ZldlFZtqcg;-<> za9={vF9se^>)hq39^amBJ5h^!?g`_E~P8Kr@G)MGN8+Bk&TM%fA8+-mQTfPq*0hSzWbGby}igGXHawaPsVHY3z}Y z+k$Ic1Ow;sh2CppI(biGHaj3zYXe9qOv=qeu^PhxC_B9RRgI}60T2kl-TZTPgL$|cH#e=X zb|Jp101ySuY!;jt+ydn%GT3p!1XGz*Qtn^B}*T0#Gzl1G|LQlE@`rfD&)l zS=Z>#7cKIETBuG|ChuZVMcgj9pEp}JTo}jt$pXWoCG`XGbZ@)??w)HU zJMKwXwI=so$HcQQBlBN7UqVZg_JxdMeMi54)(FA6IW=9LLf)}$TX=u;Q)WZh^1;IS z+T@LPUA>h?ff`@aAUk@6302lTaVlh7I~9?a2sK$D?nKN2Lq+2Jl+59a25OTz`T;HRXPkNIRu3 zMuYYya5MNb?ft94XTZrY-h@SxKBocxx`mnIF@|V^-JFG>8PO1kp|;bbF+p4qX$TJP z#I?(JX#sYGI4H@=mP+X5DQCmWkf3~*tJ|2gy1Xo~Y|{H;n|HPVC%CxRcDnCk;2cYO zU2%nKKBsMqC>^_ZdT=6$3w6A);d9Q*o2#BisUOS*%o-1)A}^xmY*Rt4@a^RZ0ixi_ zK$L$mqpA|JJ(rFPSuelc(x-A)5xX{q3rj9;y^H^W*pLTvLO+v!QwSQMj0OYf9mDRS31zQ)Lss*vss7 z@#BVz$cLMUBTb+#N!kzF3`s<tBQS9Bi)&GGTM2nw}t?W%zPkVllXbW5F_~~_x__&;H zZs0_Nf*7LO@73LY#G~Uet!EZk#}T6>eSJ$V{Ly)9@_FV?Q=<0RHD_cs0f4yBKF^VR zP4GyOvq|GCdypkQ6FhigRLXwlC+0^3?%Z(a&mX7Fx0flij!t3UR7)%g56$KtxvcCy zT8bL)48`6YU{=j_*_xE4EYTs`!1oqqPgj--QBGhkPK$2nH?k8YVnPbVtlIu9U$g;p z+F~+m+tPU3y_!0rH*PSvSk|q7D^Iefnqbn^!^I3KxR=h)M?qKNL#Dt{-mXF zy>tZ+Srx=jLa6#2sEo*IN{?cAaR43jPjwjFW#DxqsfEM-| zqT8IbFz+Ow*Cqj9;?77$@G@%H7YSv}ioP&35A+jMqmc^Nmqvpm<94fyfXI*y!8Rg* z$f!to(v>v1xo&VtUEE~KlUdGXl_|@GmzMO67EZJrqZJm~)}CnBt$s9Y9;$D?4&Nc= z{OGkp)vNTuPI1z>eT}#(|ApP4jKeCERHp3V3Kw*h)mI$Q^sgVTJc+(;BCCI$%%lco zVa}z3X}rJpQuB4{lFPg1o!c}rooei*9|}Tf$AJaoxO?0vlvvVGQ4S|2ogCW?Ch~@h zD(pwl7l8+kc#1ErWu|2)i?p7n78)qVmHpujDoU&@*|7__IJGXDyG^iIOZ1-`R%;;L z=x)00F`T7gjW|#TErU8VRFHq1^8Y%Qo7xX)OOn}>sq_<5roNM>8Wp!zUHuG7!C2yqW1N`V zem=W%s+23~4>%5Oj|J(=%7zO`I_F3|<{DRYtRDr%5L*)(ADYW;xlr(Zdd2bj`e4tf z%HD&TUDE1a)1vTHW#;c>L9?ezV)~g4xJ4|Uvo){qIDYJ{TKuxtAIc*@{Sg20%_3S- ziGDG;@cZ#(xl-a^)o$g-As^-macBbS7$TA%{MF5Kg@A$_gJMcn*w$D}{2s!m2TDgE zd+Zqvd8SCMUu6)ZNBS^EgA(85k=GQ+(G)Wi+&$l`7&&K9`7<`3?;9P>mYKkIDdqGH z2;}rlObv}3chB};y^cTrX=fIdEp5HG)Gg;@i&c;L(DeHGZ9)diz?`S05Mjd>haaJ0 zoVdT5oiy$d)k6Q<$&X=2Ye(xn+kFlDiud)L^Qm;4&i!8T9`+4Ma|SMpDM3T@|Dp*=)jVqNyMD-Z${JRPZ)W9sOZ%h*|U8Pm2fd zukPz?>b1_LYeuU_ravYvI;4%QT;JphH1nX(4?5W(tWguKAM%U7&#^b)J?D|C9}gD& zTKx{af4Y^&2D-^2dL_KSDeNhCLZ`T`AJ*LOy?(z%=J`y~^-bWzYoyo1848c;q6rr@ z3(t`V`DVb&Z{gnWW<$+_z+-vy7U2DNyLG0kc@0ix%T(Obb@z>h+@@Jk<5Ywuz301L zPkAqua*+ElmwsrNI!o8F&Ca>%e@d_U9K5^|30*wHO4zB^EDjyJWMJWq!4~F8Y zhZrySpDGTWWrGfr&{6D2k8#`SVqNMszI3n3{2sACI5u6+R?yVTU!#V;2u~HrRwq*M zcKwtqvU&~2jQE3`Vt?ps2=MrzD-$91SmIHo=MUN4(>;3!ll+d}I<{3a^K^VvSKNI8 z@h3rQ9@zR9wsQKOFGHQq-Ws4fiH4bP9Jk8{P6qoG64J5g-PgjlDo|Qq*D%O->lMki z+8Ea$;;mr_(=$9KeHJ~Rzr4-R`NWJ?^>=-zaGq`&W>Y1D-gqI&_0O@b&8D;H>eV%E zFXE3hOyYJ`O{XGKHf%S8raf=>1b0-$t1>5ie|@BMs%g{$yhv#t>8MUM`ETb+M8#`) z>S)nS|2#U&RTL~Wcw#)-^HnXb5B-35OCnd4V6IdN<*m@(O5$^4Q6Wu&LmulB+g87^j8XU5-5bF{}dT%Zq6TqQc1t{t0Uhb;p>^(hyZ{f#oKC` z+Snnp<-x=tZIHm~-Wu||HN=2y*>#cr7!9%eF5s%YRaioH(CW6^s%-F?;3Qb>OV3WV zRkG`Qc!65a^n?J?ukMUc?X=29?@Ojc9iF`WIAIx?!Az?kgWq=0Wo`%&`$fLSm{xzm za-iMZjMB0sEi6;MW!H4l5mPbp`HS{^U*3-J^rZXMLNDBmmP1#6^fabVReheVHkKmO zt#7%O=MxG~;ge~U4paU}D?Du=>*9i?KTDFor8#&mC9?M!1r+ti7-SFNkPHAzMjS$Q zdQcBYzXjSk2L)P}WuI3$_u5c-+Pwo^zVE&Bvc#)^@I|71@=t3L2r=R}_j}8LzaZ&` zZ3O{;L4fQA>$@Fp8!hmku?Qf=J1hLSYpKcI+}(pV{Ndyg|4pnGti~LRbgtSKYP~^T z=!_f650;_fU2A{CSev*NUp`qLjiUAQ%WyBEBAebnZt{eAEHN@g@|GymA6F^`2o$#p z4E`XXC`NN z6ZgKfQiqDEP?BzuaJ_I_xSfzn9Y=d{uR7wPc+tjac9iGEM6p6I#{Og2(96Py4T+0C zt>+pU{8`Mt5O3jcjQmcKt{`Spy_vr}U8BH@IMpc5DwEsNV0`VTPukJiC;aNJJSYyZ z7i+CC#-KVZhKm~HLn!gc3$~y*8;_gxK>U@HK)*W1rrUYVGeV=@D&~@gVOOTo-O-@O zO1^{6;4deAax`6LRO6Ewy98Pm*F1LAi&@*SeoUr%WjLRTrhQtVQ_20+E)>kSTvWYQ z89}6_%pwtsLCo0eiacmvbzh|KW-6l8X)j{S_Q-{v%}GpkDe?FxL4)b(XED5e<;d3y z$|*vg))x*vLFeB0u`s_=lc!@d9}};zr;QV{t9va+ArQ%7OWKL>kxwz?5j&!<4?Kv69O=aS^ z4F`K!-!h<=v+)iSsejV!-|s=8%(oyw==nI~Y?Z}QtbYhBHWQf1&R57xi)q78?(#zLxMC#F=>M-P^9Ybj9=O)GAw1>QY)iCxh3}v~Q2O zbzC-uCMfvqC6U|j@{mVSRckI#` zQAQq#N7jXD)jZQtrg4+IMAan~c(|I`ll@91*Ib_Sr~J@C+x4vrK_JcF2?%VX$=LcH zLc$Lj38BR5uWZ@D*OyYs0oZDa<~=h{ck+O;p$xP>&FD0d%Q}2GxLm_u>#%~U>)_D&PcC62@=p2dT|0ybw^Ra}8SiOb zTLZdPY52_h$`|-T1Yt+~(nhzw$Rf#X=GCZJq+C=G4y;(9?E(3{IIBQ-?x1sj2@h`> z;u&i_x0`c|ziJ$k|3}$U4%^?WHrZK5lk6;2PIg_IEP3gYwDT*$lnTsq(S(ztW=8tx zo==#`wJL7pwOrXXn|NGjSzx8~86XzsX0JIf$!kjFw0nvPz-1t}4H}~oz z-$!$(h`*>`K)G?)!MWAuyM5FDwgXSaoIxN!_I4hs zX}TDa?%|&B7j}ny!&A@~Yf;v{Y%7f>;qVDx^bP))OHq?*`GuX5ks0JhwI)<0c9+q5 zF~;^L#?4*``20#fVUKElKNlO_iYwI-tvU(+KFp?DmcYvXCKE4{tVwej`3Ud7=0n8A zo9_j8@A7;})Z-fA&&pAE!r)C~7^~Ddo9M=W_WnfHN^UR0D;`;@BK3H~Izb8cAZs9g z5G3_w*rMAQ={vbYa*a)(=qUI)lJbK&evwI=%bQ(&*v&@44{Qiam#yoYPvuAFraj&; zPXRvDPI~#Zg=t%i#PJWUYdYl~(^U=R)z4q{CSz|!kV`t6<84r(+AR1KDP4(6kfHh6 z0hDP8-^Vtb@n;~cFDOh(XYsc?lb#lhv$8@y$ET-xF*7TzHQlefN6}bgh_KwB9HReF z0ERzFmT%*^QwBEli0CgIR^Th@^T{OSQJEa9npd46A|R#SnvI}sYZ98`a)8!>@gs4E zqO)qL(g`d<=bAb{swr1cs0|-etc5pLpCa{en9;}T5k3H<$z@mI!>pAh5l>s&NDZOvJ zx#GgMXr@yCp87mWxnl2i`g7g*8A+_|^Tp#5@}2Qyq=IUjM_ob5uJ{`m8YR^8R@fq> z3i__58>&@ubCPPW;&P_JXd@|M&xjNY3dnrA586mC0yOvyz6$d88*Q3z7wGh2m}5Bo z)b)*O<3(v!W8O&Fx-7vQ+ZJ!puQ8rm#Q-i~;?iqOhfDzZ<`-|P%l-L3Do?b3Fo|!XVw|*jm=!8pul&2Y;ZG=bYZy6WEMMH@hu#-1BEp%o){bPCIB18FB33m5v{ke=#GmZ8G`ZqKdyOmnpHgWm(AhQ{y$gEvT}_Vz%`GC1O#PFYWyFd>H!93E#E?Snd&XQmu)@ z5=Y9h$kvC}M`EWF+!j+Y>&O))JRP)2XL)a)jIQ#taIQUgrN3fi{P5nr2i!7};;JnH z!t@Rc8OR4xPmmP4EPj%{7cXwCi(&$S#S4jRL7uHPN%jt8q8rV|z*s~D0(F01J};BL zwRnc($_AzHV!mRO8I@Ke<8L7>v!7}+utag^J?#cl@+~tPe&uc+4}Hr_DR3rdkY~`` zi7=U43vp|EzInT5P*Gs{WIR}-T#h`1yP0inXY3!Z;+)D@Yj9U`2tu8phP@uGmP^|* z^rhA9X_Y;h-$3@2rKJP%5Vaw@!_E$bNk zCUrQOEyIYxqt~)DK9^Rz6-54o6V*wMNG@JDvGlRv7w>Z=^m6PO^!A77_R#OCo-NLl z7f@yAxYBIVFU3me90zT-!iX7dO!a270s(ZrWm&v#ZIG|t*Yz$8a&P$LwZ+3NBA-QK zuvgq3#QpX*s73wOi!P!ISqE|B_I(onWm*<0S-a2};z>WzxF zyKmfFPO^Kg#WQvY*=zwcyfYF8sL$*P-pNiOkeyGo&-O>hzeovm9oYFMqz_k|HO2bL6oc*}pXS$Q+(DpyLqye&O=iSFQo+_UJ9j zNkO+AghiU87bE`X?%+?}rDu^HNN(JEjG?OLWVd_rQU;b43qP7suuG#j&(vAjA*q$l zOZU@nK*6NvRU^nNkK}G|@{Kba472GLOdr*RT)J*+W7jkVQ{Aoyq^%Zhj;R!TQ)##+ z-_0d}9CPQGfN>d)OrtWCXkkka-!j3d}mG&<>Z+WrqT-!g9^~=^5!z?T$CP@GKLU52)XA z;H$&-j^94!fA85*e@KXC`Q3Wlt2;eQ&RnV=dev%abvp*o#> z+vJq3#JeHhKfp71#bPhHSmB@MFhTgr!jF&r9SKLop8K zF5g5ql%|!-y#SQLiJiKp5N#rx1{<^6ss%;dPglqU^uMxelTg}+(e3%@I8_rJo3|Oe z?0)!l2k4<-L&Zw$8j&&(o8B(4kH?3im+OH@-eSRm@G<`%R3TS~2+T1*D2DH;VRBRO zHHt}KYwwRBGt{S6)-j2f@aq0g$sT;j)n|BhpM;j-!0#(z=niM!@}j098<~c2zd=({ zM>9tV?gx(#b4}^xc@AgcDHgS~AnPKzN)PJ)Ni1;Ut>o!JooUGZRRN?V3sYDz^Pisi zYjO$%Mibj%?Yv+4a8hVZTYSq4lGQNHoC7ba7|se` z4G8jUT6^j>ZWnzeeoFs|(u9un`U6r7?AD=O!jrAnA!g3?Ida3Jfv=Y@U$`NR-45ZS z9^4mPjT&jtf;QtQZ?*h~eP&7Jrv$71(Lc4Vb`%*K0sy@bZ()TPQ^UNC zFwkqfd~x5%(M{%ROVNiFJ_c4seM6zc1E3o^UdQlQ!DXh>57Qvg?QHNBX?IhdxF2n@ z@mwZO1^Zx;pbOrEpckVhS?J`tB&PI!qJA$~l0-$rYF8<$B-q1Pe2*4u9ePq9e_dTM z38|n5NVnQQ?O*&*l-)0Pgqd(x`#LEznJ1jk@e6Cx98-)itRVxIJ#O>|qaE|5RkF{Ic^!9U75z3cra+ZQV9R@_(uKHJ zv&Vs(&(Ljnwf=t?dk=7|+xLH5_6RpZ2)Aq*Wn`31L?R7)COcI2CNqi1$`(wS&$e4XQZy?Wz{7PF6){V%*$RDh|w@5Ijw`i`s@SK>X5z(0LqtZCT zm37#kG*xirYX{3cb7X2xPvB{S#}86?op;fdvp~f4E?;k3bmi>g)3j-v^2+iO54Vl! zpeC`p`&oB&zC72|>svm?>#eeH?&Y+^8GcnZn$|C&EjI%E%E#iBbhc&1VRx16c1jX* ze8_sH@x6+?rYKEnH_GaF6N8$t-OF*U`khUk9Dh#Xd}ZMKW+0WfVOgE56(;AKw~5=A zy@tx9RZ42Hc)ay?8s}Hccvs4XVAhviSBE*VsioegdLJ?N8HXa73}CpG}ngBUoSgYu#h3f-bT%Qj1s<#}}<-$~R?I#NKv(ba9ZJgD{SWLJ{OJ1m|dnBh9AXClL`@trvtMVNZg(OO>d(6ug1k;xI4&$3>)-N%0S{gLzI;q>u>se!h& zZR_F40jHH)Z|4+F!(2ex*!==f&EhwWm z7RY0QSN>wpyiNPT^y!NCD=NZWl0(N8t|}BLO$%XP4cf~_WoZJ`)e3yPnkn~GYcASn zu@?4gX1bg}w8mVn7uS4Ao@;H&Kk`+apBS5$jpyQ{dSQBrlh(HsLj~lh@OKDBss=IN z85y>qXaqNvkKTx`v|(<~-`I18{Tlk(C-iNfJyiuoPJ0CEr#f$`yLE$)eHt2dcA-?h zF97OX_ zT(z2LW50!od89z4`1~FA0dsBx@9FdZLh5R$&9Uuq5uy*VK|j`PFNWs1W}ckMa@ z&3O8T%nt}LJ!R{N_mJn$ggw7=zwY2wvH7dOn7jWy zW$E9@zP(U&N9}l8mJB6kcVW1!=CY-Un%Cwj-(V*|_k>?+{CJLjPRLWBOsOBO>Vypl z=C{~rm63#29@ywT&smDUYQ!JReiL18iS4v|4ILBj!qxYf_qQ?_u!S9;dU&O5H2Y?#J*i;2L}FN_4TYh8wK z4K;?DHC4U28gw6~a z023-?P13rz?fe`Nz|Z&W-_jG&SH?>yjk2LT#`gJYaM4JnL4^MvQ7ARsh~bXCG0<3G zg_xtvb>f{doA|^a&VR%83RCaT6^2hP(h>bR>i!JX6$)QtFT1&HiGAo1_BblhHd=)e zsR_wNE6UI2I|Qi=G)zpr%#+L-zmB~f{i$rfeQ7FPv^I9iJ~X}OVZNN0sw`QU`!*=x zvoPpFMVD((2sc(m%!bKiMuM(wdNB3AxzFOn!ixz8I{1f`MI3ne248P*Wt#iEF3|cU z3syji0)=0g5P0$-{pDgd9g9deC!04UgQ7~8hvoMM^{&4`X{h7q`xEg~d}5SlIj{dU z)@|A7!e;kZaz)9>Y?psBG=?B&kf;#5t2F;WjXl|x^S7mcKuN^L zJA|s-Qo1^4=A3P|(!Bbs3koD(cDWDUpQsz1&ffeBI5Mk3A+NvWZ_s#&?unL)!P4$D_JvNr}fyP;OZ@QYzBD@^~9TnbtRx|>XB6%W8f(YS{FQX2@I^b8IAsOb+TcXfujdm*sY7RV=jTC+J&z?kEdf&bfluk-0ll=H7^T3w^+#4Ygwn4 zdwa=8u<@v5Fk?o~I&KLk)#iih${Sn!0sZY~#?OXr`}2+Dtf0FX?}pQR*A@8lE3e3YStHZ1ZxD?k0h1Xri@bI$Xq^dk?+4MVT@1|{#KxNofhnj^i- z$@Zuv|3zqXG$E_9T0AZ>$A!p|)QH!zQ$IAFRTo-|O%&tN}3awWOp_8W8U z!OgV);M1)~k@YaI&`$mWYCs;KwPFuiCQ?TEyykU`E5mG7VQ9w4?feeu`7sOxYz#+Tt4w$ zVkQeiPDp1lia91g6B!-l?7`Y>Aoe-Z0{<&u-7PYh`!{w*~MT{3yMB5 z$t-IMC3>7gQFe{RNZ#&={qtrAaYr29R%LnTMZ)P;1+V)yNJzy0ZhcyFCd?p#*F#{R z#)snP&e9KsD^AOTQ%Fj?Gd>Ek8ED2B&f@{pE1nUmtU3Sfu41h0w4n-}_g1z1*yRozlKw@Z;uLj!Rk>k|^`((_ z$yQ|~!jUuY_64ukQsA7*^Nl_DyZm~~^Hewb4Haq7-5RD=8zCZ^#JV-ear1PO#`n-l zcRk&abwh}$H_~M@eQECneFk^H+0(v3ztZ-pi1Ip5LshEvK&z)}N0Cn;R6#t4?$uAu zUtc1}@!;A-*wttHYseE)42+YLb7q}q{w5P$OvH0PBY)7uh?JW1dtrZ?o{Y&p9T%H( zyWiJvg`Q|v8{>yT10h$vm4;e@`e~JP#XKcq{8VKkgpxO+#VhHj=}3)kZOuqc)tI{k zNW~Ww35>K}9JCX$(ir_YU#&iSM;G$k@(sdRX4b4Rsb zMPTtrQ-CD%!oJY-5vq&l=iD=YVFYdqQMY6~^s# z?dwl>OhjMem_im4{(v-waV354BUMCxGCcL*07D4F$D9c?h@4bGCbJFC4Bl6&&K^@q zZKXRSObA2i2I&2$}uiA~RBGo^Q9~(Ww!BIMf>+5aa|yttt8xH+Q~n z*?=%vOddxy`7Nf|W6oroWdAu!0!vsR1eec-@tgG`0vygVHKvy9H~ivb)TAfG5J8XO zx8GD_-SR->hofHqc;`ax+kIzCs5GLH0miI&LJ|aH>qju1-+}+%* zQcGV;Qn-39E#zW%)nlYMu^XFWK6??_{q2?n`ZwkLyxEBysLEX?I~D!VH`Sn5cMA1X*k{#b&n-p|eH)&~W z>*Gpf(SKt(T3taE*89n5&2{z}<%QFY?Ck8f;qDW$MbM!#hVOnYw;lF>UkQnz!zNcX zxqT}!RmHlPFb7uRVn|&uv~u6+8pywC#)jzCWBweIvfLc%tA#~4jh=HU4(!e4cOc-J z*tDgDzFp(IHj#RUep3iEuigk*eLtIP1()v{awds|pQnMeGzW_N4&~oF1GWg#5KbA^ zHUH;Gga)>;%8l7}kcOkJNAymjs-A!GmFlHX^W6_1rZb1zUjf^_IXeC7k~4?0pnY&> zB2?)v2Ux7F`JTXMz#tT{Iu9bN^s+!5`ro~Gk0AdbratrLtxr9f@9N@oTb!14KU}6< z0+q&9nQ`~yL;K6A$Z0ArInm(YOOrG5W4ib#UfPeK41e()EULHnZG4jl54zH?N3P#t z7ZMUu)n=jC4V)SWX|aX8T3Q=W@E}GF6ue;~COeOhI*aq4{qf%)5H+BpT(wW)>$*u| z+(K=atCz&C|qH%0m;e5o5jdPc@AQ`5Kig5?UD6<|YH)cXmr%HvN}l8C-U(g(mc3aonRxU;z-h8%w#(*UL-x@`j^{p&?R)D!E03RX zvC0p$6;d~;a$y|#uQ3id%nYXY?ck2hZSdN2aH-~1vm+5oHyF)n2yDEYed$1Ja|<>T zvBlcif(0jD2CX-3|6QxUPdC!=GRQaarUB(FgEj{#7QkJya_>o-o14*>-{YoO zq>mbcDFiLLS>WPe@x2i+^|xVBOPs42q}0OtlN74ZhxLCy#y(&lHTY!I!u)?ezw9lw zNIo9pIUwwE?_Tuxui)yHHTVb!U}moSifM2#Gd~3)sJy-yQnZ7FyL&?EjXK!AuGL#2 zL%+QD|BM@(V>RBjEXz0*alV!I(2guj%|@%%D~7pb-XRE=djDhA|@y4W|&82v>i#ypx zTLLTJpRIB?YTaL4w2qPk=7Q+xEp3?u5a$dlB2jqqhlf5xKhy~x`tDKa7jLO=h_7vJ zQ6Ilzk)s#kgN&+<3e!F!)};(ZNH{y%H<*AIvd^m|vH^>zIg z<{*f`PK&4mYAOqeLINt?p2SVukxsm@()>ULEe!?N&;v>vW#L&@H~EL~|NX4&Uvmgb z#ve!?6%%2n#YpAHyNYS4k(%7@n_GG}Zb>37E~1a-PyNRzFfea|@fqo(1N+221j@WWQj z9raIPX#ITdAr|1di(tftK6+H8PJppU17Jwoy-Ah{+q@sb)-E6*+wA*wfgLo_h%E}x zh^@6Ir1TF@7DGRo0R%`EWa*dS$xQj6@&fWb1*!FIuZ7-#=qz3LFmo3Y1Ym&=MhSQx zI^QS9#wwzDyU)u%Gq)&k2W%PbM*R?^>;Gd@e%mPrw4D+MlFaXcVncYCUDD%oU2nSk zT7gNTgbPdd3s`=lXO@uK*bccTaGl-)Aa%f75O;o!`krH@ z7}5xSNOSRSF^C8M@9C66>H|P_+5#xzOtWszjxQ2Z2ZJ;RVut|2Qp>()!2D;XxIq=n zCY|`B&dX_Q;xSMwq%Xt(ldr3l@s0F_ALl5DWtji@Sojv?3-ps6E@N+r&~BM?;fsXV z!R|3Y=knsoW?Zg1`Fb{8Im# zUyv!*54bu+fZ5J+@!^o*P_0+(y$W9(A|(wz7W>APhW*3FNfY=okq!-1Xeo8uwXho4Po%p1c7H zN7*Zmix-RYfOH4suC!#eKgc1onv_)2=>NTVYiJ;q-4Ba5xvCH^UFV63h6xbR(s9+x zh!LSa1(;LzLL8Uzi!IAaC08vK6I5qqLz=olVq6oY0(R}6lZzbvwPqr)W_0`JeE1?y zF+h{&WS;8`kM^BPSI#?G;Yx(N0@^qZHsfIR%z@P*p!Bjuw;x){Rx!jMy&2}@i~lcb z_-j`n_0eu@>nc3>h%jXY&0+w&s6G2z3&fOjz3(XQ=IM`wQ<<~8h9@%vY-?mbC^EvKjsDpV7na`u2B7Ko4VJ3{ zfg=O7*ImTC=AWL+a*_VaxFKfIW)T+y z9(}@=vWGNe@d((~+pdGe30IJ2<8-_GYh{Baqm=aW*)(aP6A^>^AL zRnAymZ8n+I@hBoHD#M85R+?Y&xT>OJXLs6GUG7+g`Lw?)8nXT4;~k0bXN8#0;el}L zrL-78)?xCm&CRL67S+`p$u5Kok!A*f0*f^}68Hi$T#z$OjT11Vwd;L%J8-Dj;=W?Y z7uY1`i$c9L|C)6SY+eLFeOiKiwI&T%6l6Cx_-s`g8k&F~o1m8rG%$vWL_*}szG4EJ z6^~hS^s2slPp-G}Wr8oF=}zOplolUyJ-ziSLjB)^=;QSZMtP?kucXwG0TkWCzHB>s z1Fgb+X1LhmwqK*k9NwQ& z1)uN^cw_>K&CzJ#?Wm02za2Z=*M&14f`{!eStoTGo_rM(w&7b~#-t@&?8zi$Wo4!5 zt`CLhw%-8$VOik6;vY9MKrP-Drv)X_|DRb#^sYsL9Sp+$MbiwP@3eeodOAq|5))gP z${F8PGrNqYPy`-C%UHrQ97w&|SKM0a$|2F9XWrF)JB$5a+XDlCiWQ7qBVdBjPd;bB zZ?B>e|8)(%*^cCX!uQ%GTn_Wy@*(jqh*sZ9_Gn<`PKC_K>)_6aLXZUTr4jxU0uDlN zih4q6@$-{J{=QPW6tTefvNTwva6dde9GUZ8`4vZMMjZN)Vo#KQo91x&mZ32_ib#OI zD!9)k{%PmHDAmcJO+JMBIwB~$3y2^rOvl}$7Y@>ifuZbPoAA48ahRoNc@)&SmT+adfx?PTR|5LB*iM2+7=bB;-dx!2hrFT9ZRm> z%~JDm-JDBpO!S1gniu!m|2LTM_C{wk^sJFtZj3A;!&$0HB{o#aNMW>G@Ad|C!ANv1}6(Ctww5Cj3k0ntAPm{DF)LPAbxbFTrj!%xC+{4op(W~ z!r~7D=UZExK$CZ|(dU3y5fR3UPqbmw|BsQCPc?QlBuVDy=L0D8D(zE&-kP@iJWa5$ zbfOk7Wc7gg(7eu4$mw$%RuE_qV2~I7(>j7J@Rt4MI8I|b1inza1EANt4)Pxs+lzw% zuP>W-dIP_Wkz$BF3j@v;^yoJWW4$;hn)Y*A7z-_i47918-3j(EAJ=PqD*y0wSS^?j zXP6J0aWwOD9S&wiMTL5&xaVPcKm)JA=o&|sfIU@K-H>Qp0V@l4IT} z;4XT>=*qsn`_h@JH|~+PHZ3|qTpn_U^SrRX%8eVn-3fLu4FombVt<+jbM!Qn!!$g^ zfFxIt1}6GWBs}!i&6_U*Qr!2}dyC9ed2F_%tBFv+aGD8>wlw)g{$e`suW-VE)ooLF z|1{up=mF~^dY{sXKvc4J({OpXqS)iW5i0<0+(ba5keOl;KlzHUSdo5_DKW&vE#5T{ zNj6CIC6a>-KnA-S>Hm72UpNy#38FJ<0esXaQU={qm}^Rg=v*!EWy)^E3t3%*6xro; zEqb36Pan8rEMN{7vuKa~{IKWuGhl;jzfOeQ^iq15+aJ8KlrwrBc3~cDkv}aFLDZON zUJ>xaJ?OB2ZG*A9xAzW@O+UarqA3u!<~~pyBLygt0U_*Mr1h166dO9%;j@3(8ue^2 z`~C=D*Ma*4INnmah*%*js-@8y9V;u~t;^VY0)cL^U=;~6>qJ>XTs#pD&c3a$h|=}z zsgPfy?Oud^QT9}9nc!dW2Lp2kZO$j~rAHD@`KwDIqX|Cfl)dv*5hmqee>Y;TPc59Mzh{|Kx?wp^fo5 zrmT2@6g5Kp1}+~!**(BmTs#Arc16I&_ddi%o9@6oSBvMnY?x}7+ok?&_+^Yca$zQU z*US0-Fk(M+c07a$dutVsLJ7l1lXVrcvScScfglRe1%e_UR11$FGc?N^E9oWiz{KS2 z&B7=#o$McEK>aRFzk~g+Fb3n)lV6tDFU6+aTZR*fu)-eTO3dlm^6|H)>P-a7c(>)3 zV1(!Qry<*-9RNOG9eAsY%cEeE<)nwY{lNx%SNyU$pvU86m@0?}qQqPjPCUxYRs&|B zc~pko&w8%rOUZx-of?7X%>s^0z-cs-Dx_%Xx<8=zfiM5^sXy$4m*|{-Bc-pa3GP`X z93OMGeB(w!&~r{qB>zGPM? zehyq?uWZkq;ZT!O4%vjI{C|Ai)P6l*MM=B1cx4-|3hGFp7mI+n9b6yc+9jm zQ2LM#d_sp^j$XV5Ef<*-AUuq4T`kX}Ee)WaOL3G2lsrX;4~SV?JDp^4KQZKvn;P#Q z_Iv{cI$|3w)lp*bugH9Yf``4r@~9W(IvWAc86Q8K=auFAZ{%!{JoH=)IsOANQv>B}Uuyc1J=KILhe# zl@e3)pS(yzNgoJG)IpX9GxsaRah%M|pTldxc-~2s!I6D^8&KoZg*ce!o@B3Mi}EbU z-FAnDGj&bjV>?9pq>22&%KxeyQCnZZ*`n5W_UZ2KK0Tse>u7*@;fud8wpdX@@UBj_)N*b`NCUh|OTz zD}hMxrF2nIh;DXA<0L%9mVT6}R$XLblCJdRgupG@bo&&xLs4+r)jy5v1={;qRbg)d zTidYQq;2=y)yJ^I5%VQwlsJF_=c7^(x00c3a;7llL zdD3KnTrJCOXPJwZ5{~-QmA63m-}E}cP_A3;%Z2=K?L!p2LYa*|a19XmwaR#*3{$_Rka49pj4O^oNs zkrSl8Vp6?9B1lbL9T3rT*a-q0X{Y}*Y4 zNy>2jm8AJxr)qID3p;y{)0jtQ;})KlmR9#j6Bv$?DaERTPMb46;EqydOS+dxp_F$( zpz(+J!+Qj6nO*C6r4u~_(P}nyc(;2--x5-`nE(Bn9Z5G(BeR@KX^;*_D4eBOTM1W_{X6}TtRZ_Tt42%SR zlGMhlAH>w4vR#ju{R(*(bI6PbY+imigf4BKw&*$o`BG2YBhY;)c38M4D=qH2i6Ba( zJZ#j7ynE;Zqau)3o%ix}H8o`eE>arHTOj|2>P&sg#4`kt;`2%YIs_54 zAWyvNqJH6n^AnLN{*QNC$X8?se@682pH2m0P1-I2CHBvC9sW_F)2&vPcE6=(DMEf=VyR@>-QJ70pxtXa*_(vmA~?TtlDO498QB8gmD-JNW5zk>y6bqrPr zQ~$X*={QWgb2<`ZhT^{-kK+hTM+xjJ*wVdsu{MIU7WnV{LaW9}-g+CLL6)0}BRz89wCGRWMCjE+ zL`-7|<@onJnY4kR?{N?!}&5y1kla7M_U2u*Z6?=CO;nhIZt` zqy3y-ueyt$+ca-5O1O)xv18(k_+y03KgA_p=XhU~@(~DKA(0_`rwn4mJFuf=wFMQB zS@C0TR~^6S=$LVogGv-I{}9V+UnI-M*-C&?`JG_$Pj|>%es7&Ra?IcfmHhgq6A!v? z5%CjjOk{QL=UJ@FAw7l8ePCibeq#sYAx1@oN>bQu2;P{>h>`5ODRZM=ZM@D0yE+(E zd$>;2mL&c|Z4#H5Rt;2T;54J_Dsc=6{0|60kW=wOHJZCRQ5{A>;{8ME#rS7y83+NzP?8!bZ5sTxY9#Blet56sN0EUsU_u8!yw}K2I>9ho z79?+0EmtasY=-GH-<+Z`C3{4}&rQdFCJS)*0>JQCw%ZHPv1S z=@%2Qfg1~;TK~(|Hi(_dX7s+FXqG+%b#HZ(TYtecZ<@cqmlnSFbD2xaQUFD@G$ZWI zMDGLB+A#6$*bk3^%KshU-J5xyU$>s7Fz{D;^O{Ja7FG2a(VV%EHtAyD6AwsPrY7V6 z>lA>WfNz_`&v~cfpJKyx^UPmGM??No*|Q*fs7DWh;D%zUteT|jCI^pR#n)fRRllzodsIK2&l`*_~xC*lxW(~ga{t&UvPI%&-_5aee@~Y z3mb_PwEF(dN(&BRMD;-1bex{8bAG_Nepv6R`fSG zv1#xkb&mWY1&LRw>r8(|s8UxE`TM6t7-Fxbg~Iy;brSJ2{jtP-;fj?2TG!Z+K#e(k(jU=;vD z!#wSDcMCJtiTJ9e+}p(=c@Q80t&B^wA4x-I_xXX?g*(p{f}MluQ%xNSqOvR18zdgU zTpeIF{9_oD|6Eb;GQakmulr0Yc>~V(d8d=@Z*xHJng)r_PWZ2qKOrUm@aGr!A%~JD zhXM{OqvC1jYuTj}saa`-ZLAfFc&ga39jw%V9D1eLs-O3B+dvz_){#yTQswxpQ;*1f zj*N|5Cn0I%a9`{pb;t;wpHprOvWI?W5hkpcoPr$=SleP1VvGvTz~9 ze?7NfPnCL4i(}``ZtvP8$CD0*_dll8E&FofGr9Sfc3(1YZZw@lHqpE8q|&CfkJhsl zjMYbI)!y=rH5lIf`i5baYc!Ww=%}dwB#f#fR{F$cl?lalniL_0e2uSpgmF&AI^m86 zKI$eabnXMu424{*s@&{=e9oJv-hIFP(4$kZi>Vuwq>q3?GL4M+S|?-(GVVHrF^H$j z??Wih%~M)tJ9bm@c$c|=aT1dFEun0T?yJ(Wjf3Qg!YNZ75b|H|s)1g`?lf_QtKW*k zw;;CZ1UW+0b7odM5Sx8h`vUtk7`pC4+&jc~)cSMv-pTkAs&k9D?_OM+@dMiNO9dHR zIxSuxxzUXpQB%Zti6wKxaN#jk4XiHLLW?h-PUU({1+n$Jj_D$2C>o3IPlWJgMIbJ1 zeu_^S1zDa69726>LuHR`RvV}C?IEk+gEeT(A!$hy9&oK9LM?|7e~PB&*@K*U&+S88 z0$a~kB4e&qOdVlHmJ+w@K^N!m%F_PQV;K|Kk)Y!$ zND3>5I8)_mJ>NN_spvNDgJ+(RKVSt_sAAKdR~;Hd56ecbL33M_jozs10?&j$ma0$A zM6&z(b>nLZW*`xVo9$}rD;Ba^y<0~_w8+&&@bk4*ZYm^BGmI$=0VE)MY9&D)oyiOjHIKU-L#le-|iEwyO(V5I!Ph4-vf z9<9MR=^wt`7~-Oe9<0BjcSuFWq2`_BKC5ThU7C~1W7V)TmBY@ESv28Pbf6-9t4YG+ zT_IvzOHGoAD$|kRrTc4+`|}^P)++<`Jwk`@HO#p3g$|UR##nQPwDN1!@s~>)ZJ&@8 zq_my)daKEiEU-qsufqN6NkUH1%2{h&4vhyc`?@@%tFDtT1E<}MHf@WtVm1A?&p1mP zE;j_P-vQ3#xf0vww$bx>F2(&(_KMbq?nSX?nJC(5_uGk}#5czUPN-KQv_f?SY{CYh>eMsf^lee>n)nupecNv47}6D~fhRb!8Mr z?ZH9h#<68Rziz>*Uc2^%uDejz>3Y&<=&HO`DO0gG6Q6N~vb|2`fH~~ydH)3h3bv_= zYsq!KTX!||SPiC4KSlF=!Xkb$^X=t%(U9t4DzE|f)JAo=UrAoBKadfvdg`*VMnp9Ibo9iV_Yu#l z?DZOt!l=U|J+a&E?{06-tqNC{Zd{^j-dTyUv3wCk{7f?8Qlt`LjODUXo~EEDYRJlD zePK)`<-I;t>qs7vFm(Fp`j(I0#Jneo@ni72`A1=>P%VoepkBaYh|f@5V+iu`jza?w zIVHSicDZh^M14Hzua>FOcZq!~&l?>U%cz-MK%E@#)OPziKXWcg7)jnGTFmtqUNGq= zawo3R8Mlkf_dGaQskQi>!ANfW21?7CcT{iQWOLl8_Yqz*F_=ad9HCCISDmC4ze(4$?XanR5+7tu`)RQ#pMSO+7^0@9#$9q zpe-$&RyEav&|v;q21I?2DKwVYa ziX$E}Slz`H2yXsWVZ()!#gn9wIXgpPVMvL*yLx#+(kX@yLZkBw6$!BP&L88SsH+m3 zf*=XRe$=ziRGCh!1m+oTt(;@<_SnL0cR62o+?HIu&tcFQ?9Q) zTjX%aU0}3_#TPv0K|1Cj(RBQ+m2k&;MP;vVrKYU~@}Jxh98Dy@7QO3>W;y!|ML}oC z_tH}v1kASI*q=kc`>7%VP4VQQS0s?pMY)JJO#?%XWy^GI zD&+SK%*JMXxRG}B14@ZUxf?TSu0)z(I;%byS#1L(jL`SiSTgcS8X2+E5nX+o-NG(N zw6GZP<%krU$p4_hYSEo1WD*+SBB}QGw*(xOHJo_FVmemiU|+EQ0v{ z52>k6!#61#d^dkk@Np|{e9<9IcZ>qF-pb8?KxmzDTs5GY5uzD8-6kY@xHA)P9;nGt zYdd$ba=1fAG-QJNy#~b-41Iij!ThS&j=)-8o&|-&-P=j6rtUjD9-Q0G^B1Vp}TyRxWUb`0|DScF|iDA+&QjMq5 z$aVK!T>fr9oR#oX%OveIiL!x-4y)*mJ`!2H|xaFvFasKc!w(T|A_bU!NZvN-9iZmra0JC{6 z&Mb+k)Nq#g9^?af8f`}pqj)++!#pYOux9mV-I66-1L5P^_t#4R?7zNB+4&K`@R{~ zBOaR^$xMzQqSm!JRZ!P1u_C(ad09AE+$OiWr1p9C$xM*&cL~qw2b?obl_*PBP;n^1 zac!S$^EJ>5Q$=)AK+vS=WqS_qCLpr$58)e_r@&m$V$V>-s*c;rnxJjQ>UK4w%>||` zJjIJ)tJG$`Vmg=0DCuGgwuGuEZk@OC-o0x2qsT1m%!mFQoyBZU*)DiRPCN>uVaK+fphWK_(^to?|{9_xB?64w)0E`jRhByR73~$SMk< zEv;5P<7rt-eb2@9lP;wM$Vcvj9B76WE!FT{HD9rfajUK9^@KoSiwD9Y=b!+v&TUc` zSSd$QaAKIsaH&{=$W;2do?O8X7D{I?KS(4y51*y+txWS%WhWPL-OWGauZ1|V|0Kh? z8F0HJYC?srbjZ|TntNuue2+?rthTB8ojKB{aa3cuO2cNo`Me`3`0HGSj;m=-$On5Z z8{fA2yCi>wiuf=lQ-XU zP*ti1<4}%TI}Ts5RnC4=bAd58fxLGiD0Fbp_A1}2)h{JcR=w#@54P3cQ1N7hoXcSn z!8pZ;{lODs`xa%wI$T()Boz^f3WU7T7vY7A$&kqEh`}a!OYs*j@AF?xU3TGC7@8*1 zi6~NRC4CRk4*%}xw(gs{+kc8|5!((8haxH>qFWEvKVRO6$h#qfaO3c8`jQ z)Yr05yh<-`k6u_NI?w+!`D9>BjE`X$)W|o-pOWr3FP+%u!}bc?c$)?BSS_1p)tEoE z2=ubDQH#ALOfJvR^R$gZOSK!Ua9-?(;!F>SHnQ*>Y`Gw_7A#`DAm8f#)+hR_ZPV%) ztw$DngRJTSwUk=_?fQZq{Cr9QqAw5pTEkvi+*Wxvs=bEWGY}JS{Z;~z$y$k#5@Cit zio4OFrldHNu5j*(deAyITeg}q_p73dV`<77m0$hJ5;R4U#B9!$tn`H}Gj2yz5Rlw|-*a!=1TX5_PVnDvF$GE^puRi|*<5qw-d9Ub`3{>BI z?HfsckC&g-$mEP)qiy{qVgcV!$R$aaU(ysrpYXXrp&1HST9FrLej{{o0zl~DFN#w< zb$4->=GG^h&-3X>D4fq9d(4~cQL=Ffue8(ir$2R8pfmA^!MN)>Zo;B~JM^YDJR;%F zDhgYL(C6TS%$2<1oMGRMX^3td0pgzDy_mEAkb@ zBx?8}&)xo=GG4xRw>k6e-7uaajgZ<#WzYEQ)+OyWiX!>SWR4d;+sJA=t?=^H)r{C@ zYjPL}-WTJc4$)|-IX#=hbzHiEo2r*vac`e;$FU~Qp6bZJYfhT;?1Em!jonqO*DTwo zX^`7@=RGI=J?=AT{U|1a>yz>;Q*viI zTnieW^<;gZVZ~75*627b$p*psZ|s=66G&yv-#JDHYREB8Uhei48;csrI!AVVZO}ki z$NrjFek+Be8UOt+r}yld=^VGWi@M@I-n6c1i(Y{u9Rod8-WOqj(-McPpS`KEJ{~Mz zah(hm&Wc>^XrFK`XF4mLn6u1Beb4b6FQqYotsIJH5s_zCRJ#HBljz*qW7jwp0@TO- zXBz@Kj+9T}H))@hDXW=LU#@mB0d$|SvaiT9e3sS{3mcC*8EuN%k zfx0H110QA6iEudMLutIWmzbIK>SMhD{fUYk>dSG(;JRDmu;|2?ZrkHT;W-nZ*Eo;8 zd>RcM#2KHU6ICm&$2jwOBV$|3vwhhEqU@^ObdrZzS~a8<4>2B+eV?q>dRIs*$v%9w zfbOh5i=psE>qoZb#|Mn3>7Ku!5%W7nc+m||yR4ils~FP`DT>+;nef~X)!XvjzY@vT zN-SAIWRK@6Ev{Am)JQU?+Wb+;z~a&V0b}R%;ZC@)_t9pFMq@zM!}s#*Oqp)yKBe$K7XX0&@BxjSXB7~##D)UA3L7fFP_a<1hC}y zhILoquJPq?WfO+&o^Q(<-hLSHE$VFa_+gbTWV(6 z_NA{RnX%95OBotPQd2}U;yM*T*UOKwH;Z4MQ4epXmae*ebGw)Y8}wyeCI!JFd2RMn z+e1=HN?ZNVL;>N6yHhoHz7v#Lj=iA0{-e9+UHO2Q_ee&5?Uoc)|6Yu$#M{PP|8+P0 zbLWqI@LX$JcG@GcGU~j32t3XD;dRU`Og*h~Ez<8Ge&_As`@&{beS5S0G?{tm*h}Dn%IFoeA&kBD_FEAyS{ptNoGDCwwr&3$??8fi?)ikClKlus_zr}|a zeATYh6*cyX%WSaa4w>TaJRo!*+rbr=)x|wH@LUWfvsdSS#RzFeYnOJqhCiusoR2?RBePW+n^?SG9jW=MMo07G0#=o&5jBs; zN_FJ%#<=XnN9yH~yVYF|fp&Rmq^kNJQNsg~)Pw`Im1T9cU1eo9tkm0M$(Qx`@A$f2 zU%T?<;X2i;l=fMdPl>|3M@LPJ85kYDPM@iG)^)`k2h?5*yA&=?8lNg3zJKXti(pJS z)qnuS(>T1;)sK~2HT@2oTuKUESUB4$i@uL8XtU-@$;HAF^EVd&+KW2V9l#&VPdDw* zXc z=6&|-_{>4!1q4;2&a*(>ZQ^!gA)QC0beF#w2${bf_-eH}{t4a3xjz-DdN`v&Gf*o+ zG{ij@cJ`7C>%pP>)JugR51$3S{8gM!_^1Sfl0ChGN3S2B)Xy&)1+TKV-iQyFfze=2UhvQ})-V-77V+N^|xF^I)|{~aCglw6}vKC%Q? zUE2DJZD((`6>^1J7KR_sMTuv4x7PC=OdtPz)3q6*>AcON=*IOVQnS-@ez_{W?#h0d zRP!mmm@_TCh>7Zb=Ca7_`tH?8E zSxr;K^i!<=eGfy4Z}St^Ld^%L4Z4+`MmTfehO>Gpd4HfVgG=nakkI}WI{sI@)LwYi zIHT@cYZ5i@@&^tNBPC~AtpRKJH~6l7KG_iyTIiIvP7sUsJYze#=KDq2z>`(4d2Xfc z-jQVfE$=`aEDs4)NMDNIlGe^bLtv7Uq%}`^$O|%_`c%faLGX?JFq9VlAgkXq#D{go zmwR&h?J<|F1k;NojhBWJIaIWFMvz$-$iJvwz;^ifrcixWg@4RtmR_Y!==+S!BHXpL zZ=!b!e_ZhxiU{U|A|7x}25tfiyB=-zm)NV~aZeerbC+w{KUi2Xp7`mtHqqD+|ERDk zrbF^%I8SwFHp%UUrpN-{Wt8jT|3}+fMpe~@>%vF~h_ryxC5V(L-5t{1Eg?v|=n_F1 zL=;dOq`MmwL8NQZ-Le45MKAcCh3~u1-e;UM&NogBhw(#)bI$vD?!2z+-p8SUcr_;a zkWZu<7nD;52tHuU$U?hdwYFpzjw>5+I=gz&{9Kr1ht%);}#42`FofqTFY%d(FI zR9PVV@=)2Wpn)9cmEt3TaFPBx*8@wp^cHaV@qz=r>sc6#Qk&tu(l6t-cLi&PwOIEr zF2V?23*<+OqQR^BNq6wwcVz+4IwLkDqp!7R3gb>2{dR9aeXHYJ$NDt)$Pm|n@x^od zqpr7CZfH9LzcLbo?3TFQ)W#jO@S-JMB`o6m`K6|FYA}&-MxN!Y@0#u|gm&&K-Kw>|> zJ;6!{JTJM4%+x|js^}NEl2>N`-g7$SJulP-C5!7ki%GIItHrVD)wF$?`m+hClF|RQe$m+Ukze9uJk zBOo>b2glHFkn^9-coLZ)uGbE39SNcJkid7Pf~ z=uEQtiV2-FY-*-#rV>$@AFijS3^=OksU4G8M`El1FCJnSiJ@DYzy4%V^mjIpC_=kZ1LB@@mP64cIk)P8``v)gj5(B-NnDR*QFWi4!~JC8at)sksJU zSM`n5>!b4S+Ho}x;@e}}^xo=i86v|x%DcPou|irQ95a?P$zKfFH5FPr3Ez{B2()84p? zuPVRd6G5fy^_jMl4>V1lq^mKYA7X<uF z$DzrGcjUeRIs=Tg&lj4#o*_|7X_o1muet|X%jeJxLn_UBGXdmdDL6-u0J zOq=lrd)o74TdZZauEMqij3JWLEtKa07Wid=6O_sj=XwNIg8zxMTU8iE?t|lYLW^v3 z1-x|9GoG)mnAIPj+8fUE*s$P3=69Qp-No5(;?1TLxnb=6i{BjUuImph%HDIzHA`SG z*S+{QI;bXBkp!@&RYeJ6!JdeMsddVN zE>X&(B90qp_xzhAH_iy9-lSELORK0lNIOY!){J5ippclhs5n03vOKIe9!}iYN-Ei$ zDoiV>6=*))XV%i)j$dYRPNk9GDeQkHdwEK%U8?>=Q>Q_Ht4UQ%W}7*UPtV~o?_=wP z3-_HYNP9ndQvr)%Mb>Q8rC?Vj!`@Q`D%uIW5C8u)kT<>VT2}u7^ z?Z67i5$2$nsy1eskNR>tmG_bEYTo-!bZ4^7_WIZG$%P-9X2ZS*8|lMOIdRssJ0!gZ z{p&P@D%4Dden>XtJ#EQpVPHGd4QtW%wA4NhPcvA(;beotKRQ!#AQOTgB~HyIv1iiFVCuE=6IrPDG54Ku zWZv8T7t95)Y=dt{R3zryqTd6hJ#2LP{34ByHRnK0(RSjG@ne1egLYJ$Ew;i5l7wIt z(8x-ik!t4R;3R;()xLu6qZH@GtTwilxj{s`%BP z>{Yo%=FafrdQwpE{d)#d$usY(E=RMTu#f}SWp5`?&9jWcgxSn?iJijt*YjSzG?l4O zuvDM>^9?RY%ZBiX5ar>;F6a5{rxaO1ap`t`fE zW$m)|cuG7hwVz|@0GW+2PnmrbN|%oMTQ|T!@%uNyGrQ2mk7b?MWe#C`BN@b>_7A*= z4hiq^KnrTL;gE;$zA?jqjFFR$R8>}jmSIz{si7bE505z))LuQ>(6WbM!Op4O=J$1$ z81x+)i|x7`O(zkGjA+2eDDc(x+|(z%T{K-f*+b-pSf|sz{RwxLa$`Va1+6^WoWI@` zfD-H7D0O)Hm?_8U)Qzb6n6v5+U*$Lcg&L^k29bS2GHh6MC|ltjY0;{pUsP3|_VMFt(W71fI;*4f8sammwvC&u z5)YtrQX~J}ig&(+D)JM}8OavJ|8^hEn(!N1xjy4cHYF0k_qOn?q{Fw}H@Cuo9*l9P ztH$ea4zagB%Nji5E|)7ES71TeW$hCeDN3sP(T622u}5{uno`-cIhx*SOnTW$X(~X zxGvh}ogF_zOFG^~#Z3p^$Nv_=4W!JFVAd3QB+7&NY1lhKb5TOvmS907#7_M3U)at# z@{o+Qz>hu2#r)s7SDLo@bA#mbFP!t1$zO9gBtLpzFZ+;oft(CxYSz{Wlr+XSc5Rls^qgMZ}6jU6=MvoSj^8V&s{0@yQdQ8v;Pf!^(f5hZCNFT6Grr$Oe&Zx-B!h#;KoD4WdY{Qu-7jI3Xhduf$tUzmlyj?MRQWni-easfkZ-;M) zDnNGaZ}aIkV9V|VRZD7@;-Rh}pxDdOwN~A>@rOGI2f5BZ30LV~WkIgDfSmB2~!& zg*s5VGravTuwCiqvUQKeCNDop4@)kUmAbLiW2B`37U7F(nBIoe=Bjem3|+G(4+xkwLAVcYEPYD(;F7U+TtZ6M!+Cq zP?%`0gtxghW39FlbOZ_FY*}Lf`^XTj{bQdX;iW-!2)8ky$sVKiT8KA8HnfR41aansb>?oLP-)0hL4CJDw0FX8wXiL<1 zALcd!4MQeszIoJ(gNyYtl+%U`H6X}3AE|?)Ij_x8^|iq2g#2s+Hy?y`V|gw5_bUhcMmQT#8d-C`d$Fi34m8%#A~flk{a(+dWfBTY>E_w16d1 z(&kKpbxOY|Es?rKkzXTdSPTRw7IcY4(lo_hf%v`bqGUna=Ej!7YOfPOV&lQNecWkS zzID=X==YwQ{1zNhdvVKiCM)E%*`f9FS!7vA%83tfk~#!pwLEBKFr-d97rA2I*! ziH(1>Re}tiC#z@eE(Q7v?*=uF%fyI+_ZXfdc3uOfxc@2pSz+5ol4|g{GWg}EvT@E& z)8?Czjcl+nF?e$rYq%=Glh{B_kdsO-jKexZE@C1^FqFz$$Hve>aHQYq8x#lHU1xws zkrSCHZV>n3gCr2II!+oZUEquTBw_Ij9eOh32qveaQ_m6aC>%3@7+LUanrG|#ws_bs z3jb|BB7kOjM7+YuZ|m4Fc4}ECbxFG&4A6bpEsJe;z~P>g-Pa8Y^fk!pU#Kx7NcL4H9TnaOZJ zL{eH$15E;8Z3g* z)m{!_jUZ@eafzIVrC-F=Wh+2XOF-exQ*CP zBPR^2vzC3#DKmWEp?43%&1?mP%FcZTYC3i;oc%)1q@A0E*K}4m({5-g3_nV-QPCl) zrpjO60NI5s2{m1It#Md#2u=tXX%@UV({J{4Dbxe?cXc&D<-c=FqqhqJzz<@ojjw0K z(5b=J3;M{mX=J3g7v%c+k|M%FRr;GWjf@uUMz9XH-!-9zyhkyctxH!RY z$>4830Ff#YC;EOsrbjDzz*>BKfpv0iCfSSSPe9xz_`&xq@a<`ha3X<+m7FAO{dDdF zcxQVV;6kT;z99LJ@2_?9{r_CmNK>M>GZQEW3DzZ)jDaEs2osCMzAr2un>tVOE4rxE z(ItL?1zU>EluuYxRucpC0zi~u8Ek)^0eLx5nkq*|>5gc{FmPJMi3&XTpJC50yDtXz zwu<0Lh?F}XCb0?$CHr6Q%X#6om&-|;5O+mW$k}jRH6LKF29pPbbY}U*+n3_5ehAwD zY%-UQ+*y=EY|SUrCRpplwH|L?lBfj?Hi-8z2mOfnU)VrdO}7MYRbo=C$F*eG@6jCHGd{qw5C8xSJ6sEAS^*ZKOUdyU z`p$05LfS&qRF%(oKp~b8?iT-ZDkc(pKBxt*B|N)S9n#NY4EHDiu^-(qPb* za8(JCGF$#K{~_9ZigJFAWsZC|v8vv5XEY}ZpUs@ab(Wu{3w6RI5|VnzYxxJ8l{)YT zOm@d&EZq}j{5|?>6vtcU2gwR9)fcou4KOk1`d7F>Z^ zjgR@HYH?kKvATZwe%W7|isb}4Kug-18W~}rwqi+FWt2{La+m>Pk%D3Et4zTDFMM&< z-8wdv=MD19wLHNB1G1pZY6OpB=7|Q%{bh42Ah`=5Ke|uYEZb}LopHX_AECcWB z>w8Zz1jd|zP?fUe4ZO|WPvVTPKtOgl2`0h+X6}*0CJtw&4t%?U?f8BFAi?mjacKTG zj6?SU)CqZh=b|J-d~eywo7|N%nW@_pPG53R4Dnc7ox`?X9)Y*jNEh9C`TKoS|Ksrw zu9a1)8lf@x!MHVtJDWX~sD* z{tLLFa&UZQK0;LjCgc zOKo7TvZ2z>G*4!-F~}L8c_rwGQ_6;M(y?JtV+R4%B2S27{v} zR;ye8^#ofsi=z0f1ifAW1Gv3@W`hC2wToELkz$ZV9^U`_QEtNB3*c_cpR1X+ZdVEW zL*S+n#lp@RIbDRjo=EjO%46`R>e6sy>J)nJUpk~57YvanNsjN65 z?QMo5i{q%%&v6*ggvpBvl9Y&a6wC)@iB>wWwD(?@rymMltjVQG^qKUl zqn3Z{m!=i4?7_vK22x#4r-qx`c*DymP9OAn`-fFTE%UqXgUpmF`=CY8k^k%`;z-_d zI8X;|MdTdKH>m~(aeW1R-q2w7)kyGL)BqP*WsRVd#b!T`JP-~F-pYM`VER2ME?|Zc zAfRJzIl9UOAyO5A`~uMF#4w5koY@LyS!MP%5F`ySoXzgs%2NVf<<)Z_af?rweNHXT zk42cV^|X`A1hhc2$=Y;5vQFW*`*Bxnop?Qpc`W_Z_-+hyJw{irp(FT9HwhnT15vm* zZmJMLHwLdGseLY~3N>YcA5u%lFe~;Ko+WHy=GY&NO?sA{Kq9nO}4IyS7(Ib zZ9n`Q>ruX*@M^yClOoIeU$7iahS4FCAbw&lW|+}c$pR088!~~`$K0IN?S^_7E-Wwk z9Ap9@iNT}!9~#no_^oP*-c^HP%ux2ZvI3&Pcd4nIA928%uCem7CWHn~hRbJ7cwCSt zhke~IWaTX>c?;B5K~AQml7PG?2E87qerK<>+C^hCy2sZ_f6OSBr?FDa1cVm9pGH);AiUX>}{TR+R}J9ire7e6m|NANhvAkL%#T%%W0ljzF0AT26)rjotrT!#LToC(>DFa%U)VnLyFaex=t%BvF z?ZsQrN-HTkHc|$Y7pKd;+!k?dp8HuxYh;{6hJi(U<1;HSFHau;VO|#R^-aS9?IKyN zVm(50s>bx6(dite$qc^4WI6jQ@4DUdw}ky}Gvos$`S3qY*ASNADw!*iIYS_DUlYu4 zBCZFw`GpaFbhHD_QM?oFxnjQKzTnTNsiz$iPr4c_?EAL-c?suM=LDNUv(CA*jru1t zj#ElZ(^n16d<&GE7m;L?8k&Z-ClR@CJ&B;&ZRL}KKwOf9#}|0nrZ?vHi}HF!aWz)99hmxLNtLun^ghaQLD7ZN>k7ECt%mXbc0)- zFNHPGT7_nXhy=Lpp`_$RHZy_F9qX42m3&MBVHC^|+)~$|A8R~y`>@xOmkXyz=I_8c8O0Ar-_>sOicrN+eM zXY=2LJ9lP0!b%(5%d1Uner`Vw{qYt*I^&_@c()9iYQpNE$TgvIa&6N29Wp*ClV(Me z6=gLVR$etK23`#-U4ExFcoIP^`LE02Q&VB!P*q`=(d;907`pCcT7@rI3iY~u?csp! zC%rO>^L1beG;uEh>_UO&{PCYFWJ!sX9#N`2ymgx9^CuJ3feS6(1}O_n7WVG`GGDrO z081a{hUTy%8IQk*LdmZN>@(1#s{%hE$N$#)XiWsu(d%>Yhb(@Os5w9CbB=DoIRX+T ze5k3&%)32BR2uO^2{cOAfPnU=TB=7g6M&mpI~AmBp~Y4&2Cz#T@K|h7JzWM-45cVqoSBy?F_B0S%E< zr{yDycNqixppo0zZq}&RC;TZCukG_lPA`UsvI;4*VVPMaB}1is4yUQ!A`&9>lNcQ? zEcMGCl{&B`3mX>fjRKm@YLnF=mY_IF_HpF7T87`6OvPM&15{et#h3xU@0kYZsHicv zUufO*9VzAhDWj07G(CLwj>Nwv@@R5Mdjvp78)-$qZ!d8s`$ZS+jjFllc}c7z<3wA2zF0(-ZSBaj(P;TJXJd#Sj}}#&ev*%RM5I`a$O;BGhD(nxuY+ zF*Q(L<+{_HppkY8l$W{U!32+K2K`rCFi4m6-(N2x@`5pJO=z}Zjb`W&NOYNM=i$`% zI_$RfR4W}56wmX|Cpm2%q*G-rx$&*M4BidxrXAk_4`h+N%; zV;%zzq*XS_r5na3f!KO&B8o>wwGJ7F8PJ?%ywWCEc=L^S8a{po!q~oho2@E1GB$W@ zxmBicK)hT3-rmN!z1+2l|IKKBtF$R*66U}9E6&wY%>?TUze^TnGT*GTR9bBTy1%nk z`oXqAuF~hHkM(0tLU71NcH6F1fCD6q%*e~Z@CAY&zX5-qFtVewy4iQ}3Z>a&*QDZ! zP%Kd4W5T!w4FrV=9#Hyn8`)TO+yubn(XIb^dGO75`FctS|CA--@ETv{i6(^Drgn$%xSW-*0DJi23tLZU_RiT-N$+1{*& zlamej;o;YqGF!0nHkg-53L{Ox1IMWYnKhLX!``)zqucy#)O4f#usqy{NzzI;}l?|8a!2& zu5a6ko^yRR=qb{ER=|9n8|%O8H=w@qQyID>#XIk*%n(Q6=qry)aV$>H03~#{ev6Ag z+6FJPDIMp6MvHChzfnJ@z%?OAFoTaJPH=UI6tT9deBrL)K0n#gNcu!--joFvN$>}q z138I2*(l$Ndzz;>R5n*YAYn9{my3%<_5%Z5tq|z)d7p1iKq!5h9Rh{z56+1X%cOq7u@)vq~k)zo4q zA6F4U7d#O(+ChzV5t_M5noYYdxp>zHi)dttD>3Ol^U>bRYGK7mQB^wazjQljLi3wO z2l75L9bQ&))2$a|?$=vc^fSulukb%nXMycc7x5JVQGv0-N|V@dU60BA7YO6!_=#!f z31dEW%rA$hG)d=$`=(s6~Des9cD(P)6o>$o|>_ztguJb&3h|$*#<0!Ge2Ky zY%BtC(^PdF$DKaEti1h>clEW7p@a4vids_^wn-xI5vYA>gRID5-!lU(Ajee?GxgSs zgSd8@TVe7)=#3Pax>dWP;OO40loMj9F7%CwqYs~1Jir1Mu}+By-*YE|up`3rzqa>w z!&9!L`I)Z`Tn~xE>)8I2<*1Dbq3XuLcxMxB^J;Qba3RRh8=>C}_4~P@V~~ltGWRJf zOUU?VOj+SVjLsPT10wje{cfRHYJi>soBl%m&O#}9sbdR#SdRf(H}?k}4)kP-gdD|G zyAZNQr|@wxX`H^;nC51?*R=&ila2^nJZ+#GrpY_=`8d>SBJI&8Dqo)-X`R`6wA4qWWAA(7NEEVwV!3I&VstCwTNL@*6V~PovPlA6jR*;wW zxajG{>%ZoTj})mA=8P}!vOJM=?A{}bk=a3Ema6ddanV=H6C-R9xw&Xzj#cF!EL6{l z%|8}S#2NC$^9!_BE?l`=vcr}M(!60jkZGE-v5H@WcpV>U z?bbO{>$D)Z-&Cu>@456H))zMq4~Z{`E(8(zO(-$fN5qp)|T8j=x1I78v&8chB2 z>OgTKQBKz<;2M-t>NBsJ#2HB?sI~%{wqAn011y7}AqrvV0@?@%0`-^Y&W)*|Gjj`#ecp#?1C?moA}EwZ#l9BcYA*kY+&6B2Ytl zV0FCR00UAMqg^O;qK}l5_8R;TQ7qmIKO&HzqBBkNIi7a)q=-@zd-coo=ttP$ppceB z8{)P^)2j%OoPG}yj0zow0Y*=B3~RL76QbK_Ta2}hJGSs@N5*_jYRxY}je9})c*FP2 z)tP%kZ#lSENZQm|F0h$?9+0A|z$FKpRT{fIW7YvBe){_UUg5y9f zrJ~&z^$OG<^&0$@E_YmpbqyVOTEJeJs25#W$EC(aWjptWku=VP`PH+UMj$_gXj1l@ zA_@IOwfI7b{kZR-St)ltdJd=I;SW=!4^6q-xxAAT7bJ)XU!6{7-4H{Bzj*cd_^X(C zSN!a@=jZ1*U0LVEFE?rLL6zAD5_dmj;O$e;^E1^^daq$O=8oq2716HJ|Kw+)2npK; zLo_^*?W~clBbxlTc+=`skX6SpwU5Mo8Ew@`FEG2m?`?s<({rOd1`Kic+*RhtV{#8w(3p6cSY0qd*>WB zJfp}lRn?3Kp6P-G4%~32oDP252I`tIXQoo^tgCT|`e0ZI$L3aa6a0HifQcA_6`c;XwDw@`aFOHLVv0L}@krbmxAEe-Fz=zR1a>hQCID z#lZjq|1>^L@93a6M)Rrp;~uHmF6k2HwEDzpo81!D+GH-vglUg9HwgbHk&{`CRM!Jh zk}U*|ItC=_E+%tFLECj}+eh?14w9z!ubFx2T*cf1o4!0g$0-L`kb57Vh_qeIS?4Hd zT$^^f>%GTBaehjd29_(*X{mMb+NAomcpnWXLh31-KN0ZWp;)0{g#ZO8+FXj&?4szKQ5?nt+sS-kD2V_URs5Yb~-Yctw|S=}M*t7+^Ux ztO*(z$>}8kIbqdMs_(zr6gdS994foZ9>?ru$BIt0^D!YPpO_Vl{NB|b8$oy%Vdo4z z1|+Clj;29T!y@3Ci|m5W>s{iXo9hn@6OX}n+g>Pq)R&5FrxtRTeEFeeLNUI7C!bj< zMEnm4v_dmVTx|C;`KY3}W<27Kxh>EGD3)rBqCC@JY5kL;kuqziPRa0{%Yc&3c-+Vh zBgBDhYgk+Pafrd2Uw74kle`w))}pe@SvK=S+Ft3=9SN$gv8EFXyHVu{4g1oHpf67$ zU%b~oRd_$uz*mCb<%Z~yy?&R$y z)Pj|q4n;9yj`+qu$_ZhQ0@a>kQl2uyzhlN9RhV3#jP{tD`-cABrpJKW1~uMSlP$7z z{RQN(pVwY(1EpN*9K3zvc$wSCBO=lEDYTgl1F~oL0<&J{E+k%Lix6s+j&%`qy3g5i zq_Jsm9^qdf2m!i}>c}K`+Gn>V$JR?jBWwpOKk*V+xCjY4M??E0jw3>qVpS{&K7e=hdgchz8~C*S!x%qHrzX=Q*J z3ZD@BD(CNWf;RlXT>eRr$Rl&cbAgR{H6rf$;p}4d+o+cjtQ^@K&5O=F{+D?${5%um zhxyr*wF1|we5_&X9~u+USJ3chO?H|FdF?K!R5S=aA%X;ZEZS_+-QJSEvoiFxKtFx| z!@tOV4ic{+9|$8r3Dd;my4tDhvgb0b?fm`c2i}a#jyBOU*cHk!^>Z*kNYFiS<>FAL z4yJc7Pj$RmRX=@w?V$Sw-N_9yHg>I5H~X|wQcX@@xVnCQ+NZ|z&m8FjC{<0ooy$Y! zLxY9#ALd&M1uuWM^%WLJ$$O?EhBBobx3^~X67{?K13V zmx4za`z~P0FML^?)NTo!Ca1QD@{s8Vh-i&4|g?Ly3P?WvB$u&mO-l z&Jl~dtKE<`3M$k8NSpt$VJ_)Luh_va21RoL6ZQ=g-+gtv$o>fy&Zc9GEcA|Xe6dTV z*i#etDbtVblCNX5`oW^f1g4;5E%I9zE!4_Xqye){Qc;WVenoc+U3`+Q!@s{5DUXl& zg66(L?C|*M902@n`mt5Y8|ph;O22XqJMg3pBs>7wNCe|o3Xl8d7?b`7-7iE6DKp&? zOk2zRPYKL5sS0mf2rz~1jO1vtD9L%$9F6#Cl(~Iuw9;RXBjn6?8*C}Kj0d95P*ei} zhAz|9ccgM^WNa?7MR^d0IKj1@_#khG$Sf+!S=stRaONBC9TXXl$%f&^p8l4{+C`P{ z11oTHE=EIhLc>MIG9=y>)=K;Y1iK=R#^3Z*jBuU+P&a|XkN)0AeXs<$J z{>@(81C@=EXjC#DU_j1^$9N%Ymw9694$1ri$mG@i=vRMCVnMYtTcKE70-mJA9CX?D zEYaBgGg36Pw*f1_bs5{|Q{q_&y8O48Z=H#=Iz&V1EKb!xP_Uq=thKQ@WrNdIsz^=^ zlV;ZzUeHKFb7}$rT ziWN5e4f1mzQC0L^SDfKl}FIHM&PPMza53l@lbU*ox^`w+Pt$#QZYq`}h<#Lz^ zQk5N)3omB@@;BBbAkb+|PkcH|zmm=|UgiiF@iMJukpH)|L}1^6;TXy?P@-RQOWz9i z+h>Y^nUG(qpblf9hi^_2#pB~Wc*;1f@}>FX?9c}y=o_@`Q92csG-3VEvPBHlKQ$ED zb5_Rq1lpAfRV75o*IibA7Wf@c*(`NH5#XLXX?;oZ-XB39u!Kt&Vbb_IzqEkKy-p${ zgjU!nNdq=-*OoI^vP11>(gCn@rFdo^th!S?v}`Vs%aQsD)aA=d?}C8AE5YI(m|b;; zm1V--J@CgP?J|ZsJ0=&`U}S0RlH`)Sp0Y~mo-L+6Etj2}q*)2a1reGODcR`+%Q#&4;0`;7i~#=c(9Vay$Kld3^!5QkLFbYO(e4CSK(4U}bo@-FZz10a z#;EpHjWg&u<=bj>$XROnFbp7KAGfGieI@62rj;wgd9uKa0ihw_kfo>H@W{q^;vUHG zxyYKXf?3FwM_v0KZQUKp3bJ6!>@sFo6n&4tHIK8Gav8%2tETD7>hM5hsem0HvVG7A zT4=yEia*WCL6D54XJoXNQ#CBonyoT3DK-JQ!1$MQvpCrI-lMd~m=TT8t9)ePtFFd? zaI@~(LX4VTxXQnDeQH3(_n@Q9+Kx_Kq9p{QO@0g2#tC2<1KCZBJ@Evk^ft$zHvjhsnADR+I@*=-&oVp@>PQzu>8C2>#z=D zCZ(bMLb`Gv;_=A*fnnJ5s)GZC3!z=7m&60s*Z)4j1E2reHUk#Z5W54DY~KNK%W;O; zZs#W3m#)$C6ae{txUk_=2dWDz5$!DxtY5h7_7cX9j7g{a zRfD3>VszvLlne}xhGCSMe+C3bm3ZO~n}+dNg}!xHlY_(Z&Hdl+{ptuV4U>~Fmaj*2 z1G^lD8wlN8M0Ivj6BQ6!=u_eY*c;BmI}C41N@>h+xxr@bqIAXX=ILf8`%DQtNlXd( zy?Au_41H`6In;v$>*;C*<%&YH-%mhhxya8xgG&|c$xKiO4uf-O;Or1N zOFmd?P$KHGEX@Pt&b!Cc!%AY3@!P-AUil3ZVZgKWg+H%+`uF9hZ+gtG)bEq2_yS)3 zJ(ZC+r!pR4RkuT2^Y>6@8L%~{Ovayd{x83JKZXRA7IJI0==G)BPn{DU!aFcgTB9gw z@qt}gAyC%IJ=N`(v`}>qw9hJhB<@JUZ^4PxC2Oj>it;EQ2fI#e%G7;ERNa1~=S($# zzOb{sH(X>>HQ4Q->>;06oM$hx>4uI057~K#C)rFKW*CQpT`wi>Xj_T}lU$#1YST{l zKQFjLUNT={DOUs4P$aRd2C3-F2ba}!|6WIjVrvdoq`&HWWQ*H?p&{ou>} zA3OjyjCPf(L=?zBfy#UPjx&fCIv@$uI^70qz&jh8+11nrlVz#|s0G?6cFHT!QleTq z(?D9h|8H76JV9JQg8`viI%dA;{ogt9{W229kH)~egB{E@z{jr()*{~>_y763bKt-- z;2=&G4T@eUb?I;}+d}TZ{-iwX*CPJ+WtEDM?&Z~gblHH#zxv;Uh6K-~Z*w*|r04(j zEreqhOKZq^M!Ue-#E0Z``DUvnJuPH(Xm^*wq?^a?D3K3Dp9TJfdjfnaQb@I6`?qSl zvFE(@=c;#L%1-U?C<9X#qTThTZmImdM4q!9&kxB|oSOd}?%4n7;r{O%@yLwB@i%RU z(3^S^?Dv5}=&SF>*@6Urb=$&YMbG7xQw4dR)Sd_4LVJys)ZC$ym+^nquRO}Z4Jz|47BPC4@3<>I9IC*oo+x5@51aV z_FR#{ektYBzPn$QalR5+m?6V`Q|4QOUG!xgMsd_X1n(&fz!39&r!6E(t1ApLY70>^ zXzUHSASXcsIqmGnv_%$)-s_)jprC-rD5FT4^b8@rlRMt+xam5XC zQ^csV2yKbbNMd5*@qdpGMmS0AG%U_V;UVayA(&j&nSIBb$ERzpUDPG@^(c zb%6B~Cv7C9YxX5I2?pR_jsb}yG`~1m} z0(-$0Niw;Bti(mnR*89lrx$s0$A4NyXy>axV#_qtF&X+i5rJJ@1vbme3TmBp?y)lR z6e4&)Oz%psLE4qh%Y|U1nIJswOaT>`_9X#97j0~ipl5%IfM%=?enbgC5DC@vJ?(uI zrL#L#l?QNv%H(0hQ#Rv4=k^cdTxGlUt?c|!jTNarNf^GG8bVVjLc7c5(-B4(7lXTT zrY=X7!#njNzVHA@$EyRJRt`BBPM)7_q(f}TsG2~Tt&+GiS&=Qc6if_tY}(POVHfP} zqI8RDO<449{6!zB^-1BeROy8NS*pT*=YBSt?{-C3A&_x05=+6_dcoYOcpOh!$9N`h z+>)MXYseY#qg(2l ze}n)ac^rWgmMV`NmHQJc0I*fdHew=d%j?Tk2av+g8W1@#+NC3J1JcRHJ3#)!F5U0A z02T{2z?ver-ot=;VS)kSQmBg688lG4cJDUD z67>HdK8HR+3g)KnhBb5UGumT^$i#x*W2oQ<4+_;JfQ_h>R1FA{Sjp6H4`+vhE!2}L3)XjsWWMiiJgE-R6 z?L1iZe^h5d7iD_z_gqt*An1*a$x=9c1(5WjCl*wq9pE|^y94!e2&^FQHucN#i9>D# zHX(HXcpllFfHVUZcnfl$IG9mvz-|5S@o$&^fC510P* z$Z@_S{wfmlz9)qj_|U_mMx8&gIlr!jK3sB&e(+3|%K6(ZyHDCW>NVRD;4fLWr&l@# zfJqC|i+bIIBJyYxK^6BUI*|Xi7a7!?EKK13!|-94XH9>y5lsn@n6PK}cj85T=@AG% z#n2D|J;9G*eXpZoITFDJ&{%{n3-ZTM@TuS|5zNq~aW-vNYJ1Y&y0MJ>?JF+HU+C%1y@N0+kLhxLP^OFnDV>U zEGvossAD~=22{|kx`*YIO!<~#`n7FYpm}y|;m?2T4s6M{emo|y_X_>|zGpmc_Q?Oo z-g^f{wXNTxL}@^w0SPKO2Z;(O8AO6)CFh`sfCL33X9OgJA`)AIHW`#01Q7%zNN93K zKxneiWbn=fJDl^YbMAYu&i&)PSEq`qUAwAXYpxl-`GqmY7ewEv0oqBKjgfsN#&M2C z5MDMgg{+0V=I?=G-yDakndh3fthV<-j_F|+$tT=NMg^Q}dnD$bAgzFWrE8dLD-FZP zWxDif5`YhK*VA-D$iD&M;ryYgrWsJvdFtuZ>gac9nV`gXGeF_IYX80$U zC@b*pgTW-;hNJLrD&$2oNSkWtqkT-^%S_wn0f?`4`^zGM9yz|N-qp{Z%}hOuBm(Hj zIL(xoP4!3H{TiOlH8(zjCX;V#KrE}HPk_fV05ou(PQ3lHNY{)-dIxCL_dn>_RAFhy zh;@;aR8DG%yAQL6t!JEUb6`M5f+v#wi;YCE*Ui1@pr@j8^@#|F*&h#hr*MjhJi^C{iyLYwe`w{G&!J&6Sp{qyU zW*+mDlHbl#L^IWc?+c+sIUo5-wQX@o1iNp1CcrETNz4TlW)Y!JeAw!*LQ^%J?Lqax zZ=2(V<&u4Lzm6YHRupz{J`;Inv4Xt*xP+j?8I=1WxxctZ21v&DnF zb^duwh?sI1P4H7$d&jvDj1C$1xp__BFBe2r{WUlE`et>D8$w2zvzg>b(tHn=MyFs> zIHFJfiX55nT}9QE1_SYw7^SeQ--g`e+T3-R6arwHed?g0;3(Zz`JAvr0zqVN)&Lih zU@WG(OSbV9iGYH{jPYd=d;OcyVhX}Oz}6A5bm$s8x5yxXxqdV1@iWbp76o>jER(!8 zOT2M$KPcwJbW{XUHxJb|WD=vx_sj!L1a_m1gA2KkUPf9SyABDzhdwkqc9SCzYXy&z zKUfwWP`KmTy-#zy8^KqDB&jFcc{d8w=JKZGUi@0nGF151lu<&75)`5vGe>+OQ=}!^ z?-3lf0B*wI%zV1fj=^yW^RKUo`2Dkq!40mT@u>me!d)h25z0wQ2}w+UzTyab7WV0x zer=h&mY(yBl59O+q|O!5RTw6Z#o`6PTyv2=<_Ep;=8ELaYiQOz)ndg7x2rh1UG~#=}!rG=XHLgHz*UIK3mT zB8O&!^ad>ST-+^wM$?I-trdXAcZHUw=tt*A7w}zQ7xW&vIs+QYcfXS?60t=_I*})P z8#8YapoyBkr1BB~@X(5(#auXCi#ox_lc5Bg7;tF?9`PvTIhGIT#zn&|#cs@Y@ohiY zl|@o)!B7Mrq2uipfcy098&bG%KCT7^oe0)9-``Q7 z0EDB9dz0v3J6=%Fx|Q^w#Y1t8CoaWCZ;yM*veW^oINSHXQHV zgW>x}mxyM=@m;q@z&vz*%@MwcGWnpeHPiymKpe4v9*mu>wQZ{;n2?15IAgBgxZc|Y zTr03QT>h}Z0TyPXSYT5e9ZYtTG<5Em?_QJq=HmOA^!=|_9Bq1ONUsTekX#MWc;p9xMp?m%{O&e0&tZ>sv~Lpn2hOYsG=CS!}FtQ@VG=jM7~7fhhO{g@X` zL(s1aRD+-o3&ed9fno)Lz33z@U(j7xhg5Z};!%DgT}gqP{05x)@%yf^JDft97I|6tjz)5LyF6&3rY@-{r!(V!{(S0vKfu61H`yOH1{$+rT2wi0O?&k7* zo^*iL5)EB_ufmljWyLRxrcc`0&;ab+%Q6}xhDMfT<`#A|Wx!?YfgqwS*gi-xDhbte z(AB)jn+NztgUjE=*Coookcex)!dUwYR=mRN-Ro1~*8w2y*99o4{#o`wPstNJ%RQ@Y zu}P;d#{rO6D5eNk@~~E$oei z!}yJd3Sg5^cWivTM3_B+UDT2dddS;ub%_oYSCh~PAatEWN(2DpE+4||U~K4{KYb1R zaFrnwU+3=WbjZ4HF5I3^(UYC%U|#g1VNh*+J~T#lkGiQwT1#`6;*r5S#AH*`yd&5& z73y09T{?bp3yu2ECR{Odu@?6q{8|FldqX<+=>Ut{_l#bLNHYa`z6_F4l5AEZy-YhZ zMb9KAm`67&ctmvmrA(N5Sj$Oh-^t@;6ZMD`7`hxyy`YimHu@;J1MC_Tws#%AWW98gvxdeKhu zjGZ&()mNd}cvivn(UJ@XrUOjs1q@QaPX_=1-f>Wxf^bP2w0%3u$9*j*yJYY%Ek^wjF zo-_%%L1w3m<7GHg%!XJs3xjMSZ6^?5rGKU;T&V(0it}-JGUe{Yz2kMi;;F9+Noi9@ z3!>@5BR-VT;PW4-gLz0FzXBbZ8!CuPc&-5e6$Y1LEEDF2a!{q`q*LN#M(L(+kxFXQ z`R>iQX32K8L*ZWQk7nGD(f~zlyFdTB*sDQ#Rp#&WKp*-&M*0NFVSY=WIgJ1K07Gog z4yuBsi>gFuYZ1P}w}`cP;p7HB@VN}Wdv8hX`aow<;U{x#nmu8=QOc@UZt|;8odm+fs&E8-YetErZYS4v=e6%Q$i8F*))ZozI__zb z{`^u7Tm{zHD>z-ONZgu8-B!Eja2Ytc++6^3!6><^o&;)@JPHm$Ao$AmMR&ty3Pq6TcsoTGDPLx}UDAsJ^rQTeT)g5tIf5ik4{JfP2;QFr{y0KE+0pQ*}9{yf*cw_%Lo5FnuNCfI{-aClN!D`%yP zy)NE3X{CPl;cI$kJ~`?Vo6U*mzlf^FM~P;^sTAlprg@2iDci)94|!|}ZrDwF>`M}% zJxa5$w9b5xb_O-Ze8`oMjulla;1Vp*GknD63J}+#=ktsq{~Jw-!X_Y1?o1btE7%P%F05vKnv9@fYk? z(3j-0D#kkOTU_Jl<*+3UiJlB_Aw>E zA~KH}uO|n37ImfUf39B9kO2M6J>`WL9#@0pNC&AbJ~?||bKNBnwHJwDQ1@Ji`3+uU z-k*MTJRH?R`uToVQl~IySDLt9tES@A%FBD|%C3_sWmWCYUjg7y;7N)@O|I|O3~03p zLFrWC+;nJsCNnX&6{KQrN6rPh6hfMQ5 zch^ju+h|&R!Tq}2le1T!<~EvZ>61v@6c_0E0mX&n=t%Hl!3I^LhlF#ongrvZsHh*K zx4r?I{#4dLhMGX_oi*F^69RV-@oGW!1bXEP5M2z)djmPyTpeA59P8@qL`np~fi-j! zAQ#n(UF6Do5!Dq8z=b1KWzPd^R0eHMYN}Q={MUdSxyx`BkXy?wGO5k`97M2uQ$Qi= za?TKDW#nFiGYWM0e14gw@Gx^10T#7Xx;a}Kpvo~tf_oOL4pqJ2JrW{7BTv*=oVjods5rZ0@H#D+`L zJ~hZm>zhq{DIL)65K78Wbg~)aw-~o*N}q(a3#ym8SL{myL~EI*LRj!I_c!lpY9w;( zDnKc;Zt}Ji1W6AKwg70pCD}_nD|C3f#`7Pq+X3w8js9wGX1dq{JWO39OXdBmT(l9u zwm!Gq+>w*5cNfvv3QK9vXbiJxR#A*xAJ8>Yi$Fj6P&!g+{5_>BqcM2WJs-X`Y5r2$ zG6XEw!^mx(-~e)oapNzL zgIm^Ho?el?zsp{i3A30Gf$l~A+feqS3x)0WrqD$uxe#~U=G|OkB>pKhqFwXqq$wwo z`uoBodO?6wUi=2w0>N||f8zj7SstS0Hx?2g|En98?aJbePhoBJ8}uOBw!vJwX3eRp zFsB9|(9^fTr!y1$#u|1JRlmJgKLRTO^{19>+$NHezaKb?E#hqPclHCI?EZJ)x(eR7 z`O_;xnGkyK1x3KffDe4m1XkDBIC~)?x+AV2ZPbhC%=7uOr|C}2?1uaRMdd)$Zan>Fe=ys}t zS?;Wi*n$S9XrM~13vRxG%Y~oDpLFEP2AvCuh7x;z2TSiwE#4SzZojAg#|yds%?m*d{r|=bU-x^5NkfTxOfP0_!U%{f@V{3at@c9W1gWryR6g0cm z2A&)4t*@x@VLg93bB0)UlQR;?n{%xH1YiLGtUrLJ;Z9Or9|lir@It%_r`WPTQ7(T2 z%8hYJf~ghFtPpQ+6gfQs^R3o&J}p7+!DhwVj$;9 zsZd&*{^~WkTy%x>eH8mRHnBJ|fA>6g)W9-ArD2y>?!oStR*TMaoe4(4MpRv6`%K_0gv>P zoY8KKF62nL+7zTqdg(^WWB@FhC&*x3lxtHmbf}(p8L4ulF2%jdnsV6x$703(hl|CZ ziSPxv^<}N}{DKcJ(4Z0owZZ}@3ErIN7p@Fq5wmV3SOkjs&M^Gm3k}SQh~)>V^-NpP zHe5(Lwa@gOdRiX7>xz`BX6u2qxCp9Ri3@m&^fZ=bV)Cao@vdYYV0kxq$#063B=n+b z(oR6*Q*OUV%wXZz%kE{J0dkeAOeqFF00=7-8epo3VUU*E5D>`p0iJr# z(8op`rZ+2Cp^c9+b^aKv&HsVH<`>o6fx<3B_a?$P3!DlJA5%hl{DmUs~}N_5`gDa;hrbl5?Nnj850HMVhbSFWMDlgw(0|cW*#Rq%>5HkNHCNL)d$&1 zqg<=#n+oeBnQ62w=GEW zPjF;-XV$nZW%~X;oJw)a+5$Qz`6nQgu`F=0#5G^HP&p;`k16xDe#LV(>dE z^#rxg?dAv?*NAZQQc`2>M(dwx&x(k|gRFdk{l*kQiOyg5s{s-89eS$4Z=b^tlW(@v zTNQ8d)ILl)N*YD#MSevL#annH_0juW`61GK35!0O{n8mE?`SJ(o4{5e^M2lp&sK6B?1Ta)=^KLiK zC{d3%{t4sq&z{s(yEDXPza<2zS@{zP^AGNWFr)MfgcQJuxdI_MZkXgdJm+K3oim7@ zX^H;_HOg-OuGC%_O4fIzvI1}oDv=gsuT%5eXOE7OpOh)$}9 zdkPwov%Dgs!j}_OIM33JfnXH3kmmA&@;|`l~pZxhc+up zFmD&6*9HzR3t9%#zaqrm52jGpyz;nAtK}2gpz(g{4f>-=5G@;(=3B)`>c;4Qb!NI6 z$>o8MS*;jTpJ5IlR0B8+6?`V*8LGxPkTc_p_(7BF3xFEhggwkXmM&uQ*CM+WXy?F< zZ5A{873r`-Ou`*3%qkH%44mJ5pw@sOxiZ5>942b^pXngyPtO_WD!esx-F=a2$G_t{wnc%inUCuP`|)yM z;s`cLW}mmmA7Gc{$@1koRC<>T zG$QiEcMXg9Xx()eUc%IUNP%Yrs|Bd^AqyRWUf?c{WjQ`5G`_ zN~omcJiw-1?zu*ZPsJC8i0;jp4oYeP&6M19s;(3SCA}cPQ-VDWauS_n#VK4N&KePv z8jYyw)u4@;Ig0=W`AwqpGk z`pYmaa`wFp*Fgir?AtUcfaRzaW0-jCiEkJzB?}|F*A(_R>=%59?B3DQ*7ssg!x%*L z@Dxl1Ty|57H&%f(MB#AdGwcd^eOlOL!wIH%05lOdFpW9@MFGQ@Hia9s62XFOk;95B z9?dMIL|gGM5mG?C3GnPOB2f~r#WK&2-lT^!O&)^&!+Or=O}&`T9f2GybBeCF9IU0< z->Pwp_!lBGQ!{`Hnr})V?B#ApK=Jacn#~uO9iD>Cl+c8&~`0Zm%eYHVGgCrW?@T+AP+65tUBptjq07=GvoGTLIQ zM9ntMWnbH75v+j%Xm{We>xNC;o#a8LG!-iuczY^Z-MZhx>7r~JK*dr)cEWyD)?Fdh znRWQ_n+J(p)WwRLtzl0I1Gb>#c4#Ws`i*JgVRLf5NZ*z(gk&plDg3tqvCTW!Da%*G zC1c`dftz9G!Ejd!G{OXI4^+aGrTk62Uxw7>0NbQ)0;0E?%kEj&Sr|5tReYmq3xBS5 z4}1}Dl{ii-`Se$KbQHHpIJO$CZ^~%4;$Alv7tnH+BxEWOSaKLH7mpwjSR-(^xg^XK z*~gw@PhRf6FmKKxvC?u(I3!0M|3_?c?%z9VgoY}9%wgXGB@QU8MMUi8Jk|Z0o+tNN z_>|(GQiT$C^!`i*G~@_783apf&p=@qF+W^!ci4*5T@I>s`q#S8ZFQ zL@`}fsfsC~JK4(kebubZiJ4i!b$lCg^fX?EuL@>fX2}ogvja*zxAM&=1Y2pd;6ElZ z>H@`r^}+?$_&GXY$2c=|EW5`uYVwFx0Fi6j9J~{j6Ztr{@se{V+W-r)W6>w98??Rl ztG>|osef<##a>uewDO&LE-qO*mTtkNtFt96*(!y>6$gh?zfH$Fq=`&cLMk{#3Hnd! zR;jZvf{z@vPH);@&228mD$XRVaXmYm{v8YpN;_${31}!JH2{Oj_rn6mnrPB3mhfN! zOL#q5CU@>Oh?KdE88#Bh7u1}X+J$rfK=?y5{N$ez{zH#{{hs5Irel}!Ynr^V&C%Jp zsF5@>ToAW>^O*38?M2k8%^MY!L!Li*a^nj^W>(}VCb=-I0K&^;d|A2&K$xaOV^hGH zJ@`IkLzKD6jB0h8SmXQv7SL4?_7)y8%ijD25P=J*=9wIl73?z80SmT=1ODwn_J`eP z^?_oa6yG?J&j792HlXqHwD`1abWIDQ&r>OaQgQwmT zb2rQ&Ltf6P5ije81Ii01D|(jk(hK-+fvzXQ509wue1a7=-?8EWnmJ5?X1;dBI^{Oi z;s!f5yM+lrJAM8xde%v)G^F3-?<5W3 z0N31&oAbd4_E#f@8-BKI#-w$Q(=yun1Xw(p|GJc0py?Cq~YHI!*hfK)7|oOHHE8k)P{0oc!Y9l16k z)mTu*$5Y!t5zYo82AJhyl zOuQFes}`M$5Z$R1^;Q|~79iJG$XN$c3iG`vQ&}K+@=OmDs{ovRy9R(lZ=1N!gdB(~ zXtq*X2+Uq?rQ}$x`VmcJ`B5|+;qc;km6vBqu<17xUL|kb!#mBPnE-^r-SNHfs&CVw za~Z?s^s**RnGo)bX~BIwxEJAm17BZ%Qu%|R> z)2`bbfz^z!Pl8WrmJql2tpbpPNUSa^Yxq;+qw6tWoG^g2*EGRE2KW7o!cFgIF;dt>{?zu*NuRlZ3m)%V#v}0Pw1BktvtW7_A(s z+4|{$rAc#_;?WVO^2$6Z1^3anU?yj$6ec^-Mdrr_hOl46g5ZXp%WIt1=n>S^0 znr!~s`R{@s{l?G}d3W*fn#2=%?YlGc(4F=_I8~=I9j$gTag|M+KOUxgC=h=J7j7{< znf>VSb66s}DHJ*_+- zuT?IRYe*WW&z>oPMrLj2|JDh?5vrrGW{y4rDHlV&c@Q{VzGB}*9xQMvZk-{GABVT1 z)z_}Y%5SrUdH{f~3j9b&<>~@mB?HCyT!gr|o65n|=*uCm*yD~dvP+^@qcUQu=F1&M zZ;;TE-UV_u!Hx#@r7hcGdxonpbpcBx%V&pQ3RsCPrWaANES`6k70Bz)b?Nl+AP?=L zE47`9!W*MQpPL3um`T}OVs-(7n;i)+MQ#zegUl}x>5ENsazfHof!=H6MbaRxrPXr3 zA0*x@FN->)nlfyY?jnFaDy}1<(EFjr3HzZmdHmxLRN%ik8e2d*V?Lxq=R5lYmWl0| z2?NE@&+Q(Jwx8t^uk!8`xbmwl4E_2@{Y2$!;@{#L+Z-=5rQ?ii0nzWg%D>~}zt)s$ z)_PQzv_hUWA4o2LDRMeQfb4o8b>M2D!1}cQ z!lyPyKrxp01yz_{(C`p!q!gsf^B^2(iE&{kKvk!PMIh(j-x_F6=g1oy7eq=m$k0 z{|CLP@1M@tDWkM_Y%Wo>k~Jn{jSL|z>&M)#3mA)+l1~LP{^DN!Y5mlA`Q9G=KKdy= zzd;4G5q00EKDjfUUccGGfkG|L7{J#m`vn{Wh@#H zButlC3U2pouE7uGkkqQ3p?Gn4EZmPNolw9so>Fq$+WF2YRo;%q=BT9R~G@RsZwdb_F0*vLl_PIni>X%9wwcg zvy#&t#=~rc-in=)b7#CsvT-}rgEhGQvyD@BiSm8HCaTtB5LqmK*!wxp^9KHL2e|b0 zRYRs;T#+|~*_uZcJAK0JZTcsJ2_$XC_9UB~Tfflu2fYQNFD4PzG!`$y&u{7miH6q$ zuw5?dBz-U{dxu~2VK_3d=Jz5acSoCRSWT6KiVcmoCq*~Dalc@uSO7H3bB4}@cr-E01dAQY(&JmAhR z$TW*;{R9_k;VJ3_d=-#=aP`!d*og{P608wW|3x%)vcajf^OG{Ho}$2Wx3--esUIoz?+!|!6QnJ+q^^F^-nE; zU}skQAiXiB&RckiWf0;teCmkhI|yFbc|sb4$X1nTxCmu0F_Z6b5N5&@sD8Zi9K-8{ zvCzI(OL?crd#f2kRI9ZU>Lb(KvMH-y4+?^1Sjzwwe;%fYD|mV4v49-;qo!a&TTsD2 zILwUT%Ui{gZOf>`;f>a@#0%oz!Dk|%7$%Oj7}`Lieam`f}||X7NljeVHlGdm)_QhC+xBlhv)FWDL0i`;^-)Y zEY-^|xbaYC-dn2C6!+Zup0~`6G&>4%^eeMIJz07$Pe8D*ZqY8D;lH%kRPnvP21*uM z&PgU_gSLZGn}hg8wR>MAu}SlK7M#;OK(XJo_d4BUG8EZ2;-Qy@g!6G53+KCYpc;pT z(Y+>PlCTJ0kg(HLl>#a4*T+8!FMF`CFN!vj)@@*sHC5?p`mI&}+{Sb^&{fE$y4mkG zQ)psH#nj0x>%)(!Zt80MGt_VyT?}tX4a$|LeAWUI>d`CLi*)gy&QmY1D#pFGB2g<-hFx-J`<%DmuS@yUVt^hPFpnkAKjoLIk z2JDn9AZW-X?26VNi8=|^k0`+cIX|P@pSl7?ueJkRX|CeWF?;?8^|Ln>jDfoJ0i|Fs z+3Y!5K}*_%9vPW9yk2}u%HYfQ!pItAB0?EyR&Ris4)i2gJ&=2|)oTA;B1go|gtRgX zbBxwX4;xsMq%R19L4JUp(XDOLJ2WlPHRcySa!5Kmj6HvBT9tkRRt%y>`3m@ zC!iF1NH{Zz1w7-`xyz~VvUmf<6y6X>8gCD2ByL($;`s%uZ8iupKiCMj`W&P>Zgij7 zc5a9q<=hG6UN#5R-qw~d_yNhfAuEsmuV&{uXsV+%C14hV?=>95;SsKWYKBcUDmU)qxGcY(HKoay0{Q$dBuC`fkB5X*0Ru;vx`aT}cLp2=78z65Q}OE8S}SM$UzLW^p`M3z#fH6RCoq zCF@N*MuUfWFO(52w}=J4rr-^e%AHpNjXOWZcZW`2#rQvV5|l|1Bs`d$Qmkikn2ExT zL@A1YMxr=L9qJC1m&JoUQ%R95_Q6_9k0MMS(WzWp5e22+YqVaeV%DHH{PO%00-C4S z{R!5HL9J?6SA0W!jVbBbt-HKHdD2*52%jDeG3GuPbZG!0+PU~(apKXiKu@%+W4({5bnmXTA6nAnpk3#5yI9qu{|0`k+eKD& zK?Teo_KH`#0o)tp(OKs2vV+VQR~GB~CsmO5^trt+ndlrUliKb)k?V7m46rNVW-OJJ z*@#KY48V6$ngA}o*r%IK58w1vIse$cBItfd`izebAfQLSsOKB&;PAcT zSZ4m|xsIMmIqj`97F3942hg%ORRKN)SaJSAegCU0SatPOJvi-I;i>dVvCga?HFQl_ z{flctMhYK;$9Uj2L;0Btc{uGPCNe1!%L0{w}j2R zFR1!PdPqTB*w+z^*~6f*df_b{{1&m+;LIe}YQ-3OTfkHeu&81DX5}K`d&Lr)oqZmL z5t_~8U%}F23V+3PI_QnKGrs=+lR*O|jCYAMPGH1yQMY6zJ>K0O2S`C?!-CrAe^2`9_&*;tkqbm=#^5$z{KeM&spj1K5sGmyXmnQG79KowdGfb6Tf0d?)w0O0a!Q;l&UUvKL8 zSP)bZDHohy@-G4UVln?iU#!tpHtj^d>iGWIMk-bE$v8#N(Ecr31JE-~vHTEn>?fgW z@^$6UOt%a{RUX1%Sq*t~CJBnEV#^U0lzzQjy8GpVa^r(jsHD__Ct3l(OeaACx%f1G z03*%?{c{qDrw?iY%RC>VcDRCcP3rnF&gLP%-+qKAsLZ24=%fQs2?L-8Tz4AqUh z7HE)+Ki6M|#7HtL3Nv5A!McUX<>#EQE-yj;O!@2F*Qe{YKR!i6hMz*wnDKOVy0Q=s z_>9>bbsk&Wa|n1lU`#6yH2C=qfWEJ>yeqn0Al6%F5NSMlZD4&#^23A!e-zj1nXkV? z{V#oeM$j&=?p*H>ltewfIV>vTx`2EQl(FU|P5jGl{h?rc%p28l%3!z$&FqsGDq5P~ z^B?p}^8KMO{t>{5ZvL&C`Io{tD>9Tpxpe4M_gLv2-HU&Nys?sk;No+UG9XM&wcutT z!N&*@`Ia(HL*Gym(ku7k1ppj^vS-PQaU4W*8=x1gVztVARl9@7iTs&$sS46SP|SROWu-o+~!CXj&g08^NhX#?8!_Pko4w30xypFkfNIq4TLlx zoe+`_)LQ^A4gP3ZdOrDhl>;wET$iUZAMjHiA65aaFR{koKGVL!hqdd&&iPqyBS8f| z3gjWy13)u|Sa2sb97L~vCrK^_Pc{uYm-}GwxZLrv6%~-*d0x9(d{?uQZNb_DD3*<~ zsTYE=JO*PSgXG2kVJrZluDxdk1iJvB+9^W@fVD54BxM54LHaaGl?u314z7gT91L^3 zy|hwQp7ix#3{-I5TX!YrR9X=XlT3w?v7lTkq{VGM_4S=3j=56|SS{uF*j@PuSj>Xs zkNsW+*6<37cJyaB_47H~vJE}8R{?boMhML>3(4D!WH_m7W{5hb9{{{^;7W@GG!91W zR@gRrK&AZg*kGw)wFw;&+F`79V6?gqoBsDo14=Z*=bi6X$`q1PSO2q)ardCzY;!nh z9I2J(nj&I*q9GlZm6M|oO`E$R!ZVh^Ru$5cb6mcRQZB6dp`NE!>$d6kazhu8Mb9Po zt#KmXV3r?9X0&F1I$AH>Q>0X&V!sD`CsK8F2 z_nM&s$Pu*mbQCY11SbTFbt-F36ls0D4&8V6mHjO}y+<#iethKll?dK%5wuJ1r|Ppi z|9X@bLTMWUb&3f4WTn=%Rc{w`@2$cQU!vz|!p8&UOnnPg9B+5=*Z00?-l_E0DEw{e z+jnC7IAI>>>l*Y$|2Bf2`t*^?0R^AImrs1MU}hfl-t^q>%?RsOy951K^P2&fn_GLY zftM-^IS>&03zA1!Y124e#(?vLWeyXDydT<_^Gud78-Qsp)Tu(Z!i5~dE^bVzL190? zB*h(dH^cODVKm-D@&t-qKt%F$;|@tMzbh6z1L-@lt0ilvGa$VHdw5?>l;;OitI&%) zei-(nSw#n(syv~lJsnC1r*rt2b5|H=zu(i0fqmPw+~%Ma)s(?x8h`gNJC9WAvwPPM6=5i}V>nP<31@6TeQL>TJ-9IZ|hYcA^Pkyx&Zg=^%k)EyvT-~ zZt;r}Py1?W0R8oHSxaI!-G!!x&g7j}2SwL2X{LU9|E_=PYA+umFk`g~6pbc)wGMcE zDjrWsm0h0G59i7`71?s^&#+Nj`(u!QCW8~g5?hWqS&eVtq_v`Vawe?v9s8|ATc^UZ!Ogv}h_l}!dVZwk z;cHid8@_50^=!xCmSYVzir?)#eW*AZ_Kw>4q-wzo6QMUBx9~zlN_ecsEFsTR{T9_jyi(@dJ!a6 zB>AbZitTi=un6^madL=?yb>#OA7>?4UV&0S_6Vm`{u^tpQW6gFg9&Vs0=#Q>K)u0~ zf{b`B46Aqvb1C6yc8jHUZ{@=dP=jf$q^yKEGSHtL&P7yVoHH(Yn5utTY#S1XINppg zQR&TnYQlTwZwtg5174Y1tAQWcF@VVGa;}6GW(%ULFTsSBkard+1G!!``Kz%kICbxa9;*I| z_#U$|`>|sb$VSpn>5j*rPMAMl0U?kqQTTd6iBSmKO^!K>=&F)$fIjnscoSCDfRGPs zSOmI(Z-l{*io?V3pUC?UZ68D``)=;XnRvSaE)Y1pV=s*2&-S;!FbI|h+p4e$<=Gq6 zmF{Z?;J5R`RkBq-LSOyr$e$vKpK4It$os z=#Q7r7HqgYS=!7sO^j{w>p1s4oNW%FmM#W6fzvg``r{|UP0-H$t_w*6o}P9~vNl7_ z4B*Vzn%3@=qQ6BcAppgX;%-Ju+v%2SEm}<-vF&}-Kl`||P>4;&kNMvCQRz7f=xpJ} zo}R536P6>$g4nk}3X9MmBUL~FK4v7X7RgZS60W1?Adl0f08Z?|RfW?{FX2XK-L1Xa zkaEe<%Hb6k_5sD4*@YP4q|?*ye|q}eS1!*$+wUQd!)?gUfxlA7@P2pfSLj+95Z5)S zCL_Fl<@Eb${`ff8eak%C<9>}zkpF{wKa6mia`mBJ<%5&i801?|}V@sjY<87(~>!$+$jWOuO% zuYp(?)^Ru^cW1{HXoWsVe%H<{S*_9tDRUl?2blsUi*IibhyNQ=WtGcb}p&R;r zMFcX#sIq8Tze