From bb60b1d0f4b35a51293485e38101396e29248b83 Mon Sep 17 00:00:00 2001 From: Keith Massey Date: Fri, 6 Dec 2024 09:25:17 -0600 Subject: [PATCH 01/27] muting KeystoreManagementTests.test31WrongKeystorePasswordFromFile --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index fb2fea908ef9..9353534a9d83 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -248,6 +248,9 @@ tests: - class: org.elasticsearch.packaging.test.KeystoreManagementTests method: test30KeystorePasswordFromFile issue: https://github.com/elastic/elasticsearch/issues/118123 +- class: org.elasticsearch.packaging.test.KeystoreManagementTests + method: test31WrongKeystorePasswordFromFile + issue: https://github.com/elastic/elasticsearch/issues/118123 - class: org.elasticsearch.packaging.test.ArchiveTests method: test41AutoconfigurationNotTriggeredWhenNodeCannotContainData issue: https://github.com/elastic/elasticsearch/issues/118110 From 3ae2330630e5f5d0fa62e4c41288a6dbae47678f Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 6 Dec 2024 07:33:06 -0800 Subject: [PATCH 02/27] Ignore cancellation exceptions (#117657) Today, when an ES|QL task encounters an exception, we trigger a cancellation on the root task, causing child tasks to fail due to cancellation. We chose not to include cancellation exceptions in the output, as they are unhelpful and add noise during problem analysis. However, these exceptions are still slipping through via RefCountingListener. This change addresses the issue by introducing ESQLRefCountingListener, ensuring that no cancellation exceptions are returned. --- docs/changelog/117657.yaml | 5 ++ .../compute/EsqlRefCountingListener.java | 47 ++++++++++++++++++ .../compute/operator/FailureCollector.java | 48 +++++++++---------- .../exchange/ExchangeSourceHandler.java | 17 +++---- .../operator/FailureCollectorTests.java | 9 ++++ .../xpack/esql/EsqlTestUtils.java | 18 +++++++ .../xpack/esql/action/EnrichIT.java | 1 + .../esql/action/EsqlActionBreakerIT.java | 2 + .../xpack/esql/action/EsqlActionTaskIT.java | 12 ++++- .../xpack/esql/action/EsqlDisruptionIT.java | 2 + .../xpack/esql/plugin/ComputeListener.java | 10 ++-- .../xpack/esql/plugin/ComputeService.java | 6 +-- 12 files changed, 135 insertions(+), 42 deletions(-) create mode 100644 docs/changelog/117657.yaml create mode 100644 x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/EsqlRefCountingListener.java diff --git a/docs/changelog/117657.yaml b/docs/changelog/117657.yaml new file mode 100644 index 000000000000..0a72e9dabe9e --- /dev/null +++ b/docs/changelog/117657.yaml @@ -0,0 +1,5 @@ +pr: 117657 +summary: Ignore cancellation exceptions +area: ES|QL +type: bug +issues: [] diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/EsqlRefCountingListener.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/EsqlRefCountingListener.java new file mode 100644 index 000000000000..69df0fb8ceff --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/EsqlRefCountingListener.java @@ -0,0 +1,47 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.compute; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.RefCountingRunnable; +import org.elasticsearch.compute.operator.FailureCollector; +import org.elasticsearch.core.Releasable; + +/** + * Similar to {@link org.elasticsearch.action.support.RefCountingListener}, + * but prefers non-task-cancelled exceptions over task-cancelled ones as they are more useful for diagnosing issues. + * @see FailureCollector + */ +public final class EsqlRefCountingListener implements Releasable { + private final FailureCollector failureCollector; + private final RefCountingRunnable refs; + + public EsqlRefCountingListener(ActionListener delegate) { + this.failureCollector = new FailureCollector(); + this.refs = new RefCountingRunnable(() -> { + Exception error = failureCollector.getFailure(); + if (error != null) { + delegate.onFailure(error); + } else { + delegate.onResponse(null); + } + }); + } + + public ActionListener acquire() { + return refs.acquireListener().delegateResponse((l, e) -> { + failureCollector.unwrapAndCollect(e); + l.onFailure(e); + }); + } + + @Override + public void close() { + refs.close(); + } +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/FailureCollector.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/FailureCollector.java index 943ba4dc1f4f..337075edbdcf 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/FailureCollector.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/FailureCollector.java @@ -13,9 +13,8 @@ import org.elasticsearch.tasks.TaskCancelledException; import org.elasticsearch.transport.TransportException; -import java.util.List; import java.util.Queue; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.Semaphore; /** * {@code FailureCollector} is responsible for collecting exceptions that occur in the compute engine. @@ -26,12 +25,11 @@ */ public final class FailureCollector { private final Queue cancelledExceptions = ConcurrentCollections.newQueue(); - private final AtomicInteger cancelledExceptionsCount = new AtomicInteger(); + private final Semaphore cancelledExceptionsPermits; private final Queue nonCancelledExceptions = ConcurrentCollections.newQueue(); - private final AtomicInteger nonCancelledExceptionsCount = new AtomicInteger(); + private final Semaphore nonCancelledExceptionsPermits; - private final int maxExceptions; private volatile boolean hasFailure = false; private Exception finalFailure = null; @@ -43,7 +41,8 @@ public FailureCollector(int maxExceptions) { if (maxExceptions <= 0) { throw new IllegalArgumentException("maxExceptions must be at least one"); } - this.maxExceptions = maxExceptions; + this.cancelledExceptionsPermits = new Semaphore(maxExceptions); + this.nonCancelledExceptionsPermits = new Semaphore(maxExceptions); } private static Exception unwrapTransportException(TransportException te) { @@ -60,13 +59,12 @@ private static Exception unwrapTransportException(TransportException te) { public void unwrapAndCollect(Exception e) { e = e instanceof TransportException te ? unwrapTransportException(te) : e; if (ExceptionsHelper.unwrap(e, TaskCancelledException.class) != null) { - if (cancelledExceptionsCount.incrementAndGet() <= maxExceptions) { + if (nonCancelledExceptions.isEmpty() && cancelledExceptionsPermits.tryAcquire()) { cancelledExceptions.add(e); } - } else { - if (nonCancelledExceptionsCount.incrementAndGet() <= maxExceptions) { - nonCancelledExceptions.add(e); - } + } else if (nonCancelledExceptionsPermits.tryAcquire()) { + nonCancelledExceptions.add(e); + cancelledExceptions.clear(); } hasFailure = true; } @@ -99,20 +97,22 @@ public Exception getFailure() { private Exception buildFailure() { assert hasFailure; assert Thread.holdsLock(this); - int total = 0; Exception first = null; - for (var exceptions : List.of(nonCancelledExceptions, cancelledExceptions)) { - for (Exception e : exceptions) { - if (first == null) { - first = e; - total++; - } else if (first != e) { - first.addSuppressed(e); - total++; - } - if (total >= maxExceptions) { - return first; - } + for (Exception e : nonCancelledExceptions) { + if (first == null) { + first = e; + } else if (first != e) { + first.addSuppressed(e); + } + } + if (first != null) { + return first; + } + for (Exception e : cancelledExceptions) { + if (first == null) { + first = e; + } else if (first != e) { + first.addSuppressed(e); } } assert first != null; diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceHandler.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceHandler.java index 375016a5d51d..b53ddea3da58 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceHandler.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceHandler.java @@ -9,9 +9,10 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.support.RefCountingListener; +import org.elasticsearch.action.support.RefCountingRunnable; import org.elasticsearch.action.support.SubscribableListener; import org.elasticsearch.common.util.concurrent.AbstractRunnable; +import org.elasticsearch.compute.EsqlRefCountingListener; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.FailureCollector; import org.elasticsearch.compute.operator.IsBlockedResult; @@ -54,20 +55,20 @@ public ExchangeSourceHandler(int maxBufferSize, Executor fetchExecutor, ActionLi this.outstandingSinks = new PendingInstances(() -> buffer.finish(false)); this.outstandingSources = new PendingInstances(() -> buffer.finish(true)); buffer.addCompletionListener(ActionListener.running(() -> { - final ActionListener listener = ActionListener.assertAtLeastOnce(completionListener).delegateFailure((l, unused) -> { + final ActionListener listener = ActionListener.assertAtLeastOnce(completionListener); + try (RefCountingRunnable refs = new RefCountingRunnable(() -> { final Exception e = failure.getFailure(); if (e != null) { - l.onFailure(e); + listener.onFailure(e); } else { - l.onResponse(null); + listener.onResponse(null); } - }); - try (RefCountingListener refs = new RefCountingListener(listener)) { + })) { for (PendingInstances pending : List.of(outstandingSinks, outstandingSources)) { // Create an outstanding instance and then finish to complete the completionListener // if we haven't registered any instances of exchange sinks or exchange sources before. pending.trackNewInstance(); - pending.completion.addListener(refs.acquire()); + pending.completion.addListener(refs.acquireListener()); pending.finishInstance(); } } @@ -269,7 +270,7 @@ public void onFailure(Exception e) { @Override protected void doRun() { - try (RefCountingListener refs = new RefCountingListener(sinkListener)) { + try (EsqlRefCountingListener refs = new EsqlRefCountingListener(sinkListener)) { for (int i = 0; i < instances; i++) { var fetcher = new RemoteSinkFetcher(remoteSink, failFast, refs.acquire()); fetcher.fetchPage(); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/FailureCollectorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/FailureCollectorTests.java index 637cbe8892b3..5fec82b32dda 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/FailureCollectorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/FailureCollectorTests.java @@ -7,6 +7,7 @@ package org.elasticsearch.compute.operator; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.cluster.node.DiscoveryNodeUtils; import org.elasticsearch.common.Randomness; import org.elasticsearch.common.breaker.CircuitBreaker; @@ -86,6 +87,14 @@ public void testCollect() throws Exception { assertNotNull(failure); assertThat(failure, Matchers.in(nonCancelledExceptions)); assertThat(failure.getSuppressed().length, lessThan(maxExceptions)); + assertTrue( + "cancellation exceptions must be ignored", + ExceptionsHelper.unwrapCausesAndSuppressed(failure, t -> t instanceof TaskCancelledException).isEmpty() + ); + assertTrue( + "remote transport exception must be unwrapped", + ExceptionsHelper.unwrapCausesAndSuppressed(failure, t -> t instanceof TransportException).isEmpty() + ); } public void testEmpty() { diff --git a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java index ec9af33dd669..5535e801b1b0 100644 --- a/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java +++ b/x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java @@ -10,6 +10,7 @@ import org.apache.lucene.document.InetAddressPoint; import org.apache.lucene.sandbox.document.HalfFloatPoint; import org.apache.lucene.util.BytesRef; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.Strings; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.NoopCircuitBreaker; @@ -30,7 +31,9 @@ import org.elasticsearch.geo.ShapeTestUtils; import org.elasticsearch.index.IndexMode; import org.elasticsearch.license.XPackLicenseState; +import org.elasticsearch.tasks.TaskCancelledException; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.transport.RemoteTransportException; import org.elasticsearch.xcontent.json.JsonXContent; import org.elasticsearch.xpack.esql.action.EsqlQueryResponse; import org.elasticsearch.xpack.esql.analysis.EnrichResolution; @@ -129,6 +132,8 @@ import static org.elasticsearch.xpack.esql.parser.ParserUtils.ParamClassification.PATTERN; import static org.elasticsearch.xpack.esql.parser.ParserUtils.ParamClassification.VALUE; import static org.hamcrest.Matchers.instanceOf; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; public final class EsqlTestUtils { @@ -784,4 +789,17 @@ public static QueryParam paramAsIdentifier(String name, Object value) { public static QueryParam paramAsPattern(String name, Object value) { return new QueryParam(name, value, NULL, PATTERN); } + + /** + * Asserts that: + * 1. Cancellation exceptions are ignored when more relevant exceptions exist. + * 2. Transport exceptions are unwrapped, and the actual causes are reported to users. + */ + public static void assertEsqlFailure(Exception e) { + assertNotNull(e); + var cancellationFailure = ExceptionsHelper.unwrapCausesAndSuppressed(e, t -> t instanceof TaskCancelledException).orElse(null); + assertNull("cancellation exceptions must be ignored", cancellationFailure); + ExceptionsHelper.unwrapCausesAndSuppressed(e, t -> t instanceof RemoteTransportException) + .ifPresent(transportFailure -> assertNull("remote transport exception must be unwrapped", transportFailure.getCause())); + } } diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EnrichIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EnrichIT.java index dab99a0f719d..c4da0bf32ef9 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EnrichIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EnrichIT.java @@ -143,6 +143,7 @@ protected EsqlQueryResponse run(EsqlQueryRequest request) { return client.execute(EsqlQueryAction.INSTANCE, request).actionGet(2, TimeUnit.MINUTES); } catch (Exception e) { logger.info("request failed", e); + EsqlTestUtils.assertEsqlFailure(e); ensureBlocksReleased(); } finally { setRequestCircuitBreakerLimit(null); diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionBreakerIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionBreakerIT.java index 37833d8aed2d..ec7ee8b61c2d 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionBreakerIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionBreakerIT.java @@ -23,6 +23,7 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.junit.annotations.TestLogging; +import org.elasticsearch.xpack.esql.EsqlTestUtils; import java.util.ArrayList; import java.util.Collection; @@ -85,6 +86,7 @@ private EsqlQueryResponse runWithBreaking(EsqlQueryRequest request) throws Circu } catch (Exception e) { logger.info("request failed", e); ensureBlocksReleased(); + EsqlTestUtils.assertEsqlFailure(e); throw e; } finally { setRequestCircuitBreakerLimit(null); diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java index 1939f81353c0..abd4f6b49d7b 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlActionTaskIT.java @@ -36,6 +36,7 @@ import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.esql.EsqlTestUtils; import org.elasticsearch.xpack.esql.plugin.QueryPragmas; import org.junit.Before; @@ -338,7 +339,15 @@ private void assertCancelled(ActionFuture response) throws Ex */ assertThat( cancelException.getMessage(), - in(List.of("test cancel", "task cancelled", "request cancelled test cancel", "parent task was cancelled [test cancel]")) + in( + List.of( + "test cancel", + "task cancelled", + "request cancelled test cancel", + "parent task was cancelled [test cancel]", + "cancelled on failure" + ) + ) ); assertBusy( () -> assertThat( @@ -434,6 +443,7 @@ protected void doRun() throws Exception { allowedFetching.countDown(); } Exception failure = expectThrows(Exception.class, () -> future.actionGet().close()); + EsqlTestUtils.assertEsqlFailure(failure); assertThat(failure.getMessage(), containsString("failed to fetch pages")); // If we proceed without waiting for pages, we might cancel the main request before starting the data-node request. // As a result, the exchange sinks on data-nodes won't be removed until the inactive_timeout elapses, which is diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlDisruptionIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlDisruptionIT.java index e9eada5def0d..72a60a6b6b92 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlDisruptionIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlDisruptionIT.java @@ -23,6 +23,7 @@ import org.elasticsearch.test.disruption.ServiceDisruptionScheme; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.transport.TransportSettings; +import org.elasticsearch.xpack.esql.EsqlTestUtils; import java.util.ArrayList; import java.util.Collection; @@ -111,6 +112,7 @@ private EsqlQueryResponse runQueryWithDisruption(EsqlQueryRequest request) { assertTrue("request must be failed or completed after clearing disruption", future.isDone()); ensureBlocksReleased(); logger.info("--> failed to execute esql query with disruption; retrying...", e); + EsqlTestUtils.assertEsqlFailure(e); return client().execute(EsqlQueryAction.INSTANCE, request).actionGet(2, TimeUnit.MINUTES); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeListener.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeListener.java index 8d041ffbdf0e..8bd23230fcde 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeListener.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeListener.java @@ -9,8 +9,8 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.RefCountingListener; +import org.elasticsearch.compute.EsqlRefCountingListener; import org.elasticsearch.compute.operator.DriverProfile; -import org.elasticsearch.compute.operator.FailureCollector; import org.elasticsearch.compute.operator.ResponseHeadersCollector; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.Releasable; @@ -39,8 +39,7 @@ final class ComputeListener implements Releasable { private static final Logger LOGGER = LogManager.getLogger(ComputeService.class); - private final RefCountingListener refs; - private final FailureCollector failureCollector = new FailureCollector(); + private final EsqlRefCountingListener refs; private final AtomicBoolean cancelled = new AtomicBoolean(); private final CancellableTask task; private final TransportService transportService; @@ -105,7 +104,7 @@ private ComputeListener( : "clusterAlias and executionInfo must both be null or both non-null"; // listener that executes after all the sub-listeners refs (created via acquireCompute) have completed - this.refs = new RefCountingListener(1, ActionListener.wrap(ignored -> { + this.refs = new EsqlRefCountingListener(delegate.delegateFailure((l, ignored) -> { responseHeaders.finish(); ComputeResponse result; @@ -131,7 +130,7 @@ private ComputeListener( } } delegate.onResponse(result); - }, e -> delegate.onFailure(failureCollector.getFailure()))); + })); } private static void setFinalStatusAndShardCounts(String clusterAlias, EsqlExecutionInfo executionInfo) { @@ -191,7 +190,6 @@ private boolean isCCSListener(String computeClusterAlias) { */ ActionListener acquireAvoid() { return refs.acquire().delegateResponse((l, e) -> { - failureCollector.unwrapAndCollect(e); try { if (cancelled.compareAndSet(false, true)) { LOGGER.debug("cancelling ESQL task {} on failure", task); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java index ed037d24139f..9b59b98a7cdc 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java @@ -16,11 +16,11 @@ import org.elasticsearch.action.search.SearchShardsRequest; import org.elasticsearch.action.search.SearchShardsResponse; import org.elasticsearch.action.support.ChannelActionListener; -import org.elasticsearch.action.support.RefCountingListener; import org.elasticsearch.action.support.RefCountingRunnable; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.compute.EsqlRefCountingListener; import org.elasticsearch.compute.data.BlockFactory; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.Driver; @@ -375,7 +375,7 @@ private void startComputeOnDataNodes( var lookupListener = ActionListener.releaseAfter(computeListener.acquireAvoid(), exchangeSource.addEmptySink()); // SearchShards API can_match is done in lookupDataNodes lookupDataNodes(parentTask, clusterAlias, requestFilter, concreteIndices, originalIndices, ActionListener.wrap(dataNodeResult -> { - try (RefCountingListener refs = new RefCountingListener(lookupListener)) { + try (EsqlRefCountingListener refs = new EsqlRefCountingListener(lookupListener)) { // update ExecutionInfo with shard counts (total and skipped) executionInfo.swapCluster( clusterAlias, @@ -436,7 +436,7 @@ private void startComputeOnRemoteClusters( ) { var queryPragmas = configuration.pragmas(); var linkExchangeListeners = ActionListener.releaseAfter(computeListener.acquireAvoid(), exchangeSource.addEmptySink()); - try (RefCountingListener refs = new RefCountingListener(linkExchangeListeners)) { + try (EsqlRefCountingListener refs = new EsqlRefCountingListener(linkExchangeListeners)) { for (RemoteCluster cluster : clusters) { final var childSessionId = newChildSession(sessionId); ExchangeService.openExchange( From 4f030efcd50781fe6e624dc08046b834786edb4a Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Sat, 7 Dec 2024 02:37:17 +1100 Subject: [PATCH 03/27] Mute org.elasticsearch.packaging.test.DebPreservationTests test40RestartOnUpgrade #118170 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index 9353534a9d83..8f1030279efc 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -263,6 +263,9 @@ tests: - class: org.elasticsearch.xpack.remotecluster.CrossClusterEsqlRCS2UnavailableRemotesIT method: testEsqlRcs2UnavailableRemoteScenarios issue: https://github.com/elastic/elasticsearch/issues/117419 +- class: org.elasticsearch.packaging.test.DebPreservationTests + method: test40RestartOnUpgrade + issue: https://github.com/elastic/elasticsearch/issues/118170 # Examples: # From 8c380079864c3b98fef872b90102bf1705994f96 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Fri, 6 Dec 2024 12:48:37 -0500 Subject: [PATCH 04/27] ESQL: Rework `isNull` (#118101) This reworks `Expressions#isNull` so it only matches the `null` literal. This doesn't super change the way ESQL works because we already rewrite things that `fold` into `null` into the `null` literal. It's just that, now, `isNull` won't return `true` for things that *fold* to null - only things that have *already* folded to null. This is important because `fold` can be quite expensive so we're better off keeping the results of it when possible. Which is what the constant folding rules *do*. --- .../xpack/esql/core/expression/Expressions.java | 12 ++++++++++-- .../expression/predicate/operator/comparison/In.java | 6 +++--- .../xpack/esql/optimizer/rules/logical/FoldNull.java | 6 +++--- .../esql/optimizer/rules/logical/PruneFilters.java | 8 ++++---- .../rules/logical/SplitInWithFoldableValue.java | 2 +- .../esql/optimizer/LogicalPlanOptimizerTests.java | 2 +- 6 files changed, 22 insertions(+), 14 deletions(-) diff --git a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/Expressions.java b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/Expressions.java index 8baffbf887e4..4e4338aad370 100644 --- a/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/Expressions.java +++ b/x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/Expressions.java @@ -132,8 +132,16 @@ public static String name(Expression e) { return e instanceof NamedExpression ne ? ne.name() : e.sourceText(); } - public static boolean isNull(Expression e) { - return e.dataType() == DataType.NULL || (e.foldable() && e.fold() == null); + /** + * Is this {@linkplain Expression} guaranteed to have + * only the {@code null} value. {@linkplain Expression}s that + * {@link Expression#fold()} to {@code null} may + * return {@code false} here, but should eventually be folded + * into a {@link Literal} containing {@code null} which will return + * {@code true} from here. + */ + public static boolean isGuaranteedNull(Expression e) { + return e.dataType() == DataType.NULL || (e instanceof Literal lit && lit.value() == null); } public static List names(Collection e) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/In.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/In.java index eda6aadccc86..f6c23304c189 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/In.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/predicate/operator/comparison/In.java @@ -151,14 +151,14 @@ public Expression replaceChildren(List newChildren) { public boolean foldable() { // QL's In fold()s to null, if value() is null, but isn't foldable() unless all children are // TODO: update this null check in QL too? - return Expressions.isNull(value) + return Expressions.isGuaranteedNull(value) || Expressions.foldable(children()) - || (Expressions.foldable(list) && list.stream().allMatch(Expressions::isNull)); + || (Expressions.foldable(list) && list.stream().allMatch(Expressions::isGuaranteedNull)); } @Override public Object fold() { - if (Expressions.isNull(value) || list.stream().allMatch(Expressions::isNull)) { + if (Expressions.isGuaranteedNull(value) || list.stream().allMatch(Expressions::isGuaranteedNull)) { return null; } return super.fold(); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/FoldNull.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/FoldNull.java index 638fa1b8db45..4f97bf60bd86 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/FoldNull.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/FoldNull.java @@ -30,7 +30,7 @@ public Expression rule(Expression e) { // perform this early to prevent the rule from converting the null filter into nullifying the whole expression // P.S. this could be done inside the Aggregate but this place better centralizes the logic if (e instanceof AggregateFunction agg) { - if (Expressions.isNull(agg.filter())) { + if (Expressions.isGuaranteedNull(agg.filter())) { return agg.withFilter(Literal.of(agg.filter(), false)); } } @@ -38,13 +38,13 @@ public Expression rule(Expression e) { if (result != e) { return result; } else if (e instanceof In in) { - if (Expressions.isNull(in.value())) { + if (Expressions.isGuaranteedNull(in.value())) { return Literal.of(in, null); } } else if (e instanceof Alias == false && e.nullable() == Nullability.TRUE && e instanceof Categorize == false - && Expressions.anyMatch(e.children(), Expressions::isNull)) { + && Expressions.anyMatch(e.children(), Expressions::isGuaranteedNull)) { return Literal.of(e, null); } return e; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneFilters.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneFilters.java index b6f7ac9e464f..00698d009ea2 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneFilters.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/PruneFilters.java @@ -29,7 +29,7 @@ protected LogicalPlan rule(Filter filter) { if (TRUE.equals(condition)) { return filter.child(); } - if (FALSE.equals(condition) || Expressions.isNull(condition)) { + if (FALSE.equals(condition) || Expressions.isGuaranteedNull(condition)) { return PruneEmptyPlans.skipPlan(filter); } } @@ -42,8 +42,8 @@ protected LogicalPlan rule(Filter filter) { private static Expression foldBinaryLogic(BinaryLogic binaryLogic) { if (binaryLogic instanceof Or or) { - boolean nullLeft = Expressions.isNull(or.left()); - boolean nullRight = Expressions.isNull(or.right()); + boolean nullLeft = Expressions.isGuaranteedNull(or.left()); + boolean nullRight = Expressions.isGuaranteedNull(or.right()); if (nullLeft && nullRight) { return new Literal(binaryLogic.source(), null, DataType.NULL); } @@ -55,7 +55,7 @@ private static Expression foldBinaryLogic(BinaryLogic binaryLogic) { } } if (binaryLogic instanceof And and) { - if (Expressions.isNull(and.left()) || Expressions.isNull(and.right())) { + if (Expressions.isGuaranteedNull(and.left()) || Expressions.isGuaranteedNull(and.right())) { return new Literal(binaryLogic.source(), null, DataType.NULL); } } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SplitInWithFoldableValue.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SplitInWithFoldableValue.java index 930b485dbd37..9e9ae6a9a559 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SplitInWithFoldableValue.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/optimizer/rules/logical/SplitInWithFoldableValue.java @@ -30,7 +30,7 @@ public Expression rule(In in) { List foldables = new ArrayList<>(in.list().size()); List nonFoldables = new ArrayList<>(in.list().size()); in.list().forEach(e -> { - if (e.foldable() && Expressions.isNull(e) == false) { // keep `null`s, needed for the 3VL + if (e.foldable() && Expressions.isGuaranteedNull(e) == false) { // keep `null`s, needed for the 3VL foldables.add(e); } else { nonFoldables.add(e); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java index b76781f76f4a..c2a26845d4e8 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/optimizer/LogicalPlanOptimizerTests.java @@ -4820,7 +4820,7 @@ private static boolean oneLeaveIsNull(Expression e) { e.forEachUp(node -> { if (node.children().size() == 0) { - result.set(result.get() || Expressions.isNull(node)); + result.set(result.get() || Expressions.isGuaranteedNull(node)); } }); From ab6fcc4e749ae575e6505498d18de19ce3db4ca2 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Fri, 6 Dec 2024 18:50:04 +0100 Subject: [PATCH 05/27] Fix mocking in SyntheticSourceLicenseServiceTests (#118155) Some mock verifies where missing and `LicenseState#copyCurrentLicenseState(...)` wasn't always mocked. And because of incorrect mocking the testGoldOrPlatinumLicenseCustomCutoffDate() test had an incorrect assertion. --- .../SyntheticSourceLicenseServiceTests.java | 22 ++++++++++++++----- 1 file changed, 17 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/logsdb/src/test/java/org/elasticsearch/xpack/logsdb/SyntheticSourceLicenseServiceTests.java b/x-pack/plugin/logsdb/src/test/java/org/elasticsearch/xpack/logsdb/SyntheticSourceLicenseServiceTests.java index 90a13b16c028..0eb0d21ff2e7 100644 --- a/x-pack/plugin/logsdb/src/test/java/org/elasticsearch/xpack/logsdb/SyntheticSourceLicenseServiceTests.java +++ b/x-pack/plugin/logsdb/src/test/java/org/elasticsearch/xpack/logsdb/SyntheticSourceLicenseServiceTests.java @@ -41,6 +41,7 @@ public void setup() throws Exception { public void testLicenseAllowsSyntheticSource() { MockLicenseState licenseState = MockLicenseState.createMock(); + when(licenseState.copyCurrentLicenseState()).thenReturn(licenseState); when(licenseState.isAllowed(same(SyntheticSourceLicenseService.SYNTHETIC_SOURCE_FEATURE))).thenReturn(true); licenseService.setLicenseState(licenseState); licenseService.setLicenseService(mockLicenseService); @@ -53,6 +54,7 @@ public void testLicenseAllowsSyntheticSource() { public void testLicenseAllowsSyntheticSourceTemplateValidation() { MockLicenseState licenseState = MockLicenseState.createMock(); + when(licenseState.copyCurrentLicenseState()).thenReturn(licenseState); when(licenseState.isAllowed(same(SyntheticSourceLicenseService.SYNTHETIC_SOURCE_FEATURE))).thenReturn(true); licenseService.setLicenseState(licenseState); licenseService.setLicenseService(mockLicenseService); @@ -65,6 +67,7 @@ public void testLicenseAllowsSyntheticSourceTemplateValidation() { public void testDefaultDisallow() { MockLicenseState licenseState = MockLicenseState.createMock(); + when(licenseState.copyCurrentLicenseState()).thenReturn(licenseState); when(licenseState.isAllowed(same(SyntheticSourceLicenseService.SYNTHETIC_SOURCE_FEATURE))).thenReturn(false); licenseService.setLicenseState(licenseState); licenseService.setLicenseService(mockLicenseService); @@ -77,6 +80,7 @@ public void testDefaultDisallow() { public void testFallback() { MockLicenseState licenseState = MockLicenseState.createMock(); + when(licenseState.copyCurrentLicenseState()).thenReturn(licenseState); when(licenseState.isAllowed(same(SyntheticSourceLicenseService.SYNTHETIC_SOURCE_FEATURE))).thenReturn(true); licenseService.setLicenseState(licenseState); licenseService.setLicenseService(mockLicenseService); @@ -95,6 +99,7 @@ public void testGoldOrPlatinumLicense() throws Exception { when(mockLicenseService.getLicense()).thenReturn(license); MockLicenseState licenseState = MockLicenseState.createMock(); + when(licenseState.copyCurrentLicenseState()).thenReturn(licenseState); when(licenseState.getOperationMode()).thenReturn(license.operationMode()); when(licenseState.isAllowed(same(SyntheticSourceLicenseService.SYNTHETIC_SOURCE_FEATURE_LEGACY))).thenReturn(true); licenseService.setLicenseState(licenseState); @@ -103,6 +108,8 @@ public void testGoldOrPlatinumLicense() throws Exception { "legacy licensed usage is allowed, so not fallback to stored source", licenseService.fallbackToStoredSource(false, true) ); + Mockito.verify(licenseState, Mockito.times(1)).isAllowed(same(SyntheticSourceLicenseService.SYNTHETIC_SOURCE_FEATURE)); + Mockito.verify(licenseState, Mockito.times(1)).isAllowed(same(SyntheticSourceLicenseService.SYNTHETIC_SOURCE_FEATURE_LEGACY)); Mockito.verify(licenseState, Mockito.times(1)).featureUsed(any()); } @@ -112,6 +119,7 @@ public void testGoldOrPlatinumLicenseLegacyLicenseNotAllowed() throws Exception when(mockLicenseService.getLicense()).thenReturn(license); MockLicenseState licenseState = MockLicenseState.createMock(); + when(licenseState.copyCurrentLicenseState()).thenReturn(licenseState); when(licenseState.getOperationMode()).thenReturn(license.operationMode()); when(licenseState.isAllowed(same(SyntheticSourceLicenseService.SYNTHETIC_SOURCE_FEATURE))).thenReturn(false); licenseService.setLicenseState(licenseState); @@ -125,14 +133,16 @@ public void testGoldOrPlatinumLicenseLegacyLicenseNotAllowed() throws Exception } public void testGoldOrPlatinumLicenseBeyondCutoffDate() throws Exception { - long start = LocalDateTime.of(2025, 1, 1, 0, 0).toInstant(ZoneOffset.UTC).toEpochMilli(); + long start = LocalDateTime.of(2025, 2, 5, 0, 0).toInstant(ZoneOffset.UTC).toEpochMilli(); License license = createGoldOrPlatinumLicense(start); mockLicenseService = mock(LicenseService.class); when(mockLicenseService.getLicense()).thenReturn(license); MockLicenseState licenseState = MockLicenseState.createMock(); + when(licenseState.copyCurrentLicenseState()).thenReturn(licenseState); when(licenseState.getOperationMode()).thenReturn(license.operationMode()); when(licenseState.isAllowed(same(SyntheticSourceLicenseService.SYNTHETIC_SOURCE_FEATURE))).thenReturn(false); + when(licenseState.isAllowed(same(SyntheticSourceLicenseService.SYNTHETIC_SOURCE_FEATURE_LEGACY))).thenReturn(true); licenseService.setLicenseState(licenseState); licenseService.setLicenseService(mockLicenseService); assertTrue("beyond cutoff date, so fallback to stored source", licenseService.fallbackToStoredSource(false, true)); @@ -143,19 +153,21 @@ public void testGoldOrPlatinumLicenseBeyondCutoffDate() throws Exception { public void testGoldOrPlatinumLicenseCustomCutoffDate() throws Exception { licenseService = new SyntheticSourceLicenseService(Settings.EMPTY, "2025-01-02T00:00"); - long start = LocalDateTime.of(2025, 1, 1, 0, 0).toInstant(ZoneOffset.UTC).toEpochMilli(); + long start = LocalDateTime.of(2025, 1, 3, 0, 0).toInstant(ZoneOffset.UTC).toEpochMilli(); License license = createGoldOrPlatinumLicense(start); mockLicenseService = mock(LicenseService.class); when(mockLicenseService.getLicense()).thenReturn(license); MockLicenseState licenseState = MockLicenseState.createMock(); + when(licenseState.copyCurrentLicenseState()).thenReturn(licenseState); when(licenseState.getOperationMode()).thenReturn(license.operationMode()); + when(licenseState.isAllowed(same(SyntheticSourceLicenseService.SYNTHETIC_SOURCE_FEATURE))).thenReturn(false); when(licenseState.isAllowed(same(SyntheticSourceLicenseService.SYNTHETIC_SOURCE_FEATURE_LEGACY))).thenReturn(true); licenseService.setLicenseState(licenseState); licenseService.setLicenseService(mockLicenseService); - assertFalse("custom cutoff date, so fallback to stored source", licenseService.fallbackToStoredSource(false, true)); - Mockito.verify(licenseState, Mockito.times(1)).featureUsed(any()); - Mockito.verify(licenseState, Mockito.times(1)).isAllowed(same(SyntheticSourceLicenseService.SYNTHETIC_SOURCE_FEATURE_LEGACY)); + assertTrue("custom cutoff date, so fallback to stored source", licenseService.fallbackToStoredSource(false, true)); + Mockito.verify(licenseState, Mockito.times(1)).isAllowed(same(SyntheticSourceLicenseService.SYNTHETIC_SOURCE_FEATURE)); + Mockito.verify(licenseState, Mockito.never()).featureUsed(any()); } static License createEnterpriseLicense() throws Exception { From 27aac9654d0b226ac5988f635858ae6404a75cda Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Sat, 7 Dec 2024 05:19:17 +1100 Subject: [PATCH 06/27] Mute org.elasticsearch.xpack.inference.DefaultEndPointsIT testInferDeploysDefaultRerank #118184 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index 8f1030279efc..887b462fa122 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -266,6 +266,9 @@ tests: - class: org.elasticsearch.packaging.test.DebPreservationTests method: test40RestartOnUpgrade issue: https://github.com/elastic/elasticsearch/issues/118170 +- class: org.elasticsearch.xpack.inference.DefaultEndPointsIT + method: testInferDeploysDefaultRerank + issue: https://github.com/elastic/elasticsearch/issues/118184 # Examples: # From c580024ea92fd561089380eb4078f576c47b72b9 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Fri, 6 Dec 2024 18:42:50 +0000 Subject: [PATCH 07/27] Add Highlighter for Semantic Text Fields (#118064) This PR introduces a new highlighter, `semantic`, tailored for semantic text fields. It extracts the most relevant fragments by scoring nested chunks using the original semantic query. In this initial version, the highlighter returns only the original chunks computed during ingestion. However, this is an implementation detail, and future enhancements could combine multiple chunks to generate the fragments. --- docs/changelog/118064.yaml | 5 + .../mapping/types/semantic-text.asciidoc | 55 +-- .../xpack/inference/InferenceFeatures.java | 5 +- .../xpack/inference/InferencePlugin.java | 7 + .../highlight/SemanticTextHighlighter.java | 226 +++++++++ .../inference/mapper/SemanticTextField.java | 2 +- .../mapper/SemanticTextFieldMapper.java | 20 +- .../SemanticTextHighlighterTests.java | 288 +++++++++++ .../mapper/SemanticTextFieldMapperTests.java | 8 +- .../queries/SemanticQueryBuilderTests.java | 10 +- .../xpack/inference/highlight/mappings.json | 27 + .../xpack/inference/highlight/queries.json | 467 ++++++++++++++++++ .../inference/highlight/sample-doc.json.gz | Bin 0 -> 388098 bytes .../90_semantic_text_highlighter.yml | 242 +++++++++ 14 files changed, 1314 insertions(+), 48 deletions(-) create mode 100644 docs/changelog/118064.yaml create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/highlight/SemanticTextHighlighter.java create mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/highlight/SemanticTextHighlighterTests.java create mode 100644 x-pack/plugin/inference/src/test/resources/org/elasticsearch/xpack/inference/highlight/mappings.json create mode 100644 x-pack/plugin/inference/src/test/resources/org/elasticsearch/xpack/inference/highlight/queries.json create mode 100644 x-pack/plugin/inference/src/test/resources/org/elasticsearch/xpack/inference/highlight/sample-doc.json.gz create mode 100644 x-pack/plugin/inference/src/yamlRestTest/resources/rest-api-spec/test/inference/90_semantic_text_highlighter.yml diff --git a/docs/changelog/118064.yaml b/docs/changelog/118064.yaml new file mode 100644 index 000000000000..7d12f365bf14 --- /dev/null +++ b/docs/changelog/118064.yaml @@ -0,0 +1,5 @@ +pr: 118064 +summary: Add Highlighter for Semantic Text Fields +area: Highlighting +type: feature +issues: [] diff --git a/docs/reference/mapping/types/semantic-text.asciidoc b/docs/reference/mapping/types/semantic-text.asciidoc index f76a9352c2fe..b3e103ec6dbd 100644 --- a/docs/reference/mapping/types/semantic-text.asciidoc +++ b/docs/reference/mapping/types/semantic-text.asciidoc @@ -112,50 +112,43 @@ Trying to <> that is used on a {infer-cap} endpoints have a limit on the amount of text they can process. To allow for large amounts of text to be used in semantic search, `semantic_text` automatically generates smaller passages if needed, called _chunks_. -Each chunk will include the text subpassage and the corresponding embedding generated from it. +Each chunk refers to a passage of the text and the corresponding embedding generated from it. When querying, the individual passages will be automatically searched for each document, and the most relevant passage will be used to compute a score. For more details on chunking and how to configure chunking settings, see <> in the Inference API documentation. +Refer to <> to learn more about +semantic search using `semantic_text` and the `semantic` query. [discrete] -[[semantic-text-structure]] -==== `semantic_text` structure +[[semantic-text-highlighting]] +==== Extracting Relevant Fragments from Semantic Text -Once a document is ingested, a `semantic_text` field will have the following structure: +You can extract the most relevant fragments from a semantic text field by using the <> in the <>. -[source,console-result] +[source,console] ------------------------------------------------------------ -"inference_field": { - "text": "these are not the droids you're looking for", <1> - "inference": { - "inference_id": "my-elser-endpoint", <2> - "model_settings": { <3> - "task_type": "sparse_embedding" +PUT test-index +{ + "query": { + "semantic": { + "field": "my_semantic_field" + } }, - "chunks": [ <4> - { - "text": "these are not the droids you're looking for", - "embeddings": { - (...) + "highlight": { + "fields": { + "my_semantic_field": { + "type": "semantic", + "number_of_fragments": 2, <1> + "order": "score" <2> + } } - } - ] - } + } } ------------------------------------------------------------ -// TEST[skip:TBD] -<1> The field will become an object structure to accommodate both the original -text and the inference results. -<2> The `inference_id` used to generate the embeddings. -<3> Model settings, including the task type and dimensions/similarity if -applicable. -<4> Inference results will be grouped in chunks, each with its corresponding -text and embeddings. - -Refer to <> to learn more about -semantic search using `semantic_text` and the `semantic` query. - +// TEST[skip:Requires inference endpoint] +<1> Specifies the maximum number of fragments to return. +<2> Sorts highlighted fragments by score when set to `score`. By default, fragments will be output in the order they appear in the field (order: none). [discrete] [[custom-indexing]] diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceFeatures.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceFeatures.java index c82f287792a7..67892dfe7862 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceFeatures.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceFeatures.java @@ -33,6 +33,8 @@ public Set getFeatures() { ); } + private static final NodeFeature SEMANTIC_TEXT_HIGHLIGHTER = new NodeFeature("semantic_text.highlighter"); + @Override public Set getTestFeatures() { return Set.of( @@ -40,7 +42,8 @@ public Set getTestFeatures() { SemanticTextFieldMapper.SEMANTIC_TEXT_SINGLE_FIELD_UPDATE_FIX, SemanticTextFieldMapper.SEMANTIC_TEXT_DELETE_FIX, SemanticTextFieldMapper.SEMANTIC_TEXT_ZERO_SIZE_FIX, - SemanticTextFieldMapper.SEMANTIC_TEXT_ALWAYS_EMIT_INFERENCE_ID_FIX + SemanticTextFieldMapper.SEMANTIC_TEXT_ALWAYS_EMIT_INFERENCE_ID_FIX, + SEMANTIC_TEXT_HIGHLIGHTER ); } } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferencePlugin.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferencePlugin.java index 3c14e51a3c2d..d7d623ab2014 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferencePlugin.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferencePlugin.java @@ -37,6 +37,7 @@ import org.elasticsearch.plugins.SystemIndexPlugin; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestHandler; +import org.elasticsearch.search.fetch.subphase.highlight.Highlighter; import org.elasticsearch.search.rank.RankBuilder; import org.elasticsearch.search.rank.RankDoc; import org.elasticsearch.threadpool.ExecutorBuilder; @@ -67,6 +68,7 @@ import org.elasticsearch.xpack.inference.external.http.retry.RetrySettings; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; import org.elasticsearch.xpack.inference.external.http.sender.RequestExecutorServiceSettings; +import org.elasticsearch.xpack.inference.highlight.SemanticTextHighlighter; import org.elasticsearch.xpack.inference.logging.ThrottlerManager; import org.elasticsearch.xpack.inference.mapper.OffsetSourceFieldMapper; import org.elasticsearch.xpack.inference.mapper.SemanticTextFieldMapper; @@ -417,4 +419,9 @@ public List> getRetrievers() { new RetrieverSpec<>(new ParseField(RandomRankBuilder.NAME), RandomRankRetrieverBuilder::fromXContent) ); } + + @Override + public Map getHighlighters() { + return Map.of(SemanticTextHighlighter.NAME, new SemanticTextHighlighter()); + } } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/highlight/SemanticTextHighlighter.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/highlight/SemanticTextHighlighter.java new file mode 100644 index 000000000000..f2bfa72ec617 --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/highlight/SemanticTextHighlighter.java @@ -0,0 +1,226 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference.highlight; + +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.BooleanClause; +import org.apache.lucene.search.BooleanQuery; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.KnnByteVectorQuery; +import org.apache.lucene.search.KnnFloatVectorQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.QueryVisitor; +import org.apache.lucene.search.ScoreMode; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.Weight; +import org.elasticsearch.common.text.Text; +import org.elasticsearch.common.xcontent.support.XContentMapValues; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.vectors.DenseVectorFieldMapper.DenseVectorFieldType; +import org.elasticsearch.index.mapper.vectors.SparseVectorFieldMapper.SparseVectorFieldType; +import org.elasticsearch.index.query.SearchExecutionContext; +import org.elasticsearch.search.fetch.subphase.highlight.FieldHighlightContext; +import org.elasticsearch.search.fetch.subphase.highlight.HighlightField; +import org.elasticsearch.search.fetch.subphase.highlight.Highlighter; +import org.elasticsearch.search.vectors.VectorData; +import org.elasticsearch.xpack.core.ml.search.SparseVectorQueryWrapper; +import org.elasticsearch.xpack.inference.mapper.SemanticTextField; +import org.elasticsearch.xpack.inference.mapper.SemanticTextFieldMapper; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +/** + * A {@link Highlighter} designed for the {@link SemanticTextFieldMapper}. + * This highlighter extracts semantic queries and evaluates them against each chunk produced by the semantic text field. + * It returns the top-scoring chunks as snippets, optionally sorted by their scores. + */ +public class SemanticTextHighlighter implements Highlighter { + public static final String NAME = "semantic"; + + private record OffsetAndScore(int offset, float score) {} + + @Override + public boolean canHighlight(MappedFieldType fieldType) { + if (fieldType instanceof SemanticTextFieldMapper.SemanticTextFieldType) { + return true; + } + return false; + } + + @Override + public HighlightField highlight(FieldHighlightContext fieldContext) throws IOException { + SemanticTextFieldMapper.SemanticTextFieldType fieldType = (SemanticTextFieldMapper.SemanticTextFieldType) fieldContext.fieldType; + if (fieldType.getEmbeddingsField() == null) { + // nothing indexed yet + return null; + } + + final List queries = switch (fieldType.getModelSettings().taskType()) { + case SPARSE_EMBEDDING -> extractSparseVectorQueries( + (SparseVectorFieldType) fieldType.getEmbeddingsField().fieldType(), + fieldContext.query + ); + case TEXT_EMBEDDING -> extractDenseVectorQueries( + (DenseVectorFieldType) fieldType.getEmbeddingsField().fieldType(), + fieldContext.query + ); + default -> throw new IllegalStateException( + "Wrong task type for a semantic text field, got [" + fieldType.getModelSettings().taskType().name() + "]" + ); + }; + if (queries.isEmpty()) { + // nothing to highlight + return null; + } + + int numberOfFragments = fieldContext.field.fieldOptions().numberOfFragments() <= 0 + ? 1 // we return the best fragment by default + : fieldContext.field.fieldOptions().numberOfFragments(); + + List chunks = extractOffsetAndScores( + fieldContext.context.getSearchExecutionContext(), + fieldContext.hitContext.reader(), + fieldType, + fieldContext.hitContext.docId(), + queries + ); + if (chunks.size() == 0) { + return null; + } + + chunks.sort(Comparator.comparingDouble(OffsetAndScore::score).reversed()); + int size = Math.min(chunks.size(), numberOfFragments); + if (fieldContext.field.fieldOptions().scoreOrdered() == false) { + chunks = chunks.subList(0, size); + chunks.sort(Comparator.comparingInt(c -> c.offset)); + } + Text[] snippets = new Text[size]; + List> nestedSources = XContentMapValues.extractNestedSources( + fieldType.getChunksField().fullPath(), + fieldContext.hitContext.source().source() + ); + for (int i = 0; i < size; i++) { + var chunk = chunks.get(i); + if (nestedSources.size() <= chunk.offset) { + throw new IllegalStateException( + String.format( + Locale.ROOT, + "Invalid content detected for field [%s]: the chunks size is [%d], " + + "but a reference to offset [%d] was found in the result.", + fieldType.name(), + nestedSources.size(), + chunk.offset + ) + ); + } + String content = (String) nestedSources.get(chunk.offset).get(SemanticTextField.CHUNKED_TEXT_FIELD); + if (content == null) { + throw new IllegalStateException( + String.format( + Locale.ROOT, + + "Invalid content detected for field [%s]: missing text for the chunk at offset [%d].", + fieldType.name(), + chunk.offset + ) + ); + } + snippets[i] = new Text(content); + } + return new HighlightField(fieldContext.fieldName, snippets); + } + + private List extractOffsetAndScores( + SearchExecutionContext context, + LeafReader reader, + SemanticTextFieldMapper.SemanticTextFieldType fieldType, + int docId, + List leafQueries + ) throws IOException { + var bitSet = context.bitsetFilter(fieldType.getChunksField().parentTypeFilter()).getBitSet(reader.getContext()); + int previousParent = docId > 0 ? bitSet.prevSetBit(docId - 1) : -1; + + BooleanQuery.Builder bq = new BooleanQuery.Builder().add(fieldType.getChunksField().nestedTypeFilter(), BooleanClause.Occur.FILTER); + leafQueries.stream().forEach(q -> bq.add(q, BooleanClause.Occur.SHOULD)); + Weight weight = new IndexSearcher(reader).createWeight(bq.build(), ScoreMode.COMPLETE, 1); + Scorer scorer = weight.scorer(reader.getContext()); + if (previousParent != -1) { + if (scorer.iterator().advance(previousParent) == DocIdSetIterator.NO_MORE_DOCS) { + return List.of(); + } + } else if (scorer.iterator().nextDoc() == DocIdSetIterator.NO_MORE_DOCS) { + return List.of(); + } + List results = new ArrayList<>(); + int offset = 0; + while (scorer.docID() < docId) { + results.add(new OffsetAndScore(offset++, scorer.score())); + if (scorer.iterator().nextDoc() == DocIdSetIterator.NO_MORE_DOCS) { + break; + } + } + return results; + } + + private List extractDenseVectorQueries(DenseVectorFieldType fieldType, Query querySection) { + // TODO: Handle knn section when semantic text field can be used. + List queries = new ArrayList<>(); + querySection.visit(new QueryVisitor() { + @Override + public boolean acceptField(String field) { + return fieldType.name().equals(field); + } + + @Override + public void consumeTerms(Query query, Term... terms) { + super.consumeTerms(query, terms); + } + + @Override + public void visitLeaf(Query query) { + if (query instanceof KnnFloatVectorQuery knnQuery) { + queries.add(fieldType.createExactKnnQuery(VectorData.fromFloats(knnQuery.getTargetCopy()), null)); + } else if (query instanceof KnnByteVectorQuery knnQuery) { + queries.add(fieldType.createExactKnnQuery(VectorData.fromBytes(knnQuery.getTargetCopy()), null)); + } + } + }); + return queries; + } + + private List extractSparseVectorQueries(SparseVectorFieldType fieldType, Query querySection) { + List queries = new ArrayList<>(); + querySection.visit(new QueryVisitor() { + @Override + public boolean acceptField(String field) { + return fieldType.name().equals(field); + } + + @Override + public void consumeTerms(Query query, Term... terms) { + super.consumeTerms(query, terms); + } + + @Override + public QueryVisitor getSubVisitor(BooleanClause.Occur occur, Query parent) { + if (parent instanceof SparseVectorQueryWrapper sparseVectorQuery) { + queries.add(sparseVectorQuery.getTermsQuery()); + } + return this; + } + }); + return queries; + } +} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/mapper/SemanticTextField.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/mapper/SemanticTextField.java index e60e95b58770..0f26f6577860 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/mapper/SemanticTextField.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/mapper/SemanticTextField.java @@ -61,7 +61,7 @@ public record SemanticTextField(String fieldName, List originalValues, I static final String SEARCH_INFERENCE_ID_FIELD = "search_inference_id"; static final String CHUNKS_FIELD = "chunks"; static final String CHUNKED_EMBEDDINGS_FIELD = "embeddings"; - static final String CHUNKED_TEXT_FIELD = "text"; + public static final String CHUNKED_TEXT_FIELD = "text"; static final String MODEL_SETTINGS_FIELD = "model_settings"; static final String TASK_TYPE_FIELD = "task_type"; static final String DIMENSIONS_FIELD = "dimensions"; diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/mapper/SemanticTextFieldMapper.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/mapper/SemanticTextFieldMapper.java index 3744bf2a6dbe..683bb5a53028 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/mapper/SemanticTextFieldMapper.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/mapper/SemanticTextFieldMapper.java @@ -46,7 +46,6 @@ import org.elasticsearch.index.query.MatchNoneQueryBuilder; import org.elasticsearch.index.query.NestedQueryBuilder; import org.elasticsearch.index.query.QueryBuilder; -import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.SearchExecutionContext; import org.elasticsearch.inference.InferenceResults; import org.elasticsearch.inference.SimilarityMeasure; @@ -57,6 +56,7 @@ import org.elasticsearch.xcontent.XContentParserConfiguration; import org.elasticsearch.xpack.core.ml.inference.results.MlTextEmbeddingResults; import org.elasticsearch.xpack.core.ml.inference.results.TextExpansionResults; +import org.elasticsearch.xpack.core.ml.search.SparseVectorQueryBuilder; import java.io.IOException; import java.util.ArrayList; @@ -529,17 +529,15 @@ public QueryBuilder semanticQuery(InferenceResults inferenceResults, Integer req ); } - // TODO: Use WeightedTokensQueryBuilder TextExpansionResults textExpansionResults = (TextExpansionResults) inferenceResults; - var boolQuery = QueryBuilders.boolQuery(); - for (var weightedToken : textExpansionResults.getWeightedTokens()) { - boolQuery.should( - QueryBuilders.termQuery(inferenceResultsFieldName, weightedToken.token()).boost(weightedToken.weight()) - ); - } - boolQuery.minimumShouldMatch(1); - - yield boolQuery; + yield new SparseVectorQueryBuilder( + inferenceResultsFieldName, + textExpansionResults.getWeightedTokens(), + null, + null, + null, + null + ); } case TEXT_EMBEDDING -> { if (inferenceResults instanceof MlTextEmbeddingResults == false) { diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/highlight/SemanticTextHighlighterTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/highlight/SemanticTextHighlighterTests.java new file mode 100644 index 000000000000..7dc4d99e06ac --- /dev/null +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/highlight/SemanticTextHighlighterTests.java @@ -0,0 +1,288 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference.highlight; + +import org.apache.lucene.analysis.standard.StandardAnalyzer; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Sort; +import org.apache.lucene.search.TopDocs; +import org.apache.lucene.search.join.ScoreMode; +import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.index.RandomIndexWriter; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.action.OriginalIndices; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.Streams; +import org.elasticsearch.common.lucene.search.Queries; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.index.IndexVersion; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.mapper.MapperServiceTestCase; +import org.elasticsearch.index.mapper.SourceToParse; +import org.elasticsearch.index.query.NestedQueryBuilder; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.fetch.FetchContext; +import org.elasticsearch.search.fetch.FetchSubPhase; +import org.elasticsearch.search.fetch.subphase.highlight.FieldHighlightContext; +import org.elasticsearch.search.fetch.subphase.highlight.HighlightBuilder; +import org.elasticsearch.search.fetch.subphase.highlight.SearchHighlightContext; +import org.elasticsearch.search.internal.AliasFilter; +import org.elasticsearch.search.internal.ShardSearchRequest; +import org.elasticsearch.search.lookup.Source; +import org.elasticsearch.search.rank.RankDoc; +import org.elasticsearch.search.vectors.KnnVectorQueryBuilder; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.core.ml.search.SparseVectorQueryBuilder; +import org.elasticsearch.xpack.core.ml.search.WeightedToken; +import org.elasticsearch.xpack.inference.InferencePlugin; +import org.elasticsearch.xpack.inference.mapper.SemanticTextFieldMapper; +import org.junit.Before; +import org.mockito.Mockito; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.zip.GZIPInputStream; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.mockito.Mockito.mock; + +public class SemanticTextHighlighterTests extends MapperServiceTestCase { + private static final String SEMANTIC_FIELD_E5 = "body-e5"; + private static final String SEMANTIC_FIELD_ELSER = "body-elser"; + + private Map queries; + + @Override + protected Collection getPlugins() { + return List.of(new InferencePlugin(Settings.EMPTY)); + } + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + var input = Streams.readFully(SemanticTextHighlighterTests.class.getResourceAsStream("queries.json")); + this.queries = XContentHelper.convertToMap(input, false, XContentType.JSON).v2(); + } + + @SuppressWarnings("unchecked") + public void testDenseVector() throws Exception { + var mapperService = createDefaultMapperService(); + Map queryMap = (Map) queries.get("dense_vector_1"); + float[] vector = readDenseVector(queryMap.get("embeddings")); + var fieldType = (SemanticTextFieldMapper.SemanticTextFieldType) mapperService.mappingLookup().getFieldType(SEMANTIC_FIELD_E5); + KnnVectorQueryBuilder knnQuery = new KnnVectorQueryBuilder(fieldType.getEmbeddingsField().fullPath(), vector, 10, 10, null); + NestedQueryBuilder nestedQueryBuilder = new NestedQueryBuilder(fieldType.getChunksField().fullPath(), knnQuery, ScoreMode.Max); + var shardRequest = createShardSearchRequest(nestedQueryBuilder); + var sourceToParse = new SourceToParse("0", readSampleDoc("sample-doc.json.gz"), XContentType.JSON); + + String[] expectedScorePassages = ((List) queryMap.get("expected_by_score")).toArray(String[]::new); + for (int i = 0; i < expectedScorePassages.length; i++) { + assertHighlightOneDoc( + mapperService, + shardRequest, + sourceToParse, + SEMANTIC_FIELD_E5, + i + 1, + HighlightBuilder.Order.SCORE, + Arrays.copyOfRange(expectedScorePassages, 0, i + 1) + ); + } + + String[] expectedOffsetPassages = ((List) queryMap.get("expected_by_offset")).toArray(String[]::new); + assertHighlightOneDoc( + mapperService, + shardRequest, + sourceToParse, + SEMANTIC_FIELD_E5, + expectedOffsetPassages.length, + HighlightBuilder.Order.NONE, + expectedOffsetPassages + ); + } + + @SuppressWarnings("unchecked") + public void testSparseVector() throws Exception { + var mapperService = createDefaultMapperService(); + Map queryMap = (Map) queries.get("sparse_vector_1"); + List tokens = readSparseVector(queryMap.get("embeddings")); + var fieldType = (SemanticTextFieldMapper.SemanticTextFieldType) mapperService.mappingLookup().getFieldType(SEMANTIC_FIELD_ELSER); + SparseVectorQueryBuilder sparseQuery = new SparseVectorQueryBuilder( + fieldType.getEmbeddingsField().fullPath(), + tokens, + null, + null, + null, + null + ); + NestedQueryBuilder nestedQueryBuilder = new NestedQueryBuilder(fieldType.getChunksField().fullPath(), sparseQuery, ScoreMode.Max); + var shardRequest = createShardSearchRequest(nestedQueryBuilder); + var sourceToParse = new SourceToParse("0", readSampleDoc("sample-doc.json.gz"), XContentType.JSON); + + String[] expectedScorePassages = ((List) queryMap.get("expected_by_score")).toArray(String[]::new); + for (int i = 0; i < expectedScorePassages.length; i++) { + assertHighlightOneDoc( + mapperService, + shardRequest, + sourceToParse, + SEMANTIC_FIELD_ELSER, + i + 1, + HighlightBuilder.Order.SCORE, + Arrays.copyOfRange(expectedScorePassages, 0, i + 1) + ); + } + + String[] expectedOffsetPassages = ((List) queryMap.get("expected_by_offset")).toArray(String[]::new); + assertHighlightOneDoc( + mapperService, + shardRequest, + sourceToParse, + SEMANTIC_FIELD_ELSER, + expectedOffsetPassages.length, + HighlightBuilder.Order.NONE, + expectedOffsetPassages + ); + } + + private MapperService createDefaultMapperService() throws IOException { + var mappings = Streams.readFully(SemanticTextHighlighterTests.class.getResourceAsStream("mappings.json")); + return createMapperService(mappings.utf8ToString()); + } + + private float[] readDenseVector(Object value) { + if (value instanceof List lst) { + float[] res = new float[lst.size()]; + int pos = 0; + for (var obj : lst) { + if (obj instanceof Number number) { + res[pos++] = number.floatValue(); + } else { + throw new IllegalArgumentException("Expected number, got " + obj.getClass().getSimpleName()); + } + } + return res; + } + throw new IllegalArgumentException("Expected list, got " + value.getClass().getSimpleName()); + } + + private List readSparseVector(Object value) { + if (value instanceof Map map) { + List res = new ArrayList<>(); + for (var entry : map.entrySet()) { + if (entry.getValue() instanceof Number number) { + res.add(new WeightedToken((String) entry.getKey(), number.floatValue())); + } else { + throw new IllegalArgumentException("Expected number, got " + entry.getValue().getClass().getSimpleName()); + } + } + return res; + } + throw new IllegalArgumentException("Expected map, got " + value.getClass().getSimpleName()); + } + + private void assertHighlightOneDoc( + MapperService mapperService, + ShardSearchRequest request, + SourceToParse source, + String fieldName, + int numFragments, + HighlightBuilder.Order order, + String[] expectedPassages + ) throws Exception { + SemanticTextFieldMapper fieldMapper = (SemanticTextFieldMapper) mapperService.mappingLookup().getMapper(fieldName); + var doc = mapperService.documentMapper().parse(source); + assertNull(doc.dynamicMappingsUpdate()); + try (Directory dir = newDirectory()) { + IndexWriterConfig iwc = newIndexWriterConfig(new StandardAnalyzer()); + RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwc); + iw.addDocuments(doc.docs()); + try (DirectoryReader reader = wrapInMockESDirectoryReader(iw.getReader())) { + IndexSearcher searcher = newSearcher(reader); + iw.close(); + TopDocs topDocs = searcher.search(Queries.newNonNestedFilter(IndexVersion.current()), 1, Sort.INDEXORDER); + assertThat(topDocs.totalHits.value(), equalTo(1L)); + int docID = topDocs.scoreDocs[0].doc; + SemanticTextHighlighter highlighter = new SemanticTextHighlighter(); + var execContext = createSearchExecutionContext(mapperService); + var luceneQuery = execContext.toQuery(request.source().query()).query(); + FetchContext fetchContext = mock(FetchContext.class); + Mockito.when(fetchContext.highlight()).thenReturn(new SearchHighlightContext(Collections.emptyList())); + Mockito.when(fetchContext.query()).thenReturn(luceneQuery); + Mockito.when(fetchContext.getSearchExecutionContext()).thenReturn(execContext); + + FetchSubPhase.HitContext hitContext = new FetchSubPhase.HitContext( + new SearchHit(docID), + getOnlyLeafReader(reader).getContext(), + docID, + Map.of(), + Source.fromBytes(source.source()), + new RankDoc(docID, Float.NaN, 0) + ); + try { + var highlightContext = new HighlightBuilder().field(fieldName, 0, numFragments) + .order(order) + .highlighterType(SemanticTextHighlighter.NAME) + .build(execContext); + + for (var fieldContext : highlightContext.fields()) { + FieldHighlightContext context = new FieldHighlightContext( + fieldName, + fieldContext, + fieldMapper.fieldType(), + fetchContext, + hitContext, + luceneQuery, + new HashMap<>() + ); + var result = highlighter.highlight(context); + assertThat(result.fragments().length, equalTo(expectedPassages.length)); + for (int i = 0; i < result.fragments().length; i++) { + assertThat(result.fragments()[i].string(), equalTo(expectedPassages[i])); + } + } + } finally { + hitContext.hit().decRef(); + } + } + } + } + + private SearchRequest createSearchRequest(QueryBuilder queryBuilder) { + SearchRequest request = new SearchRequest(); + request.source(new SearchSourceBuilder()); + request.allowPartialSearchResults(false); + request.source().query(queryBuilder); + return request; + } + + private ShardSearchRequest createShardSearchRequest(QueryBuilder queryBuilder) { + SearchRequest request = createSearchRequest(queryBuilder); + return new ShardSearchRequest(OriginalIndices.NONE, request, new ShardId("index", "index", 0), 0, 1, AliasFilter.EMPTY, 1, 0, null); + } + + private BytesReference readSampleDoc(String fileName) throws IOException { + try (var in = new GZIPInputStream(SemanticTextHighlighterTests.class.getResourceAsStream(fileName))) { + return new BytesArray(new BytesRef(in.readAllBytes())); + } + } +} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/mapper/SemanticTextFieldMapperTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/mapper/SemanticTextFieldMapperTests.java index fd60d9687f43..c6a492dfcf4e 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/mapper/SemanticTextFieldMapperTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/mapper/SemanticTextFieldMapperTests.java @@ -61,6 +61,7 @@ import org.elasticsearch.xcontent.XContentParser; import org.elasticsearch.xcontent.XContentType; import org.elasticsearch.xcontent.json.JsonXContent; +import org.elasticsearch.xpack.core.ml.search.SparseVectorQueryWrapper; import org.elasticsearch.xpack.inference.InferencePlugin; import org.elasticsearch.xpack.inference.model.TestModel; import org.junit.AssumptionViolatedException; @@ -1110,7 +1111,12 @@ private static Query generateNestedTermSparseVectorQuery(NestedLookup nestedLook } queryBuilder.add(new BooleanClause(mapper.nestedTypeFilter(), BooleanClause.Occur.FILTER)); - return new ESToParentBlockJoinQuery(queryBuilder.build(), parentFilter, ScoreMode.Total, null); + return new ESToParentBlockJoinQuery( + new SparseVectorQueryWrapper(fieldName, queryBuilder.build()), + parentFilter, + ScoreMode.Total, + null + ); } private static void assertChildLeafNestedDocument( diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/queries/SemanticQueryBuilderTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/queries/SemanticQueryBuilderTests.java index b8bcb766b53e..36aa2200ecea 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/queries/SemanticQueryBuilderTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/queries/SemanticQueryBuilderTests.java @@ -45,12 +45,14 @@ import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xcontent.XContentType; import org.elasticsearch.xcontent.json.JsonXContent; +import org.elasticsearch.xpack.core.XPackClientPlugin; import org.elasticsearch.xpack.core.inference.action.InferenceAction; import org.elasticsearch.xpack.core.inference.results.InferenceTextEmbeddingFloatResults; import org.elasticsearch.xpack.core.inference.results.SparseEmbeddingResults; import org.elasticsearch.xpack.core.ml.inference.MlInferenceNamedXContentProvider; import org.elasticsearch.xpack.core.ml.inference.results.MlTextEmbeddingResults; import org.elasticsearch.xpack.core.ml.inference.results.TextExpansionResults; +import org.elasticsearch.xpack.core.ml.search.SparseVectorQueryWrapper; import org.elasticsearch.xpack.core.ml.search.WeightedToken; import org.elasticsearch.xpack.inference.InferencePlugin; import org.elasticsearch.xpack.inference.mapper.SemanticTextField; @@ -114,7 +116,7 @@ public void setUp() throws Exception { @Override protected Collection> getPlugins() { - return List.of(InferencePlugin.class, FakeMlPlugin.class); + return List.of(XPackClientPlugin.class, InferencePlugin.class, FakeMlPlugin.class); } @Override @@ -194,9 +196,11 @@ protected void doAssertLuceneQuery(SemanticQueryBuilder queryBuilder, Query quer private void assertSparseEmbeddingLuceneQuery(Query query) { Query innerQuery = assertOuterBooleanQuery(query); - assertThat(innerQuery, instanceOf(BooleanQuery.class)); + assertThat(innerQuery, instanceOf(SparseVectorQueryWrapper.class)); + var sparseQuery = (SparseVectorQueryWrapper) innerQuery; + assertThat(((SparseVectorQueryWrapper) innerQuery).getTermsQuery(), instanceOf(BooleanQuery.class)); - BooleanQuery innerBooleanQuery = (BooleanQuery) innerQuery; + BooleanQuery innerBooleanQuery = (BooleanQuery) sparseQuery.getTermsQuery(); assertThat(innerBooleanQuery.clauses().size(), equalTo(queryTokenCount)); innerBooleanQuery.forEach(c -> { assertThat(c.occur(), equalTo(SHOULD)); diff --git a/x-pack/plugin/inference/src/test/resources/org/elasticsearch/xpack/inference/highlight/mappings.json b/x-pack/plugin/inference/src/test/resources/org/elasticsearch/xpack/inference/highlight/mappings.json new file mode 100644 index 000000000000..9841ee0aed6e --- /dev/null +++ b/x-pack/plugin/inference/src/test/resources/org/elasticsearch/xpack/inference/highlight/mappings.json @@ -0,0 +1,27 @@ +{ + "_doc": { + "properties": { + "body": { + "type": "text", + "copy_to": ["body-elser", "body-e5"] + }, + "body-e5": { + "type": "semantic_text", + "inference_id": ".multilingual-e5-small-elasticsearch", + "model_settings": { + "task_type": "text_embedding", + "dimensions": 384, + "similarity": "cosine", + "element_type": "float" + } + }, + "body-elser": { + "type": "semantic_text", + "inference_id": ".elser-2-elasticsearch", + "model_settings": { + "task_type": "sparse_embedding" + } + } + } + } +} \ No newline at end of file diff --git a/x-pack/plugin/inference/src/test/resources/org/elasticsearch/xpack/inference/highlight/queries.json b/x-pack/plugin/inference/src/test/resources/org/elasticsearch/xpack/inference/highlight/queries.json new file mode 100644 index 000000000000..6227f3f49885 --- /dev/null +++ b/x-pack/plugin/inference/src/test/resources/org/elasticsearch/xpack/inference/highlight/queries.json @@ -0,0 +1,467 @@ +{ + "dense_vector_1": { + "embeddings": [ + 0.09475211, + 0.044564713, + -0.04378501, + -0.07908551, + 0.04332011, + -0.03891992, + -0.0062305215, + 0.024245035, + -0.008976331, + 0.032832284, + 0.052760173, + 0.008123907, + 0.09049037, + -0.01637332, + -0.054353267, + 0.00771307, + 0.08545496, + -0.079716265, + -0.045666866, + -0.04369993, + 0.009189822, + -0.013782891, + -0.07701858, + 0.037278354, + 0.049807206, + 0.078036495, + -0.059533164, + 0.051413406, + 0.040234447, + -0.038139492, + -0.085189626, + -0.045546446, + 0.0544375, + -0.05604156, + 0.057408098, + 0.041913517, + -0.037348013, + -0.025998272, + 0.08486864, + -0.046678443, + 0.0041820924, + 0.007514462, + 0.06424746, + 0.044233218, + 0.103267275, + 0.014130771, + -0.049954403, + 0.04226959, + -0.08346965, + -0.01639249, + -0.060537644, + 0.04546336, + 0.012866155, + 0.05375096, + 0.036775924, + -0.0762226, + -0.037304543, + -0.05692274, + -0.055807598, + 0.0040082196, + 0.059259634, + 0.012022011, + -8.0863154E-4, + 0.0070405705, + 0.050255686, + 0.06810016, + 0.017190414, + 0.051975194, + -0.051436286, + 0.023408439, + -0.029802637, + 0.034137156, + -0.004660689, + -0.0442122, + 0.019065322, + 0.030806554, + 0.0064652697, + -0.066789865, + 0.057111286, + 0.009412479, + -0.041444767, + -0.06807582, + -0.085881524, + 0.04901128, + -0.047871742, + 0.06328623, + 0.040418074, + -0.081432894, + 0.058384005, + 0.006206527, + 0.045801315, + 0.037274595, + -0.054337103, + -0.06755516, + -0.07396888, + -0.043732334, + -0.052053086, + 0.03210978, + 0.048101492, + -0.083828256, + 0.05205026, + -0.048474856, + 0.029116616, + -0.10924888, + 0.003796487, + 0.030567763, + 0.026949523, + -0.052353345, + 0.043198872, + -0.09456988, + -0.05711594, + -2.2292069E-4, + 0.032972734, + 0.054394923, + -0.0767535, + -0.02710579, + -0.032135617, + -0.01732382, + 0.059442326, + -0.07686165, + 0.07104082, + -0.03090021, + -0.05450075, + -0.038997203, + -0.07045443, + 0.00483161, + 0.010933604, + 0.020874644, + 0.037941266, + 0.019729063, + 0.06178368, + 0.013503478, + -0.008584046, + 0.045592044, + 0.05528768, + 0.11568184, + 0.0041300594, + 0.015404516, + -3.8067883E-4, + -0.06365399, + -0.07826643, + 0.061575573, + -0.060548335, + 0.05706082, + 0.042301804, + 0.052173313, + 0.07193179, + -0.03839231, + 0.0734415, + -0.045380164, + 0.02832276, + 0.003745178, + 0.058844633, + 0.04307504, + 0.037800383, + -0.031050054, + -0.06856359, + -0.059114788, + -0.02148857, + 0.07854358, + -0.03253363, + -0.04566468, + -0.019933948, + -0.057993464, + -0.08677458, + -0.06626883, + 0.031657256, + 0.101128764, + -0.08050056, + -0.050226066, + -0.014335166, + 0.050344367, + -0.06851419, + 0.008698909, + -0.011893435, + 0.07741272, + -0.059579294, + 0.03250109, + 0.058700256, + 0.046834726, + -0.035081457, + -0.0043140925, + -0.09764087, + -0.0034994273, + -0.034056358, + -0.019066337, + -0.034376107, + 0.012964423, + 0.029291175, + -0.012090671, + 0.021585712, + 0.028859599, + -0.04391145, + -0.071166754, + -0.031040335, + 0.02808108, + -0.05621317, + 0.06543945, + 0.10094665, + 0.041057374, + -0.03222324, + -0.063366964, + 0.064944476, + 0.023641933, + 0.06806713, + 0.06806097, + -0.08220105, + 0.04148528, + -0.09254079, + 0.044620737, + 0.05526614, + -0.03849534, + -0.04722273, + 0.0670776, + -0.024274077, + -0.016903497, + 0.07584147, + 0.04760533, + -0.038843267, + -0.028365409, + 0.08022705, + -0.039916333, + 0.049067073, + -0.030701574, + -0.057169467, + 0.043025102, + 0.07109674, + -0.047296863, + -0.047463104, + 0.040868305, + -0.04409507, + -0.034977127, + -0.057109762, + -0.08616165, + -0.03486079, + -0.046201482, + 0.025963873, + 0.023392359, + 0.09594902, + -0.007847159, + -0.021231368, + 0.009007263, + 0.0032713825, + -0.06876065, + 0.03169641, + -7.2582875E-4, + -0.07049708, + 0.03900843, + -0.0075472407, + 0.05184822, + 0.06452079, + -0.09832754, + -0.012775799, + -0.03925948, + -0.029761659, + 0.0065437574, + 0.0815465, + 0.0411695, + -0.0702844, + -0.009533786, + 0.07024532, + 0.0098710675, + 0.09915362, + 0.0415453, + 0.050641853, + 0.047463298, + -0.058609713, + -0.029499197, + -0.05100956, + -0.03441709, + -0.06348122, + 0.014784361, + 0.056317374, + -0.10280704, + -0.04008354, + -0.018926824, + 0.08832836, + 0.124804, + -0.047645308, + -0.07122146, + -9.886527E-4, + 0.03850324, + 0.048501793, + 0.07072816, + 0.06566776, + -0.013678872, + 0.010010848, + 0.06483413, + -0.030036367, + -0.029748922, + -0.007482364, + -0.05180385, + 0.03698522, + -0.045453787, + 0.056604166, + 0.029394176, + 0.028589265, + -0.012185886, + -0.06919616, + 0.0711641, + -0.034055933, + -0.053101335, + 0.062319, + 0.021600349, + -0.038718067, + 0.060814686, + 0.05087301, + -0.020297311, + 0.016493896, + 0.032162152, + 0.046740912, + 0.05461355, + -0.07024665, + 0.025609337, + -0.02504801, + 0.06765588, + -0.032994855, + -0.037897404, + -0.045783922, + -0.05689299, + -0.040437017, + -0.07904339, + -0.031415287, + -0.029216278, + 0.017395392, + 0.03449264, + -0.025653394, + -0.06283088, + 0.049027324, + 0.016229525, + -0.00985347, + -0.053974394, + -0.030257035, + 0.04325515, + -0.012293731, + -0.002446129, + -0.05567076, + 0.06374684, + -0.03153897, + -0.04475149, + 0.018582936, + 0.025716115, + -0.061778374, + 0.04196277, + -0.04134671, + -0.07396272, + 0.05846184, + 0.006558759, + -0.09745666, + 0.07587805, + 0.0137483915, + -0.100933895, + 0.032008193, + 0.04293283, + 0.017870268, + 0.032806385, + -0.0635923, + -0.019672254, + 0.022225974, + 0.04304554, + -0.06043949, + -0.0285274, + 0.050868835, + 0.057003833, + 0.05740866, + 0.020068677, + -0.034312245, + -0.021671802, + 0.014769731, + -0.07328285, + -0.009586734, + 0.036420938, + -0.022188472, + -0.008200541, + -0.010765854, + -0.06949713, + -0.07555878, + 0.045306854, + -0.05424466, + -0.03647476, + 0.06266633, + 0.08346125, + 0.060288202, + 0.0548457 + ], + "expected_by_score": [ + "The ancient oppidum that corresponds to the modern city of Paris was first mentioned in the mid-1st century BC by Julius Caesar as Luteciam Parisiorum ('Lutetia of the Parisii') and is later attested as Parision in the 5th century AD, then as Paris in 1265. During the Roman period, it was commonly known as Lutetia or Lutecia in Latin, and as Leukotekía in Greek, which is interpreted as either stemming from the Celtic root *lukot- ('mouse'), or from *luto- ('marsh, swamp').\n\n\nThe name Paris is derived from its early inhabitants, the Parisii, a Gallic tribe from the Iron Age and the Roman period. The meaning of the Gaulish ethnonym remains debated. According to Xavier Delamarre, it may derive from the Celtic root pario- ('cauldron'). Alfred Holder interpreted the name as 'the makers' or 'the commanders', by comparing it to the Welsh peryff ('lord, commander'), both possibly descending from a Proto-Celtic form reconstructed as *kwar-is-io-. Alternatively, Pierre-Yves Lambert proposed to translate Parisii as the 'spear people', by connecting the first element to the Old Irish carr ('spear'), derived from an earlier *kwar-sā. In any case, the city's name is not related to the Paris of Greek mythology.\n\n\nResidents of the city are known in English as Parisians and in French as Parisiens ( ⓘ). They are also pejoratively called Parigots ( ⓘ).\n\n\nHistory\n\nOrigins\n\n", + "After the marshland between the river Seine and its slower 'dead arm' to its north was filled in from around the 10th century, Paris's cultural centre began to move to the Right Bank. In 1137, a new city marketplace (today's Les Halles) replaced the two smaller ones on the Île de la Cité and Place de Grève (Place de l'Hôtel de Ville). The latter location housed the headquarters of Paris's river trade corporation, an organisation that later became, unofficially (although formally in later years), Paris's first municipal government.\n\n\nIn the late 12th century, Philip Augustus extended the Louvre fortress to defend the city against river invasions from the west, gave the city its first walls between 1190 and 1215, rebuilt its bridges to either side of its central island, and paved its main thoroughfares. In 1190, he transformed Paris's former cathedral school into a student-teacher corporation that would become the University of Paris and would draw students from all of Europe.\n\n\nWith 200,000 inhabitants in 1328, Paris, then already the capital of France, was the most populous city of Europe. By comparison, London in 1300 had 80,000 inhabitants. By the early fourteenth century, so much filth had collected inside urban Europe that French and Italian cities were naming streets after human waste. In medieval Paris, several street names were inspired by merde, the French word for \"shit\".\n\n\n", + "In March 2001, Bertrand Delanoë became the first socialist mayor. He was re-elected in March 2008. In 2007, in an effort to reduce car traffic, he introduced the Vélib', a system which rents bicycles. Bertrand Delanoë also transformed a section of the highway along the Left Bank of the Seine into an urban promenade and park, the Promenade des Berges de la Seine, which he inaugurated in June 2013.\n\n\nIn 2007, President Nicolas Sarkozy launched the Grand Paris project, to integrate Paris more closely with the towns in the region around it. After many modifications, the new area, named the Metropolis of Grand Paris, with a population of 6.7 million, was created on 1 January 2016. In 2011, the City of Paris and the national government approved the plans for the Grand Paris Express, totalling 205 km (127 mi) of automated metro lines to connect Paris, the innermost three departments around Paris, airports and high-speed rail (TGV) stations, at an estimated cost of €35 billion. The system is scheduled to be completed by 2030.\n\n\nIn January 2015, Al-Qaeda in the Arabian Peninsula claimed attacks across the Paris region. 1.5 million people marched in Paris in a show of solidarity against terrorism and in support of freedom of speech. In November of the same year, terrorist attacks, claimed by ISIL, killed 130 people and injured more than 350.\n\n\n", + "\nParis (.mw-parser-output .IPA-label-small{font-size:85%}.mw-parser-output .references .IPA-label-small,.mw-parser-output .infobox .IPA-label-small,.mw-parser-output .navbox .IPA-label-small{font-size:100%}French pronunciation: ⓘ) is the capital and largest city of France. With an estimated population of 2,102,650 residents in January 2023 in an area of more than 105 km2 (41 sq mi), Paris is the fourth-largest city in the European Union and the 30th most densely populated city in the world in 2022. Since the 17th century, Paris has been one of the world's major centres of finance, diplomacy, commerce, culture, fashion, and gastronomy. Because of its leading role in the arts and sciences and its early adaptation of extensive street lighting, it became known as the City of Light in the 19th century.\n\n\nThe City of Paris is the centre of the Île-de-France region, or Paris Region, with an official estimated population of 12,271,794 inhabitants in January 2023, or about 19% of the population of France. The Paris Region had a nominal GDP of €765 billion (US$1.064 trillion when adjusted for PPP) in 2021, the highest in the European Union. According to the Economist Intelligence Unit Worldwide Cost of Living Survey, in 2022, Paris was the city with the ninth-highest cost of living in the world.\n\n\n", + "Bal-musette is a style of French music and dance that first became popular in Paris in the 1870s and 1880s; by 1880 Paris had some 150 dance halls. Patrons danced the bourrée to the accompaniment of the cabrette (a bellows-blown bagpipe locally called a \"musette\") and often the vielle à roue (hurdy-gurdy) in the cafés and bars of the city. Parisian and Italian musicians who played the accordion adopted the style and established themselves in Auvergnat bars, and Paris became a major centre for jazz and still attracts jazz musicians from all around the world to its clubs and cafés.\n\n\nParis is the spiritual home of gypsy jazz in particular, and many of the Parisian jazzmen who developed in the first half of the 20th century began by playing Bal-musette in the city. Django Reinhardt rose to fame in Paris, having moved to the 18th arrondissement in a caravan as a young boy, and performed with violinist Stéphane Grappelli and their Quintette du Hot Club de France in the 1930s and 1940s.\n\n\nImmediately after the War the Saint-Germain-des-Pres quarter and the nearby Saint-Michel quarter became home to many small jazz clubs, including the Caveau des Lorientais, the Club Saint-Germain, the Rose Rouge, the Vieux-Colombier, and the most famous, Le Tabou. They introduced Parisians to the music of Claude Luter, Boris Vian, Sydney Bechet, Mezz Mezzrow, and Henri Salvador. " + ], + "expected_by_offset": [ + "\nParis (.mw-parser-output .IPA-label-small{font-size:85%}.mw-parser-output .references .IPA-label-small,.mw-parser-output .infobox .IPA-label-small,.mw-parser-output .navbox .IPA-label-small{font-size:100%}French pronunciation: ⓘ) is the capital and largest city of France. With an estimated population of 2,102,650 residents in January 2023 in an area of more than 105 km2 (41 sq mi), Paris is the fourth-largest city in the European Union and the 30th most densely populated city in the world in 2022. Since the 17th century, Paris has been one of the world's major centres of finance, diplomacy, commerce, culture, fashion, and gastronomy. Because of its leading role in the arts and sciences and its early adaptation of extensive street lighting, it became known as the City of Light in the 19th century.\n\n\nThe City of Paris is the centre of the Île-de-France region, or Paris Region, with an official estimated population of 12,271,794 inhabitants in January 2023, or about 19% of the population of France. The Paris Region had a nominal GDP of €765 billion (US$1.064 trillion when adjusted for PPP) in 2021, the highest in the European Union. According to the Economist Intelligence Unit Worldwide Cost of Living Survey, in 2022, Paris was the city with the ninth-highest cost of living in the world.\n\n\n", + "The ancient oppidum that corresponds to the modern city of Paris was first mentioned in the mid-1st century BC by Julius Caesar as Luteciam Parisiorum ('Lutetia of the Parisii') and is later attested as Parision in the 5th century AD, then as Paris in 1265. During the Roman period, it was commonly known as Lutetia or Lutecia in Latin, and as Leukotekía in Greek, which is interpreted as either stemming from the Celtic root *lukot- ('mouse'), or from *luto- ('marsh, swamp').\n\n\nThe name Paris is derived from its early inhabitants, the Parisii, a Gallic tribe from the Iron Age and the Roman period. The meaning of the Gaulish ethnonym remains debated. According to Xavier Delamarre, it may derive from the Celtic root pario- ('cauldron'). Alfred Holder interpreted the name as 'the makers' or 'the commanders', by comparing it to the Welsh peryff ('lord, commander'), both possibly descending from a Proto-Celtic form reconstructed as *kwar-is-io-. Alternatively, Pierre-Yves Lambert proposed to translate Parisii as the 'spear people', by connecting the first element to the Old Irish carr ('spear'), derived from an earlier *kwar-sā. In any case, the city's name is not related to the Paris of Greek mythology.\n\n\nResidents of the city are known in English as Parisians and in French as Parisiens ( ⓘ). They are also pejoratively called Parigots ( ⓘ).\n\n\nHistory\n\nOrigins\n\n", + "After the marshland between the river Seine and its slower 'dead arm' to its north was filled in from around the 10th century, Paris's cultural centre began to move to the Right Bank. In 1137, a new city marketplace (today's Les Halles) replaced the two smaller ones on the Île de la Cité and Place de Grève (Place de l'Hôtel de Ville). The latter location housed the headquarters of Paris's river trade corporation, an organisation that later became, unofficially (although formally in later years), Paris's first municipal government.\n\n\nIn the late 12th century, Philip Augustus extended the Louvre fortress to defend the city against river invasions from the west, gave the city its first walls between 1190 and 1215, rebuilt its bridges to either side of its central island, and paved its main thoroughfares. In 1190, he transformed Paris's former cathedral school into a student-teacher corporation that would become the University of Paris and would draw students from all of Europe.\n\n\nWith 200,000 inhabitants in 1328, Paris, then already the capital of France, was the most populous city of Europe. By comparison, London in 1300 had 80,000 inhabitants. By the early fourteenth century, so much filth had collected inside urban Europe that French and Italian cities were naming streets after human waste. In medieval Paris, several street names were inspired by merde, the French word for \"shit\".\n\n\n", + "In March 2001, Bertrand Delanoë became the first socialist mayor. He was re-elected in March 2008. In 2007, in an effort to reduce car traffic, he introduced the Vélib', a system which rents bicycles. Bertrand Delanoë also transformed a section of the highway along the Left Bank of the Seine into an urban promenade and park, the Promenade des Berges de la Seine, which he inaugurated in June 2013.\n\n\nIn 2007, President Nicolas Sarkozy launched the Grand Paris project, to integrate Paris more closely with the towns in the region around it. After many modifications, the new area, named the Metropolis of Grand Paris, with a population of 6.7 million, was created on 1 January 2016. In 2011, the City of Paris and the national government approved the plans for the Grand Paris Express, totalling 205 km (127 mi) of automated metro lines to connect Paris, the innermost three departments around Paris, airports and high-speed rail (TGV) stations, at an estimated cost of €35 billion. The system is scheduled to be completed by 2030.\n\n\nIn January 2015, Al-Qaeda in the Arabian Peninsula claimed attacks across the Paris region. 1.5 million people marched in Paris in a show of solidarity against terrorism and in support of freedom of speech. In November of the same year, terrorist attacks, claimed by ISIL, killed 130 people and injured more than 350.\n\n\n", + "Bal-musette is a style of French music and dance that first became popular in Paris in the 1870s and 1880s; by 1880 Paris had some 150 dance halls. Patrons danced the bourrée to the accompaniment of the cabrette (a bellows-blown bagpipe locally called a \"musette\") and often the vielle à roue (hurdy-gurdy) in the cafés and bars of the city. Parisian and Italian musicians who played the accordion adopted the style and established themselves in Auvergnat bars, and Paris became a major centre for jazz and still attracts jazz musicians from all around the world to its clubs and cafés.\n\n\nParis is the spiritual home of gypsy jazz in particular, and many of the Parisian jazzmen who developed in the first half of the 20th century began by playing Bal-musette in the city. Django Reinhardt rose to fame in Paris, having moved to the 18th arrondissement in a caravan as a young boy, and performed with violinist Stéphane Grappelli and their Quintette du Hot Club de France in the 1930s and 1940s.\n\n\nImmediately after the War the Saint-Germain-des-Pres quarter and the nearby Saint-Michel quarter became home to many small jazz clubs, including the Caveau des Lorientais, the Club Saint-Germain, the Rose Rouge, the Vieux-Colombier, and the most famous, Le Tabou. They introduced Parisians to the music of Claude Luter, Boris Vian, Sydney Bechet, Mezz Mezzrow, and Henri Salvador. " + ] + }, + "sparse_vector_1": { + "embeddings": { + "paris": 2.9709616, + "date": 2.1960778, + "founded": 2.0555024, + "foundation": 1.412623, + "early": 1.2162757, + "founder": 1.1271698, + "french": 0.9213378, + "france": 0.86253893, + "city": 0.82978916, + "founding": 0.79722786, + "established": 0.7967043, + "ancient": 0.7392465, + "when": 0.71705, + "built": 0.6977878, + "treaty": 0.6846069, + "created": 0.68127465, + "century": 0.58926934, + "for": 0.55019474, + "was": 0.52475905, + "origin": 0.48785052, + "expedition": 0.48757303, + "history": 0.47960007, + "mint": 0.47878903, + "historical": 0.4714338, + "capital": 0.42984143, + "timeline": 0.4222377, + "colony": 0.3876187, + "tower": 0.3474891, + "medieval": 0.3272666, + "geography": 0.32456368, + "colonial": 0.30613664, + "location": 0.29013386, + "francisco": 0.22840048, + "orleans": 0.21971667, + "earlier": 0.20318772, + "jackson": 0.18424438, + "exact": 0.17109296, + "rome": 0.16320735, + "civilization": 0.15931238, + "spanish": 0.12759624, + "museum": 0.113024555, + "latin": 0.11201205, + "european": 0.10277243, + "architect": 0.0796932, + "united": 0.031233707 + }, + "expected_by_score": [ + "Clovis the Frank, the first king of the Merovingian dynasty, made the city his capital from 508. As the Frankish domination of Gaul began, there was a gradual immigration by the Franks to Paris and the Parisian Francien dialects were born. Fortification of the Île de la Cité failed to avert sacking by Vikings in 845, but Paris's strategic importance—with its bridges preventing ships from passing—was established by successful defence in the Siege of Paris (885–886), for which the then Count of Paris (comte de Paris), Odo of France, was elected king of West Francia. From the Capetian dynasty that began with the 987 election of Hugh Capet, Count of Paris and Duke of the Franks (duc des Francs), as king of a unified West Francia, Paris gradually became the largest and most prosperous city in France.\n\n\nHigh and Late Middle Ages to Louis XIV\n\nBy the end of the 12th century, Paris had become the political, economic, religious, and cultural capital of France. The Palais de la Cité, the royal residence, was located at the western end of the Île de la Cité. In 1163, during the reign of Louis VII, Maurice de Sully, bishop of Paris, undertook the construction of the Notre Dame Cathedral at its eastern extremity.\n\n\nAfter the marshland between the river Seine and its slower 'dead arm' to its north was filled in from around the 10th century, Paris's cultural centre began to move to the Right Bank. ", + "\nThe Parisii, a sub-tribe of the Celtic Senones, inhabited the Paris area from around the middle of the 3rd century BC. One of the area's major north–south trade routes crossed the Seine on the Île de la Cité, which gradually became an important trading centre. The Parisii traded with many river towns (some as far away as the Iberian Peninsula) and minted their own coins.\n\n\nThe Romans conquered the Paris Basin in 52 BC and began their settlement on Paris's Left Bank. The Roman town was originally called Lutetia (more fully, Lutetia Parisiorum, \"Lutetia of the Parisii\", modern French Lutèce). It became a prosperous city with a forum, baths, temples, theatres, and an amphitheatre.\n\n\nBy the end of the Western Roman Empire, the town was known as Parisius, a Latin name that would later become Paris in French. Christianity was introduced in the middle of the 3rd century AD by Saint Denis, the first Bishop of Paris: according to legend, when he refused to renounce his faith before the Roman occupiers, he was beheaded on the hill which became known as Mons Martyrum (Latin \"Hill of Martyrs\"), later \"Montmartre\", from where he walked headless to the north of the city; the place where he fell and was buried became an important religious shrine, the Basilica of Saint-Denis, and many French kings are buried there.\n\n\nClovis the Frank, the first king of the Merovingian dynasty, made the city his capital from 508. ", + "\nDuring the Hundred Years' War, Paris was occupied by England-friendly Burgundian forces from 1418, before being occupied outright by the English when Henry V of England entered the French capital in 1420; in spite of a 1429 effort by Joan of Arc to liberate the city, it would remain under English occupation until 1436.\n\n\nIn the late 16th-century French Wars of Religion, Paris was a stronghold of the Catholic League, the organisers of 24 August 1572 St. Bartholomew's Day massacre in which thousands of French Protestants were killed. The conflicts ended when pretender to the throne Henry IV, after converting to Catholicism to gain entry to the capital, entered the city in 1594 to claim the crown of France. This king made several improvements to the capital during his reign: he completed the construction of Paris's first uncovered, sidewalk-lined bridge, the Pont Neuf, built a Louvre extension connecting it to the Tuileries Palace, and created the first Paris residential square, the Place Royale, now Place des Vosges. In spite of Henry IV's efforts to improve city circulation, the narrowness of Paris's streets was a contributing factor in his assassination near Les Halles marketplace in 1610.\n\n\nDuring the 17th century, Cardinal Richelieu, chief minister of Louis XIII, was determined to make Paris the most beautiful city in Europe. He built five new bridges, a new chapel for the College of Sorbonne, and a palace for himself, the Palais-Cardinal. ", + "Diderot and D'Alembert published their Encyclopédie in 1751, before the Montgolfier Brothers launched the first manned flight in a hot air balloon on 21 November 1783. Paris was the financial capital of continental Europe, as well the primary European centre for book publishing, fashion and the manufacture of fine furniture and luxury goods. On 22 October 1797, Paris was also the site of the first parachute jump in history, by Garnerin.\n\n\nIn the summer of 1789, Paris became the centre stage of the French Revolution. On 14 July, a mob seized the arsenal at the Invalides, acquiring thousands of guns, with which it stormed the Bastille, a principal symbol of royal authority. The first independent Paris Commune, or city council, met in the Hôtel de Ville and elected a Mayor, the astronomer Jean Sylvain Bailly, on 15 July.\n\n\nLouis XVI and the royal family were brought to Paris and incarcerated in the Tuileries Palace. In 1793, as the revolution turned increasingly radical, the king, queen and mayor were beheaded by guillotine in the Reign of Terror, along with more than 16,000 others throughout France. The property of the aristocracy and the church was nationalised, and the city's churches were closed, sold or demolished. A succession of revolutionary factions ruled Paris until 9 November 1799 (coup d'état du 18 brumaire), when Napoleon Bonaparte seized power as First Consul.\n\n\n", + "After the marshland between the river Seine and its slower 'dead arm' to its north was filled in from around the 10th century, Paris's cultural centre began to move to the Right Bank. In 1137, a new city marketplace (today's Les Halles) replaced the two smaller ones on the Île de la Cité and Place de Grève (Place de l'Hôtel de Ville). The latter location housed the headquarters of Paris's river trade corporation, an organisation that later became, unofficially (although formally in later years), Paris's first municipal government.\n\n\nIn the late 12th century, Philip Augustus extended the Louvre fortress to defend the city against river invasions from the west, gave the city its first walls between 1190 and 1215, rebuilt its bridges to either side of its central island, and paved its main thoroughfares. In 1190, he transformed Paris's former cathedral school into a student-teacher corporation that would become the University of Paris and would draw students from all of Europe.\n\n\nWith 200,000 inhabitants in 1328, Paris, then already the capital of France, was the most populous city of Europe. By comparison, London in 1300 had 80,000 inhabitants. By the early fourteenth century, so much filth had collected inside urban Europe that French and Italian cities were naming streets after human waste. In medieval Paris, several street names were inspired by merde, the French word for \"shit\".\n\n\n" + ], + "expected_by_offset": [ + "\nThe Parisii, a sub-tribe of the Celtic Senones, inhabited the Paris area from around the middle of the 3rd century BC. One of the area's major north–south trade routes crossed the Seine on the Île de la Cité, which gradually became an important trading centre. The Parisii traded with many river towns (some as far away as the Iberian Peninsula) and minted their own coins.\n\n\nThe Romans conquered the Paris Basin in 52 BC and began their settlement on Paris's Left Bank. The Roman town was originally called Lutetia (more fully, Lutetia Parisiorum, \"Lutetia of the Parisii\", modern French Lutèce). It became a prosperous city with a forum, baths, temples, theatres, and an amphitheatre.\n\n\nBy the end of the Western Roman Empire, the town was known as Parisius, a Latin name that would later become Paris in French. Christianity was introduced in the middle of the 3rd century AD by Saint Denis, the first Bishop of Paris: according to legend, when he refused to renounce his faith before the Roman occupiers, he was beheaded on the hill which became known as Mons Martyrum (Latin \"Hill of Martyrs\"), later \"Montmartre\", from where he walked headless to the north of the city; the place where he fell and was buried became an important religious shrine, the Basilica of Saint-Denis, and many French kings are buried there.\n\n\nClovis the Frank, the first king of the Merovingian dynasty, made the city his capital from 508. ", + "Clovis the Frank, the first king of the Merovingian dynasty, made the city his capital from 508. As the Frankish domination of Gaul began, there was a gradual immigration by the Franks to Paris and the Parisian Francien dialects were born. Fortification of the Île de la Cité failed to avert sacking by Vikings in 845, but Paris's strategic importance—with its bridges preventing ships from passing—was established by successful defence in the Siege of Paris (885–886), for which the then Count of Paris (comte de Paris), Odo of France, was elected king of West Francia. From the Capetian dynasty that began with the 987 election of Hugh Capet, Count of Paris and Duke of the Franks (duc des Francs), as king of a unified West Francia, Paris gradually became the largest and most prosperous city in France.\n\n\nHigh and Late Middle Ages to Louis XIV\n\nBy the end of the 12th century, Paris had become the political, economic, religious, and cultural capital of France. The Palais de la Cité, the royal residence, was located at the western end of the Île de la Cité. In 1163, during the reign of Louis VII, Maurice de Sully, bishop of Paris, undertook the construction of the Notre Dame Cathedral at its eastern extremity.\n\n\nAfter the marshland between the river Seine and its slower 'dead arm' to its north was filled in from around the 10th century, Paris's cultural centre began to move to the Right Bank. ", + "After the marshland between the river Seine and its slower 'dead arm' to its north was filled in from around the 10th century, Paris's cultural centre began to move to the Right Bank. In 1137, a new city marketplace (today's Les Halles) replaced the two smaller ones on the Île de la Cité and Place de Grève (Place de l'Hôtel de Ville). The latter location housed the headquarters of Paris's river trade corporation, an organisation that later became, unofficially (although formally in later years), Paris's first municipal government.\n\n\nIn the late 12th century, Philip Augustus extended the Louvre fortress to defend the city against river invasions from the west, gave the city its first walls between 1190 and 1215, rebuilt its bridges to either side of its central island, and paved its main thoroughfares. In 1190, he transformed Paris's former cathedral school into a student-teacher corporation that would become the University of Paris and would draw students from all of Europe.\n\n\nWith 200,000 inhabitants in 1328, Paris, then already the capital of France, was the most populous city of Europe. By comparison, London in 1300 had 80,000 inhabitants. By the early fourteenth century, so much filth had collected inside urban Europe that French and Italian cities were naming streets after human waste. In medieval Paris, several street names were inspired by merde, the French word for \"shit\".\n\n\n", + "\nDuring the Hundred Years' War, Paris was occupied by England-friendly Burgundian forces from 1418, before being occupied outright by the English when Henry V of England entered the French capital in 1420; in spite of a 1429 effort by Joan of Arc to liberate the city, it would remain under English occupation until 1436.\n\n\nIn the late 16th-century French Wars of Religion, Paris was a stronghold of the Catholic League, the organisers of 24 August 1572 St. Bartholomew's Day massacre in which thousands of French Protestants were killed. The conflicts ended when pretender to the throne Henry IV, after converting to Catholicism to gain entry to the capital, entered the city in 1594 to claim the crown of France. This king made several improvements to the capital during his reign: he completed the construction of Paris's first uncovered, sidewalk-lined bridge, the Pont Neuf, built a Louvre extension connecting it to the Tuileries Palace, and created the first Paris residential square, the Place Royale, now Place des Vosges. In spite of Henry IV's efforts to improve city circulation, the narrowness of Paris's streets was a contributing factor in his assassination near Les Halles marketplace in 1610.\n\n\nDuring the 17th century, Cardinal Richelieu, chief minister of Louis XIII, was determined to make Paris the most beautiful city in Europe. He built five new bridges, a new chapel for the College of Sorbonne, and a palace for himself, the Palais-Cardinal. ", + "Diderot and D'Alembert published their Encyclopédie in 1751, before the Montgolfier Brothers launched the first manned flight in a hot air balloon on 21 November 1783. Paris was the financial capital of continental Europe, as well the primary European centre for book publishing, fashion and the manufacture of fine furniture and luxury goods. On 22 October 1797, Paris was also the site of the first parachute jump in history, by Garnerin.\n\n\nIn the summer of 1789, Paris became the centre stage of the French Revolution. On 14 July, a mob seized the arsenal at the Invalides, acquiring thousands of guns, with which it stormed the Bastille, a principal symbol of royal authority. The first independent Paris Commune, or city council, met in the Hôtel de Ville and elected a Mayor, the astronomer Jean Sylvain Bailly, on 15 July.\n\n\nLouis XVI and the royal family were brought to Paris and incarcerated in the Tuileries Palace. In 1793, as the revolution turned increasingly radical, the king, queen and mayor were beheaded by guillotine in the Reign of Terror, along with more than 16,000 others throughout France. The property of the aristocracy and the church was nationalised, and the city's churches were closed, sold or demolished. A succession of revolutionary factions ruled Paris until 9 November 1799 (coup d'état du 18 brumaire), when Napoleon Bonaparte seized power as First Consul.\n\n\n" + ] + } +} \ No newline at end of file diff --git a/x-pack/plugin/inference/src/test/resources/org/elasticsearch/xpack/inference/highlight/sample-doc.json.gz b/x-pack/plugin/inference/src/test/resources/org/elasticsearch/xpack/inference/highlight/sample-doc.json.gz new file mode 100644 index 0000000000000000000000000000000000000000..881524e46e18639782402236d4aca01ffcfaf02b GIT binary patch literal 388098 zcmV(^K-Iq=iwFqnhEZn#19M?*aBO8QWN%|GYIARH0PMZXjx1M}9XRi=@Md&@q!9PQ z-jC9VWHK{Z%1pA8`H-L#3Ks4b5ytE8=4s}BgHb{O)jiXsC)I-<)=X7E_e@_<>@TS= z>9x+;$IRT9T~;ZDMngbD1?6R&?PkZ$b3fKz`#=5yD4@&>pw#^rz(?|FYZR?;pK&KYEkk$IW=|wqx52+vkt&gZ!c2wfnyu+w-|QK8=ef zq3zs^@BZ@qdg2Q{8c)qIwtv|k9@^awvLBh;Ur&$PK3|US_=tZ$Jz{f7SkZmH*ja zul!(W?R!W0@%@T8hNpk%zyBA>9-H@TqKiJoQS@hj{ZqdFd2<^2+zR3%K!22-W`(7Ct3WDe>eW|AOG&qeR{v~(_npk>@UOl`OQ`L@H{qsc^Udsi|73I z$jeDbm7M@$d+c-$PC@A|?007JmW$L`3%yWi}()4o46TRdjl9}exnKW;C3JQ@FZ zY{qAF{NDBa3EKcu)gNBpZa#0f&1H1&=`fu8w&BX&4E??hoo@y-@OR_3^UI9?L1}Cx zd~&mEPUmTou);AE-TQWfm$dD9v+tgsxwG!^6MWaUIkcO1$NmMQ=!g9aTL!~A|LE8DhZkFI{o`YYHM{@d zvWt`Zq>S&&Iz#84n+L4pm6hk-&^%!0pznVcI&=M`u+aG>%g$iRcbjH|8NqDsH$VHy zH!b zJY!V3npYR{+s$X&Z9ljvKlff>%azgL7hfIEE#C9QIj8t}^Lwts7wo*v7hLmh&ffF4 z-(H6IEmoeb@UW6!Y=+%daO2JYI(A3wve2)sJ!$WswA?T@@6$=`j?n4!y*{a3+lg<* zpOd#)ROVS#-e0$Orv|O@+oCM$tV(WL4==tkP2#esZW`x_bV7NW)QJc^x^vJAM0F~y zjlZelw^?0e=)sMEKXsizS`~S6`v7+WTXS01S$@;bVYcSDOv~G5w-J4NSrnPP{_f7- zt!OXuERN;z{s{LHX_>`&EKhj<;F=SBTovEG;AwaAtx;LV`R(J~&Enc)ep_+-@YYtY z6N&NyLwNfLUQ50?iJ~H|Z>N9nM?Z?=0_{3_UCY%Iy{)nD>O9J10J%hiL>kw(E9mZy z=c*Sd>O`M#YWtJx1O<{JE=6Lx*}PDmWkqq@)2n;d%VkBB$roWO`qrdK>hyLsVY66k z9_R8~p1-ac^B7f8a=Vo{8&f-$^X=2Qt$lkLsSsqKN?#vlb2W; zqHLI=Z6Z0&6ZOM+IN37Z}}{(v$&MSdVm#Sy@g7EQ}T8=a7#qK zJxQQ5-M;5LJ2A2(PU<+l{f;X4t$6P3d>_1)YVFu)m5kKr^hT3N@R_l8-xgJdTL|aL z?PRszpS-!@gwndo^>}WNPy254D<`e$h{x9LHoJ8Dvk4R!?AuYmKK9*A%OuXr{B{TN z(DGl3(?6+Xm+qV6)1`T`USdhqv0Qn7W{uRK>)XK`x~+WxR(`6w;h{NMbB=+MQ>1&E zZqqcybsfpE?e|XWFVr+iYmBwbs5K^T5N;dWUDv*E_CW@Bn8?tq^zCt3VcAP3g7s`d zT$1_if<3g*vQGV;C$Pyu14*SnFl#Rde`=)|6O7BP2vxqnYL7vt$e_3CxO8?tr%{T7 zO_q+cWt~6*-2m^o?e7jcoqq)`lQOF0N@|+W8m%0y!7BAKMI9wj9appiYfrEP%G(e1 zo((UNLqmyG59%++U3f{3fj~#)8W?z&coCQnRW6+!db_I<+9zoy{h$fdcQzU+&4k-E zmO%fp<`@UGT+c6Uvk#ww`>v>D#+;qf?@gq{r--WW{U=@mHy%d*c0a-Rw&tSBS2ome z8F3ASN5mD(kz1MWxPxJCH!o}+bhfztnlmP*8xJOeL5<~epwC!yQdPB#$HQP|T|~Pn zfoZEZ*1;>n-F7s&sLn-W$JD&D1rXys85W7?u-J92o#-(X`#KRL(rKa7W3VxGev-)M zFD`2=WzmN3?ow;!J5o`Wy%}nsQbsUDWu3wN^UY9|acRji#iv?3zEiX^=hm`zoVBSK zlC)mz{do;#{B|hs?h=)F;-;$xd$@cUUXF zJUM!*TDJYJHIXbSBh~rc$c`q0qova8p3Y`30QI!2Wg(wVp|wiXZuXXmy`d;ztI4SD zyT>-PCwU}y_+#JitUb%hOs=G-eg9yKG%BFi)UwvS@g8KbRf|&2tQ{`!bGXDKY`Y@K z#D`r4Z4)Y?sOHW4CNyK>X`@J3WQ4&`;LkgW-(>No13I}A> zOnMC>S;daXsGraKcCbYhCq)YVN_jAT;f*^PPW^D+JYOC*Znf?<53if^OTX#p@;JIvgCV$XpxpT9n=hVm z)3xJf*KU5+Tri=V&wQH?9h{%LVfW_YGIn&-+t28s_dx{z?yngFvY)-(d<|=?b&)P8 z8PWaZ{r`B}jF$&=cr(6^=l0-R{OA4hhkt-VfA9M|ZgOaTjL!KG$Eksyb$r4S*zEA_ zuWvVBokQGbS0Fzz>*{ zX=Z&l{0CYPZu(_^-21uqZtUyMo}U|hPP=V8E(-qr+r{&LGp(G@UK)(W`2`k_#a^Zb z=p%LvF){FGwv}Ie!}hs5x7+h&z~X|TyTe`j4y%jr`9*(uKeY3^tGD$$B=5VC8`lQJ zh0u=t+U+~{qIYe3av>qS@YF-pMPA2gklwi8x9q{$dUPE=a&fj_UdA8(0Tcc4*TcyD zV*mZiW;pa`Sdds1$M)>LB#5?~J)|2>R>$yXyE*nDz=o%H!)DuKvp7f7=;L}W{{7YA zL{krcW%#V&esGb#{&4Dc{blooyC9}6{3>k7-N&Ed3x0_UdpOR`zwUS4aoMkLx-HgF z|1as#!9C0U7+VC(1=Gtw<bPn%?fw zmF}_Q=<{^??XSN4_KUB5fN=A3jN7?+!gDY^8+Y{LIH=?QhIk%bCM!xCck)LrpoSY$ z)bzVMOtEiH@>t?wwsoCAT0_wnU*Fx`-U4cQnyFv(?hY%;w`XM>i3cwnv|a={V=cO? ziI_x@6r!gsP68*Dq@_lB{Ml;lMN*2paS?l@9hr)rG&1nlw;cBI>fI3rFm7_1uS zoT8v{os_kNIo;Os?NI13wqkn>JKqd7Os%i!8iqhx7O5Orp0 z8P9&O#w1VGb?eSFFA&9bEFaiS=YSK6D%vEvw(J>D;Zu3U6v~BST$SQ=byuNphXEUj z*YwHV<|e?Hl{kzp4^Ba;>nLH2@ut~MRxeUj+{jfeoC0@uIGcTYUe<*;J?1E5MQd8R zH{H%&CduMTJb+jFmzSe+v=lF{zuQeBMQ3imbuyUC3YKQ2282^_yadjySPmRhX1qjJ zamdsX_N}mjVU&Hq8%3uX_U`Ry-QBgJ9R{JgJ}_G5C&$MZvHBR%JjKjlLX~hC@&Rd#fmF`8|3`P>>ga$pO>k_7PNXeS2Q620w7bSQs?V zc?#Xyr579!p_6$Qya~u4xib7sM-gjTgunhe^dsTspAx3|fVRig2)Z7O-VVE|E#$yjeT! zS=r(vl`r3(;Ik7^JFe{kFBw-+l*rk(uxT0jr6Xu{8JYyvpg6I^h73=^nII8nXD9l0 z+$Oc`g`sce=29Ie%jgti$H@wIDVP;bD66qw)O!Mb&myMuN2pK#aS__j2!JTWm9Pwh zWKQLjuW4u`d4Zyg@<=?elR;!cxL`G`WcH#Lq3&ABuuQ>}EW@7CU3CTc$k1EyY0P*J zu0$;Q+s|eA!P<)|Qar}Fe`$k2!d)ffwGZF}tYw@aa$3Xkd))?4Q5M(Gv}XS}}ph`FPdz zvEW!r@RW4)^bG6kWXQRsrjuEWgQaIbpaD z1ouiI!!`6FVqG$3y=MMc7aitKV&dJf$2}NCU_)gRFqxemUMhz+m&q~O9p4l36++t; zv})<>zT5Aoz;&`VrtcSM%Or*JU8wGD5DboZo@w;7HN`qfVRTj+q*{OglN3g};!AP! zzi&feyH0C40nW|$oi)R7jbvu&cC_|1$x2zkSCJ5EA6X&+J*N?b;lwbcm4uSp|QhA&7nYb!T==Qf=o)IAsh?7`jw3zGDG}*BWqzkeMN5KJFPY6Wk zutCIAGP~hSD$Da)fU*;Ih?kO}i60x-oOZ7<=9p$;VbAe$hP5);KF9V9GdYN0$-rom zz(_ZQ>kF4Nw6j9``NWvg9Hx!Sq!gK5J9baU8DbFSS*f9=L$EUA2zO=>k=pt0a+wW|G^6nto zIu6$%WvJeDtRSfVIG%?EY%+polgk|YvuA!*S>pClGp&w8J2Z50mY9VWl`yJnaqM|5 zZi-3gbsnoFFu{8vKy;^g=YsWwOC`!8frfa9SqN+AETv5yD zFs>LJO|nm+O&MB^E2^&5TzllZ0?;p_8wbaioH;X^hz5HUXMzXb_ck!-24lbL2n8=f zLzsI^{1=@8>m!^faV2U7{beC|9A&r^)dj!349yfi#y+ZPKdA8#nE($+xp#LLYlY5T zN1FO$sV!atm!|H*85w{pI;&t^qzi#RDQ5f(`;l2)WBa~kXpSjd?asnZ~dk+DJAZq-!XK${FbTh+f%b`uUZ>ep)`B^*gN}C=$T%I0YHy4+S>V&^N zH@nv57yj^{8{!3qcJt7TJ{0esZhMs zRPT$+>7L8V?8gnW4mZwTStMlokl~S2_`uT~!{T9LE=cV^<|plu-Lc?zj|1PQnI4Z9 z&YF*<5=gN5*}xC}cHbN}=k|DHkRA^`Hf}BT{#^r8ZTF4K3jK9YKM^DQIK-B3u&DQp zN#K!nKMf}XUYl<(%*EJzy?;HNx~+{@T&I;2TNw>omAs=v3p)1kxEwDp5S5!>yBS-w zEVh^LP=T&dDRr;ILyOICzr;hQHT1=EgN|XdJ$EPmmf_fTzntJNvCm&Oe{tE{3jGZR zYo6cDL+4yEn}I9&u_H>5ImCV#KZB8U;c6KDi^exU{HOgZ2aPd?^f&s`Hx6~3+e_|t%!Aj}*; zKU4ShQEdD91oX(~4CKvc{zN-Yat`8>6#3iDpOl^o)hQOQ$s({>(B&s?@+W@`c!{`* z#H+Xn^QP3E*+`<#Q8A?mRA8`1U-Dv*^QFDrJsUl}Om5af+gs$cjvHx~04zEu&+)lS!1ZM$qhlvmkxSkh3iK05gj`dxgx>a|k6CNnPL& z(~OV+U@4NkPzT=-z9EjvT7m&y!}bDEoyaKdW(&{><>Um=RF7Z`c^T&JO0ubUF69N+ zqBy2`Q%pbvhWG`LHw7BeT4a$t0rz)l?HDJyYN+4X7o>$l*N7j~97}l`7n&i7uY2mI zfPGR59D`PgHDeT4G?XRyAcoZpnI8*Wh!w-$VAT4IFD8mH)uC`fux2JA!}&6G&H?nROi*oses`PDeT0vrQ%!r z&e>{CBE{Z{#pOHp+pYQ8Q?deR(uLo!P!#ei7QEcuoyC1ioc)S_xwLkAr zJ}eU?0zr6vCQp(HF&AFO?K%OvjBr%TkkOyu+l#8!OeCME7T}I4J}Z$3)Ky+HVb5hL zK5%PS083@NjR$7R_-!oaUG zIcv@^M(wc2>6)b)`q}lDhy9fH4MR_(OOO2kOD)8|E8gRxmpd=9NhGOaBbJdtnu~=p zVLkWf5XX;P0SESmTqJ}=DmH04fkGZO401hZpP=MS0x1rB0dxc`4{XpCMujO5Ne4_4X+A|4a-VUC z#)T+Ahqj%N&~ZeJaUwo3bm85kj74>oB$a$Goy68&;Dk|ox;cjSyslEgSiSTij9uZ1 z&lP(M+ay>IFfprIVJR~XF~UdS9isgZsO+aTD(PgAGbH#}LhKjDWT{!!L-P>YQ>EE@ zd2WX}He4!DBColpA(4K~Z;hD>{T2j$@15C4~I{29;1`_0QUuGWovJ9>W_ zTI+h-xyU?|h7Rn^BW^#>soU<)-F7qd{dx1R_I%_Uj15n(_Tx_-Rjm_6)46vo%`iUS zZ^jpBfggXmAl&8cGPCVs9$gx*`|->GydcE8UyeGyYV)%ON5__&wGZvwzpn<2_h(P- z6oOq2p%1;{o@QQ2m`En5V}) z=J3!CXHt3Njr_)b6F5hvX$>*!kH-_{f8*%^!xtWpgyh=B@Eg3{J8dg`@z?tu=AU!9 z#mwNF-DCKrSIdh;;Ji#R=)U{o|NXzb-F!t51m4q(?G?L#n+L4WW6w-?&%kZ(^Q#xj z$Ze+0;r0CN^kG}&zn-|nZKb($iU10~amgg$3FPL(0t4dDlViXuOieACK5^9NZW;S0 zL2v0XPT%!|jrzv(XSm-y_2=m)zDqxMKzdYS{VRZMPGxlFtt zGiCOva4FjGgjg^CHStvY2_m z`F1Ges^H=6tp#8+nBy8H4ly#cXK|(kF6P-}!g)l(B5A(7^UX;C`$bd3`m;sfp}Z

pSn5bXnhp zM#Rhvi5*Hi6%)X+kr|s%+PI+oGjPs0i>6)}s39iUSJcKSghGL0g;Uu~LYGiZcBb3) z{^F@NT{d{81b&kjlj+!HrtFSfHNH8oG(u;U1=gBwk?&KYN#KT}C7U*9_o`*_&k0j>LeUU>3{> zCx^Cf5_6O@c+61u+DbG5nB7XXJb_>!5z(?217Z<~shGc?iqm9L$txHZwb(k{5R`2q zdXq{V8B+kIiiqiz6Lb;PtjGu{Yx2sOTt(y%(!BSJZ!T+CIGXM_M*_-<;Bv`qr{-bp zupQNwy}Ju+=FF?9b8%6a*N;^Dm1aXZmCp-g*!3&CtdlDmn<`UC#0ZtE3*nHGsGv&X zw@(bD2e&{;ie;rSn|;2JEk-BMlW48FeK|* zI(kKR#Yn9Z=H1;rTSsw}X`&=A?m&K>CTY6nexH)HT|T$udK@RWSI$tej^?W+sO7L^ z6*ATqo_~2!$H4<`1w~A;Y%3b3AJW5*I?Az~~Y4hT+W~4|c>4S@t96goXiTg!!u6*HkOqm2n zQyr7^AxycuF$wilHJlix%aw@yyJMhABrQmli;lF6I;2^e)Dok43g9iW{V9O*iX}Uv zcT=z4U^)e%Letjn?wXLXi~C*AC@czV&Jqm(;5ZNQ;*{=H*?vx!oAPwEipaY_!%ez< zxwp-imy%j`g$OzP)&e}J#PQ6G;9_E!KDQql@4V4ildIbY|m{L)pN0$P4x-I~c zkgHp(YaE*06ph1&D)eD*8_%VQ=SpL-WP47EJXOvb;g=`pWho{LRjrxi*s$g!vwI+GEvj#SP0QZWucs={Vv4wO5UJaWN3i`ikVutq6aKE zfJvc~$~C{AbYuvS68`oX4r^{5g=(l(Xci2hae-q`PrYlNVdDEV6CI0eAIN-SmZ{NF zFf&PT5v{OCJodwO3e=VH+WBo6#yBk$oVK8sLOYAM$cfX3pX1z6=3gEPg9*T*QgVUG zf98cU;x)xLKk*BvNd>h^g0I8**>I8!7h=n*;byJS$`UO};ZyqU2{FSenwA{0nq$JK z6<&w6Z&y_8yhfV8`5efcBbZgv!&4wwuCVImcJjfPkeuxZr$yzMp~FnurLM1=JhbQ8 z3W>9G<$3`Eq;*>?BYaf~%TN^k+u|=#6_dFDtxYGpq^${W71-X0_fe~CISsP zRZrGw=-*J7@tQ)CalISg zL=x<4Wix9Jr!GWPq0_PI&6*`XeNaSZx>9lb(1P3M@_i6uNtd$z)1rqywzP;6jhBZv zK15_rcnfEI+a6(tjQ1hLWKkFI(sxxpeXPJyz=Yu8Za+OH9d?T_;@i#FGtVx6Jyq0n zd5r(`H-9_!7gi}8m>G#@p4)M=WkPhI&2_}6mNLgela4I5VOh!_{vm|vV1Ml{xEx&}r5&zD_VZ@!T;wB97=7jA z9@?koXixKwijalre)2Ixmyc(c4C_D2NA%ct7Y_A4H#aO6`v~FCPh6GO$IG68DuU@87nLX*hRXQ#2N-7zTVk8+ zEz`I;yPRVel09yk!|A!RzuNG9{_6VCLcqH8dxn$I5&zX+x-Dlc+F=ByII!>jg>UfD zUmv}4;q#aNvfo*d8XwTl(bw>iZ#Q2&lO_RE?mnh*QDLjzvk0MoU>AJ$6UOOWCho>% z;@&SJb)R=+8<>Lhf4*s!G1Yww^K5tT%Z|F#+sBK=2+=cw{2Z|`7AOb!p?&n#cf(TX zx7*8!{=9qs_{C8P6ZMQI*p7OJGH9FVwovRZnPvV)t{KwT4&{2P%Oms zm?e`7jxNLtQx?EmaToZ2TalztDc9U4mg)$-Wrg8xj+*5OON@OmAi}DtWmQV=9*z9Q z6idcU}zZJOURimE7e2q?hcKSQ`2y-l#8{qlC4ZSlFwTfjD<-OscE-}%+<^Y zR$fr=sAS-f;MNa^YngGYA&$Gd=Pm>WN$MxFZv|%wiC#~)vA6TLGQ3kTI)*}<3H#z9 zRHKb)rLPdAnJ0?@y<{br9EdM~Od zve!m@dqJi-aS?@k$jji^NHn7VKvlvEP^n+x<~vO3y@{4vv=CFxxUG^mcOf;HC~GYr zvkbGpYH z74E)W2aB*!#jJiYt*@hnDak}7NtxP?wU_xy)=z*VqdC%C38xHOd&XK#ACAIPgdyr5 z397PEUab~Jz)4n2@z({3-`gNWH4A^;4noM;t%%>1vu0YY7AvgssMZ79j38WNiS%!d z`Vw2B8Ml6-tR!(kswQ!F*@EyQ8LVwdQJiCv#&K(bBlI7!}{98zT()hqR* zpGf891xU}IulVryp=2RB*kbW{ElkQX$rFvEO*sFO0he6jn2Q8!Tpg8?tuDi4{Ki!M z76+&EIf;Uy%2GVGGn*!}t*TPs5P4#)I26PXfu>XHEfnVbh$Hi)4v^jtS#iXb- zfp!W*r_kGeoX4DGwOUCK%2OM<7^MQr z7PXTgN3n()$5WtrgxZm;cqOM=iA)f#)gL_D%_fy5r4nTNbH?J-C24!*ZkZFJinprC%5sGJj|e=UiZ6h-1hG$?I~lvyP^oR4tAWeY>a$) z6KaPuDw}Ja-@pixMOC#@uVqTcfteUFGv)S5T-G{CtHM#^-p)^O2$&GJrU)%z5h-r? zSSv-(LB_On=d@nn#(9$#HH)zR2wDjeS*DZ{S0P!h=|U3ZppMxDretJNUB%W2?0SCZ z%Fn}SF;QA>%oCP!yS>a=*Lj39K<0JA6EN3Zu`=fK2qF>gW?30@Y#U}F!^SJtfGlBO zrD??!|J&IQRIe!uUn{e%^Wd65MA)S$B9=1^Loe2L>4*mY6Z8t$>7LP7WdKhC7gTYOs`eb9s1W$0=lL#s+C(BPNGTRT1{jJ z*wT+HVh%(NoioNux;_L^aD@)>jMCy|d!4F@6Qlfr8e}Gs)HtBm%F`I^pnw`utU%s} z00Dtsu#$E%LU;4+c@33H`uGy^j?vDL^6kQyklZl?^m^Yf;DfQFXIi17K7ubUKms|M?BoW6oNrFT*U*G{?EtV?{r7(T#6 zV3_MJ4G^oSh%;6bSy4z0+(7ne?;zUnW zH6g1^!1l0ELaDoG=DE+P`AXIieLdFBt98ZDn<*E_OBX5H#saqCV~F`set`ARhNpbAN z90T(K*Yd<<4{bG%)B}g@XaaHOvO~*7`@bB)^Z$DM`Lo5tnR~m#h2S}9T`DhbObeX7k1ClWOpt431g|Ecgq#_nT%OyfZaaYow(URt z&HwIjD$KI?`u4av4URa3PxLI{Zn#eFaD3tqNC_}sMttpf*>2l$e8h0=+DCF`1c=DD zUHjC|d<>sdRsK(Z^S4!1e0uLV;5`n*Ay|m__<|PVLL7(G*?qkGZ~W%#UGHmQdop>~ z7Y-x(5f%hNGB(T&hIC57Z%AY`FA$G}@qi;Y1{A8R(m&6}|L2#dXaCFl+rDvTe{y*@ z!C!3VKiOTjj^Nz&fL$gi$GX;#P2bhCUUu@{o+ywmKGq7tdE5qzw%_86I$Klil-7+d zQ~DQ7rvIgf=zPYB#e(@of5D6X?N`5B#}~!P%t{e@_1x?NM#!O5SZ|FQ-u)&hQ1>$i z#p6r-nS>1zHrzC00g1M69M8i7)#h0CX$fpBHz7=Ks04t?az)n-0=4M%?FMoA`%tmd z`^|2q&v2Am{D~if-+lGf{pObq+PCh<-+E-&!!J!{K92C>uPU$Wxs-%o~=(EnqQ@WM*+B%oF7+uN(ZX$4YzO2Dp+dW?_)K zKdGuZk;^Mr`V8AB8RuNd4Y4%#A|_5(Ld_FoCU@CZA4IJ{O-oh){WI=@SmAV~!s^bd zzB{%?@#L-ondBRd*^ba8>ECZe9FW7Xt&u% zmS|51?X8w+dEzktqe#rSWo>o5AeQfY-v!0qWwS@h!iMRcikE9G*Jm87t`+J^qy{@b z;3G?fPnCFb7f1le@2hlWJj8r+!U9E_q4A?{I=saf+TueD)jFyzWuKbo;mUR%Y>ETcSi%8ErR44w9b zs)o9){-0wX9H>}CyTQP}Mi2$bC$b@1BGFAv3XT8k}V&#d4y{bYR^yZcHhcFsxRYV&8!m;((Pv`4-zCFb7oW zN>z~sH;7}&uM)+=_+#Ho6jdnzF?Xj1bP0A_B&V)X+k+xHrSKWaz@!994 z!cMO%eCLD~X-LkX6E)Nb^$pB0FA6A$!6Htd*IAs2A}~i!^E%E~ybp8CFlWNK`g+Z# zF_DP0HtHD~DJY#aEgrxk+nlL$86=@p(6$UcTN}l#w zj%Go23C**vMgOX@&{#!!|2y_=FTfl)4UDzH=a$DcR8u7ynu@&Saan5p-Z^g1Y7rXx zUt%b87Q@#3wwaV20`wo=damZc8+y1#2|CB_vbw-ms`qh$Fl1m=aWg(Q*p4-p5zZCN zN!N$98$w{7+l z$|b|p$~RqkVza0$i%M4U74Ry`W6abC4$19;Ip0B>#ax!0_nP~Wh`_H%Mgw`meeuf{WoJoll z&EZI{=sK5(=@b)9S@BfP;Jtjy5ZoxJ@$>p3?u8^#RkuFhc?*uc6EKRjg?*Hf;Q$jQ}-((47 zVjY51O|b<8GQm**(^J;AIgw2UO34a1#VhW%8d3~QBv_{<9Et$ONF7yi9IzY-dFJFa zw)?|G7=4zv>@M#DYe}7^8h>lQZ^8rOtW@aQGT{IcxgsNLzTboxyc zXZz4SO{SZ}S_!Gx1jLUq4zPYTG!BKhJv2{&Grx`re3Tp8ek19Jg^s?eer*HGIyhta)q@h_-c zGUA|xJ@f7^w`N%7#me-BYmeB#GFnMtoYPifQYHvC7S`@?o+bD(D*&nhUyiAGr1Fr5 zcK;MA#l@+!LDi@U?;u7}{1b8PZ?>4^Rx7=1x|wYzp|YT5HD;JsjI*~?*Y=bTMv45o%gb1_c94aZ<8Tt2&tk;~8qUSVJhp(8Jp66YnK{r&qcG~;Yi=WTl? z0r;LgHzwztj=BOZZa-k z&{d#tPKc$19Gkk7=d2M(u&S-ZA%^ySA9#is2FX)ZOIy{^h8Q|6YTaE-H+pbkl5(?Te3|`vQNSdf^jC&gM8q#?8-3A2NQr z8CuuqJ92*MH{*dV!-kXqA0laSpZ$${DE|7h;fKFRk3O0Hu>bhyKm3<-OR&V|cl>J4 z;!aKpj%;veZJzr};DlKg?wg|1=&EtII4YTKKb(4R<1-Gw@rCC`|15`gm>55rL(9;% zqr>VPiJ!c6!dypr(s(|P2rq6W^IZHH59^TZW+B+A;t+g}|GyKojv z!85MU)8;c*=IJsrFiwE5|HxnTm-i%TF)NyL=yavsf-(PzRBaq(qvm%*%KscTQ)q89ZlC*pPtuj1%thx5 z=^Edh+h)toET($N=(50R{PyT2C4&6$U=ClXg(#N(blFcBmifJ#&!++wBbV_n9DS94 zANoKrN+sQKzj9C(S19DMzhIx>%NFZy#IbPMKJ$>lpZOTrc?KN9g5umdU$*tb4{E5o zfMNRzA4Bkvcl&$H*0FKrP+-QjEykpA$D*qn&2K%oZq`GWTzHi<9x?ed?9(x?R zLHB_JRADXbjLF41LH1aOnEC^~L+3wyWK;B~ATnr8B*IGf%yZ4R$4p>ap*()?L*ulH zG9{x6cnvZ=D(CwcNHX#`&6V9gAQG4x6)FBcoHb5{`AM0;_-E>Rmp3G?q$h(s)MOrz z`+B0ma5z|rD_*gTWGRa{$p=ix0EnQAn(A#U#Y?bg*R4;KtHxTHF(>BFWorctY7#4o z;v32zz;KD=40b>klqk{Uvr{)3;DfxU>d>EPN??g-eaV2jh*=_6^x6<@ zV>>omMNchbwpu z;oTa-hZ1Pbuv3-&(c3IFY(D|xc6n!5KJhpE9E0zxH>CVw^d7U=JMyEtI0fzNI7{B zBJ~|n7g#`BV0dX*bk;GwP+TUY+mOi%vCj&F0@GKH&~86fyCD0@3h_f|PqZBP6i#Fj z-b7NkV2n|%ozNc@PD(A-J0%psCd3+2JJzrZWf|3xhWcjB&r31Nq!B^b_z%IkQIa`S zX~X6?a!G!E#aV>NWr@$w+Fa_V3>(Rj z6~~!1=VhU_Y-SV(6XatVq@XvNNJ6L2Z7*jC1vy(1B@1!Ns|i56mQ*iyHHMlYI2v_5 zEZ7*($wUvfnUo&tCr)08j$Oe~SX5k;2(uc!2=|9P;|knRX1T%|uiW8zQE1^F8YsTi zA%kVnT!puDsxzARjaOK6QfN*4U1zWb7J@2kg=a1T)_IyUe0tl5Db_&}pEX^@d$BQ3 zwE#arz`vfjW&nE$XoHxJ5A%f)MWRWwu@!~Qp>QeFz)gYLoZK_2&srQM$HtE%0fWr} zOuoKWo}gE5KG^PQ6tBG0&QlNnucuj&F6{2b!4zx8DjeD*az3K@x#~j_5vR zuBvQ~=ZiHaYxT+LYM+#@w8GI>%g$H=m$BL~J+fpE&8wv~LNjDC_&-YmlrpHHD37_u z2Z1D+@{ zp`F!`G-kG}TnN`;)08#XRW}aR94P?%OYM2O0|G3mCd%srrIdmUbab)UVQ@Fcz%P@f zv4-96?hHkM&(M;SFtM!>Un;>RuQx8>G-b(J6l>k=0~U$+Dv3c|A=(lG@3qR_a0#`9 zFybqA?Gn#a)ODGP*~AEPh^aw@dD*J7SZkG{3lcM56k>NGf~1+ejKL;O@;WV40qndp z*hJ7>b;bnl%OtB|Sl@QVdyY+@SV?%F5RD{TinU<8g(^%0rcWs@0JEwmBveG4uQ+AT z7prpzfuztVrA52ENLp3Um&LePu-drOS9cVytV<{*#bj{t5kzK7CN5H}@I2DKcL9<` z(m7Guufwv5TL4;d&bh!sU!gG;GHgqf5sTEWVwd+<=q}nixzdZUn$90DO+Z!=dZoDZ zySu=6O9IptsOKU~oWwa)1{s`1Ex!#1WRHSTt?-^!(5dP;_^J^oJE# zMDv5`GzI~v6|pGohMj`Wn64n+7LhL#E9y#P0W1SwT!qKQza76o#V z!xPs`N>N%03swgiocgzG0jBGICS1X0t?@EZ@Mp5R7`kDt8`j5Vy+p|z*p<|jP>YJ~ zO5w9a?#88y7|CMF0-moRNzDo`{{+Q@_L(S_Ye~8kF`G*c;4RC?h~lwVw%@A49=iNgbW?w_?^r zR-U3Om;CIs?Z?_m4}IEo4gawaU@#Ju}v!f(*f0 zlanQ8Y?(%RK~g?twhZsdaRaZg&QHOXX;I`cbfs(s;&)u01`&r@Sy882vJv^yDowOp z)>-ArM;dsss;@H*BA1`Ef(>r5P8YmmDNEXwqJRZ-UX|kpl)KB_l8etuinNZ!ZTixT zQ!_tGTyf{isk3gd3Mlk9<=65-3tZ-o7DDd~dj;>5SPi?z@=e&DX~EJsDy=zZQD*V? zUomJ#F)5)HlD+MR<0Jy58cK<5>&tN32XeJIEuC$CJ25j{mW7LR&4!*CVIx+diwPQ7 zNc`87Qv1owr1Eo2C2=bMj8>j0Upv!9km3FeDVSu2w%f$85;xzT-)=sqX@Udx(7t>;Zhq1b204->%+WRl zN+uX}7|+o|xLohP8TvDC|6^dSdgr3?9=y2ikB{gHNpBpW&&|(iXq{Vh2zbo%Gd`iU zdHd>j_lBCnk4a@^CBVX0b>o5mNys0k6~C~j*o59+O?F^IB0VMrxBI5^n85A8`d(Kg zM4gc`Ig+q|$?6W|a&&aD>nV87J0j7DCkU^15Y=CkWC4FU_Ae8Pb=>@}AD;}$IBlLVkN8Tzf!w&+ zIQkjccEi@f`~DrrX5f4f;JFy2fTHs&9iK>w&x`xW$7YLric`zc4?K;2hf#rF;B~L? zyj%-K9NE@e9AI&|ga&@mu=X4V;@4RB*mv#aezSe<+D8^Q?3krGvqt^fuSgrkKJHqt zr+6Z^1I&CI+6m(L&^G8lInlz(3>e3sw>G(tPzoFt*-$GJH2JwXwR;EmbGw{otfx`? zwjUli{Z=<{P*F^#Vm!38M#!n41rya4^dYUxgJAy-{3)lhCY8cih6 zR4uu?yFB{llvP@VS|gO`!Z+l~9UO3%Xs-w@Tj7NX?W`LiHr)aomt;{Ci)OZf<|UP; zHZO9CTm>^&{4&%bUPJ-d7^3!k*M|ygDJ$mQ{+?HzZ;q5w$)5)wdA_G~C8uj^i`a|g zr6e)U{vyW-RjP%JJHcAhT%*J9M z1?wjC8e2$fayhb_01AvUrPqy}+tjtjT80D~w~6Mk~y8OVh2w$*F7wP?vpc4Esl7 z2iIlV;w(ciK8!?mNI6(rYY0N>U`~X!vm%}SySvH9!H8*^YjR+-K!MRSy~5xz)#!q1 z9x0M$!6R0(u7zfoJJr_&cnOGWRrrY~b(!Rk6O7;~GGvtEQI{mzS(b)l8GX0#-(l`K}&HYV8Uo?J9V zA?4nzGm*5^=rN8!AG%n|vfLO>R|V^ORmcP|WIw_pM`nkWL`Abpya@KB;*d_0 zUE_r!;vIgB^QNx&v9hV0B5e|G#PlAF4K>Bsb++Q3ozTsBl`)M%+(=7L9(0LVwUNok zn={jUt(>H@wM>N6V_GBe3R#sivs_dD7twe+6jxYEyxQs|h$k01x%bwVWaPU_vb%s0h&$kooE^C`K0FD$pVAP_dER_8xKsnI;3P;##i9y2_)tGgysJEcU zz!Ax0T3LQKHHfmRb_p!E3!TP>31pl!nUZB0ldQz;pf#Ij_<2Si2&J!_D49x@LS0S7 z2%&yQjH;-G2|k$>1%tT)8=7ihk+&p~^W4^F_znzdmdkFuJcTl5I5x;3H!ZczywIOFKDY^Mw(2_eJ!5Ng&Uu6%Sh8JCTD{eBG-|U zQ7s^XDei_;yt&>16C_GCCze1r9x#H`xFUB-Sfxz9c<*y4G908>1d7EP0uF-^eWmW5 z5ziS@$4kZG7lGC*VqH&JrdRA@jAp3reN(74tzjF*g8FlwBNIqgG$4;iHh~xioD4}OSnIdNq9nCASGapNsu;U-&l*(B(MAXs{ z`EUt0l$`%M5 zzZ8I463H1D=T&GX26OFlwDzpjI%(WS-M-&6=KDp>ACqP5q7 zy)omG^WVp)bW3vH4oiX@#} z;U*F>p;l}9F1UMeJC*fHjh1C?;3P}ej;Jf7F6)xj$r^ZILG_jLGL*>HBKra-I-}9z zP6&m`SjAj+?gE{{VoP-?P9o>93w6nGLajMO3Tr0wpp2`4ermF-%Kp0$yfR^auo5d( zO|JsT;rrrTE*=gt1vO%@Ztk(pmy#0pujN!w^mxsu{%t-oRZwbw`=CT zCzZvWS69`Voohi9lPV(|2eKC4TPtBmE2Ogr<5wc=RsCHZZh;f8f{Cn)Rq9f)g|4v6 zF}a-dn4MY0Ads)|AYHKt5<{D>a9`e{vaz(1M6LUC^RDd!sV@`=I&idbWE;wxRV;0- zdX0|-T8Hv7c+;tI1 zFF<}dbxyx@0Yb&te=cA1ggv*NWzr;8SYJ@K1Lsx3nr&+9yJ{!6vAh%!;Hzmzp^#e2 z3_}GiTmpc})mqk*y9&)DLlc+gbKlP!B!a!B@s07+ZqE%B2^)Xb;Ywt2Ug7>C?14i4 z-Y#evBjwJJ)nl6^^wP*XS|{CnEM*hJAqmkI?0K1vN2~3`S)|^o8cNET}#q!z6V9 zKvHJLD3#=Q6fd8V)T24iD>3<=TvRn6f|#wJX?NT2yFdz7lu0h;{4&_=#d3OU=4{HyVGj%OL0&=q2+_;6s@fHfID$43RSLF0wL{kj zHn5UZ9aXiE_?KgT{=bM)dup*M5dSgb><#OaF_e$x=%6#%J4R*B*@N$=}8IQ|UPa?<{1#wHLnK{Dur$2Nxgl5GSj$<+{Lu9-GJ3_@U0Y z6s-cEgU*mdX(3Butr;rl-ac`s`7eNS6%a4|W#7KXtdAdHulm{Gphuria&I?ZI~FdU zb`Cc(0dYJMDJ;Mr{{G3MRsQD-3w*!%l_w8hqFa847CwA4uk$aSo5N{*^TU7Izm7lrLwm!W=4XMG;_%cT{u9202%eW`{Ez2F>j%!- zo9!M7!p0IlE-vF^s$kb#_@}4Mw_HPSKEuIB3P0nx3#H5g58Cun`({FoxWD$V4sW#! z(|+o)>#g{L>jiOXZcutRqeJ`T7~E!fmLKaG_~zc^IsS;(V>LCX$D^UFm;u?3J^p7q z?qRA@${PPh4T^iR(5$!FLV4FSPfQk3ZWJB7PWHW?N9yQ+R^n7)08E3s?^P$Kk!W=!-;(R`uCjyeyR+1j3{4o*oh+a z`>ipd8xL3oRZ^zriVX=uC6=IazC+gt&6>q?ch?*2h}aKV<#SxtagkLkEZQ?aKwZKJ zTB+S&0UBpPE1eAMLL25MuQIK)vqW`ZuP17z`D4)wu%w5idf%iaFj2PUVcWgyDieV&VmYC6d+D;(B@XCQyZ56!RnPA1J5YtK`~ z4f$hxQQ?m@W5i!FY4~1Zo+VYD%U3*4Kc=%f5?qhJenNde zrYDOca_#kIWjYOrJ~q~f(@SRK?#{R-UASF|Ad4!1#nM&S^jsDzFKP!pS2fa_vCzbE zxTxJnN;g4aeBX7}J$#$yHS=_{x953TDpn1LQ*7;VS!*S|*+H6<7BZKOX*Q+FWJ#hI zxIjCpikEWGH{SQ*C2+))N+z~-*0o6#nMS`C#T3z=DnZGvz3EDV{VAg|qX(F;r>E(X z1TTpb4T1)5kr%ebQj7`F1F=v1WXl zsqaOgd8`nzUYD&a$t@!1kyC>f*f>@JjI~b3RXjH(2`$>kpEh<4a^tfMbfy|p3-F!nqwSoYfOsP4y9=2Xjqa?GBlwF zlUa~9UU807_{ReVna{gdQf*_8f_5F@n7(ZfB|w-74s%M5*$@cLA}E;3FkZ5u47!L@ViW^0tr~wj}lJ85I(Z7Fht%WA(KKa}h$83gs zN{-PR6k-aCGgXeRsR&QX`o1e`djZ}m99ycKwnJmhShuoSXKZPTyC71s;Gy46pe7s$ zoG!f&0zR5dWU1PWb5JKG^fQ&!@L~~HN6jEJxj_7;X_;4%9MMbk4}*{^73~%TVJv*8 z2FcxBSS?9ak#cMO_%#ty$%~I+$);m+ain*byD$3JiHxjvOe>;?gfM_f6~y&^xa3&M zkyG~dnShA`v6jR;;i8@KU#*lxQP@jFrw;T zs1*qXOJRh8mM_WkY^6B3cPM*_3`c@QKM&3KhJR;ur$~v5U>#2-4C_>RA3U{S;30$k zjU!yJsivX@WEsRt!6oqsoQBJ(vNAD?BrOkik>0Yxh_x54PPC{iS6Ik|kC7q2;zvpx zao5_NLhqOj4L#9ow3efp_QaPJYOzjHgCN8d+1ro4fq_C3P%z3BOA?ECmtx;qz+*-EXy#{dU|v8-GzUUdkA zO7Op&11??}+UcoZb@Ms>wh2(79Ea-)qsx+DmI_;@#3G|%t9Z8RC4YsEiSx*jD9SxQ zF=j!aQug{8>kwz8-4T!BO{H~5P`gq`A+(ltqQx?<>sn&mFeYMwjN^pwh_i$VZ0es3 zs(4IKgt8qQ@HvQ(BuBzdSm4&qDtB^BdqZfd#K<&A(Oed=nK)sZn;@{4=^r?(nWg%n z09OFnMahao`-)UQW?_I@j-3#SG#PF{QR|2Hp`F=^qq-_HtqeKk7f1AmX_SiYS1SNm zXRN5Ak)m^N?Nm!;R2IyJQ1;Svg_HO0t{=lQaGqD<=n2e{XvVG*ma^c&4n_YeSRu;a zy-4$jV1<+#jey306L`Y6!AQ`U-!+y9=5Q1%Vz8OZqC*6I7LaPchZZvtamD#65*T(- zAQp?${UV5JaxohCE*ZKUPWrW+$8Bi3@1*hWtMJmm3dWH zvF6^#Wffq3xO&n`Oy5O4NVns##F+_1I5cuUl;@jiGY$ubsIV!N>vZ99(U zJR$Tg&eXMQ5kV$jSISBn=kSr#%jR5+Vc z*lW;K@rr`-dP=GKF1qrfUo$ zlteAWZCE1{$uQfQhKOlim_*8?BI~Ky>^`VMo%;)=_LuKTM)pJ+;J2Hv@fFGD>+QMs-vC=+Nk_IHJ;HJ{E~t5gPR-Cjg~Ok} zyBtnN+(a1pJ^9ss){yB~Tq@tb?%$J1?Qu-R2cU4cVA6Q#&_SBUH3rC2Upe9p$zAuqWJD+*!t9%(9c^!GG7CgypaKiH=8~i ztiYrD=>kPa>fJ3#-U8zEe)Go*%eL~3u2_ThsAW+!PdLb82YY6Cw4ocnb{J92#BbVR z;LtQg<@y;t_^sa%j_FXCw&|H8jgKT`b7qX==EI&Fe7RD;n7FE4wUlYVyyfCHh!4*y`-h7VdpIB!uY@d@W;EeD3 zsaw5Y;H8)Se?t*c==_zaccmmDKnSrYYQbt(;^8)+ z7x9E-h1L3IGbS&AwHmKM0zJJ~3>~Z%AKJoe?2DsnQArO2iyg1Jrit*>`Y1pSRYzW2 z+8X8mqpF<(kO8akt_}9#mMAD_2MGQ;F=2!z<6YeE;)#+2|-J~u#ov|h$Yy}T? z4&@zdm-Z&;nh-`qYo*!cGZR@^6f3LDpi(uf52q3UTfm%{(y1%`3jR|QTBEEKIH7$r zOv^M8t5Cg#H9bmV3@n(rAdn}y*lZy%TSv7^NxEr%Y9@-}3S(J)h*XfxE31XdqO}$< zD_1JTcA@`w)tXpY1+y9CcwOtM{bWFCA-lr>9BW&W9y{0Ui4 zS6C6l73qZ%nyhPgRlsP{10rWjyC_fr8^2g7*4K350r*hm*`IJJjN+`&zt4K0m#Q2s z)a{Ho&EEuQdaM6&>667}P?XBlSJqZBG)FgN@h&k{CDUb<9R9)jh|4Te-wK}Zy$EH( zR7edEUdPglnnV<8lTVTCqKXtWwE*DIBbBZIvnp$)KS>LxO=VMKGPN*^sf-%Q-ThPG*52Z6j= zSy)1iz%r+*&Y1P+C8{b>WZi-^F|QKMPd{I#0;l-7v`=x}yeOGEdwU1A?Rm)%7i$fq zcFS2emv~NIECKKz%uSTzC5p&hW{s1KNPhmVI|o=_R%h7?c7B0e#>&lC>I-9=w*kS1 z173;5ycKQ&kPu(iz717c<9dyYXvxBm$7P~yTBMzw1~bNiBN(^#+=Lpa5#42~6GoGDyz)N6v zYw56MQO%t6p7M=TJeqcrB97g@xAu6AG-#0$gG~{ylul_vIzxtUj8`<4cHA1whkI&; zyXvOWDa(-7D<6{EV7qE;a?M`>=5f!H5L%PGt_lEWuAQ#RbvW zKUx}++y^V+s_z1B2)cWjEBayb(RraLTj3gR-ggUjG1wrr1}WxrgOs(+GzD!$XRWabnc6R$&T0uttfOA=RMP z`}RI7FG11+)$(V4X%+>|R>J)I?(kp&c!8qQ%t2zPG-4WJxoht3_5sj`YhBLxfB)NG z{hR+_?XZ&+bT|QvQsN#ZSl+LZ}v84ogP@jTp_&NRpnO=&=tha2N<>A~Y5{DDQctm57`pA-sWw110qR0=*!p ztVu%^GFT9FFYvzU+?`J0P3@^xSbrI&L~5ADwNh9w!omS8N2l9Lw$2hJ!o;ub2pJ4+gwOFL?V;2XVNF$S) zB$cb|u+x6whSGq9FKA=~KxB9&>ZI_7&NW99b zJf0>}JYVxuw5Kt+)gofZ3pJ(pyiRNxQx-D6`+$jz*Z%9Z-g zQ_>tP;GBG*Ylo>V0LJ@x_{c(XJWkzIpp^8+KX!pl8z754OjnGT?kQkF(<)=YX07Dz zR5&#)S&3N^w}#eAhI5SLR5|i_`4dF)Vp7nUBk^fH^k)1g- zxItFFEmyD^$Nm8~tlep3ID~a6$NOy(m!mnv7;5V9vQkbI!CYXy0XasN6_m+*PE3gC zb8Y`LG^gj#vn+9F^AE#%v2Ac6M0L3k_^+wA>HtSGi(IIb=veo5xcN^W@Dv-R+=A0^ zM#Hg&20Is_2GYnS0EP_vez?zV+w@@!bjkd&eN~up0 z4}LbBM=etM>z204qHdkL9zPyH5BEh#roC4lwX0ZuZRtg8X!e4m7-&uZr58z3#T>gwhEe zh&I2mB*V>DOK9W()Ayx$-O$jUk%qW&JV=l|Es5R5VSeOz884?}KING3IwfhT%S4)4 z7(Ifvf8RN_%~0Ug!7_X$$-ld~YJI0pgw{1sY=+D|7+jo5P<>l#zzh@oY<}!}N z&fL#P9srVr*XsndG88G1u*OSAO1iB!ygz@B@&4(qF=RRIjS3`O(Y<(4@%BJOEVb4RbR!HP!n# z_nZp`ITY2I%30T!w?IfinJ%|uz4EfA77Sc;g-E0gcw%R{?>s7>R*r6;N;2n-hlEKu z1Bw!M#e(W61j1Nw?J#eV-U{6*6tN_I%m5G$71QaZ%#hZgt(>4OvNm6NgkU4F75B?6 z0HNhjGk|S|YC>I(s#i^n8`GNYRO=tW(b{3Qbs#C)bs~=o6HZeaz~6BWEecJuFhIJLo4*a*c~mF58x!s zP+QILKxF+{KqxZ4NzB?a^_BBV{5aA!IZmXtzr9`_zpN=ouIHMXv;)H&=bBkRIDIq< zzeDXp#;b6ykEO;nqsvoX_Z=kJ8 zPL}ZZWm|R}YbHp+(#jr{QFlzd3vVdv-%$1=dUI+ct~s|9>#^ocKp(&@7|QKMvx5Ls?Qi)x1et4h%=|vchf@L@K0y7ps91pw=zx0?R4ohryAWDHx^nQ0PBw zn3(1e$`7(Sxac`EXQI`W2|gn#tdoGS&am!WwAlf(Od00{;nqnkSmt?!cfr6g z7sWt0AcoTIp~TWao|<>MCH%I5Xfx#{7PJ>)lKEJdLZ+ZoW60ck?uBn;q1DtT!MJSk z%UZ@M?IRroSPwf|Dsu>}Mj1)nSZHl3+h-*0LzJvGC3J-8^G)E<4JcVMDvb*W9vT{G zi*RAMj+=^`oKr#EvVpcegJu4x!`09+aRL$@I$rh}88+FXKv>~T;1y*WR3tceXxlJ`?o2*Q5AT`ueG!+5ivbqs3&8JMnK^E7JG^oH}9vBFQjt?|Pv@nv^(Bf+W0}AB>S`NqQ{GRNA?GF6dW-zQp@s z*ZUZxQsq{dQ82))SRD`^WaCo`YJshdux&Far)nYZNlU~zfojl~;5#7SWjqDa zwpKoKekUnpRK8x&8C;qK`jbjxzMdA)Ol;VQd(W*WOVvU#Y8gwY2^eot!#$#~N4)&U z5cZ%kq+wo7Wb||#bZHEFU!6BjRxc0$_9e4S%$=bQa@eqQqIocHVL@dR%dKmsjdbZl9fu|uc?t5dte>KgZ7+Kn{m^c;I`7vJ-`F7)#AjCQx~WWqqCRgx zkNYrQao^2;#x@#O*4&Oic`pEZ7d;{UFK|K5hwy5J(riCgJ_Vty-I}~S35j579 zrY{LKnaVr^hbnQ^dp`m->T(YJX%{{LD>L15RTaf|=ZXX^K?`6=$LA!IB{n7_7O~LW zZDo_;yvTydS!^IfOy14|>ZmP;9271}NMnzY!-HGL6VezEJfcI4 z7UX>_|B8v6m*g=$C=jFWOuHHtRT${YAQRlIx*VfL8n+weP+Rm4X!Q~ z%2-Y<+C)5F+sON*Jhv8y)dbQ4bl5C3Td_Tr40*UDazy&1rcGuav^h$s*tFc99Exvh zWZE1_w-#l+O$~i<+6BtKv^tJX5k{DtkP8Y>SrnYet&$ig$}aQiiqJtvu?on;SYWh} zuP|Joh`k^s9MqQ17p;D)NmJL{wRk%vXJuLW(C_j=Ty0k^AbiA0iy7!H7J$)^o`J@D zWmQ@hX8>S3XjIFlE32IghzWP59puEOMpycL(uMRoTo*L(8_rB>?kOHNeP1anqFknD7L!!|wkr{@Qc*gkItOr+d)q=WR86{} z;pF3bdpAaTZ?S=f6G_yB4aY!z3LKZ0{?|;hiSAy*Es`_IDv+fdjsAEJZ`RTO)GTYU z2{iP`Qi%V{8E|lw2Mc(;x!PIEX<0+ij9%dMxDTC=AjK~Y*R2*TsYsm}@}5yZ4hHgl zvFO&oRp(P&K52fALBTh6zDKWf1l34--IFR_o1yOl7xi@#P^HV5Ezk)GNhz~O$!O~B5U@H;ZTsNCj*_&IMQ zpcL($bl!T%n3{-pEc-QeegjJ6EyO(rvnUr4Q_x;S4S6$V0a+06!iNw+^m(abZO~MiJ<{-C^ZXg;Cwz zQC#+a`SY*G?HPzn+;oudM$0U2$0GpO+W_D5fAOt{|BV+!b_sZNA=bNKMl2qtO<cjD2^CI`V-vg~ZPo7dnqKKA)X8-W&GwhQu#Xn8c+dsSt1d;Nr zn8t5MeZmE0B|1A(CFqy%(jIn4K+@dZjX&-%2OxFU9ys99c1IUS9si@dkVH2_@sHcJ z^DZ{UmvQ`BNhj~7(Y0p}q2`8Qio9ghCB{_4>!4tY%cF_>HrC%MM%e7FJ9@!N6h{%hy{>$lT(w?2HL zOTZdc0=K0+&_W6PvlmQf>1W}65bk>iCxxWEQwF!!5)T4vE#Ht|c{8{*fRn=-(?e8+ z#a)shD12{j6ZGZxqkR3v_v@9!Ejv8M?fQNrU-#NS|1YQQVJ**iLV2h_^X2@S*9Td{ zkD%;4MZVgbwd}aAs^z4*$Mcg++I8{y`394RD~>mikzbu~hyFJ_r-H-4KH!7Kw=c`~ zd+;KkJZ9OGtl)kfc2hpr-ROuhPgoi!aEi*x|99|AipSGrWZ2x-{>rId14_Z1els;= z+kq^Z$8oRZOxS;9PPTbfK&_8izGEAd{qgItR{ zPGP!co~n7{4n`%zcRe-+P}0tFvGwX|TyFy(WP`gPgSBfe1(viMm;vfpPC*@pVk@P( zMrCM)h&q!8K2E2=71>lhYib0q#GBN>^-*`&jFvT!2&GWsaSRVAVMI@<>j3Kl+_g8X zbE4Y44h?I$^(wL(4ichw?FZXY)vz1DY{YdQ1ecwz08QZopr5sLe zf)MopIZDQE#IraU)l4)XQ=#RZ0nNmWYuY<^!&?|raY`*Zb)JG29pzBM+^nfW=sgwD#tbCJ1D&7pL zr_9>0l&Wq_Sy94mAz0qkRm+J}w>v8)waziN$r|*dmHQm&d14*Ag9IIGZX}qEPD1F0 zv;4E`>#SOF3mPW;S#TrCmr$~Rd5ZFDyUuXKu>6#c+KyRae9Y(#TEJ&np(o9WgPNYnb|wSdT&9gz8&I)*()L+fBCf6G}( zW$$Yt!cx<&HEW*(pVP1ib%I%IhQ6x|5!zfrK0%n2p;S63Koxq0On1A7ukQ`8Y0##F z2-6SCXULu9i}+OS04d9CJi!t2xMy(vZv^7QiOarzVexV8KyJ?1oyLD2s zzovn<2m#;|)-E}$Ov|ar1hUyQrGeIZ-oaNBQ*sAXA_)xyAN5uz?J^Cz^=Y#~*sa4> z9+h^_6kH<*ff=YUIn$dpC|58T!AD@;0^TYu;RuasB+6;&jsK)$UByYb@VX;58kKCC z>C0*e`1Iw6GQ4@mpuIf6_vU?IGflCQSod-sFR1E1u;EJx980P%u@A}#&!?1z9at{uOI-9!R8RorcEkn5 zplan)HMDvj<1o;^w7)KagL3{ezEyM0-lA!%vZpO@k3^(5sB7`b`R)5%Oij#C@-knD zgv$p%FtwaSdQ}{+IVSh7Ho!i?4Nb>zz*|*_NEz&tK#Hv_P(Gd0K9vgm6D&DGSf8q_ zKmtolX~+(G0}uu>QAbF~242enKsR$EwI?EF*@Qk4Gp;X8Pa35vIiXGv!Tfuf^(=k3O zgC`Ytq2mw=@vS$Iv$dob;PTu$E=wk(We{IY(|08$uNGZ!V8lVrkv@|IUu_!%F0`Lx zA#BzWUkP&BOn*~^adz!sa+%W6Y zVdp-fpsQY0WyyhY%IOZK-GPy>&xdsYa9CGbW>JHkYcz@2B60H2kV0PT{t&n?ty39( zk7FFLm_R*pUM!(Omc0^r&O9B~@nOAJD+$jSNDmOB9|wq|C5+MxS+Hd*;R@x~WgBdC zyaqs?6Gn=mbD&b%B^TIK98OFwFz;EQAR8E182Yfm@!7Y`R4bdwE|~~ZB7t(bSQmIL zDC`Uwm`~T~I7D*D{Z0u92e@j~KH($g0Q-c>HRIr?00DuU)pO6TPoYPS$!_S-1xIss zGbO-_vO`9P^MShqn$!dG6Tr+23O7Se*p*PbV%pb8pxIU+PNyAV32czXgrr9X$rw;h zWs7_&dH*O-hVe3!6(Ism+2kY!Aqe<11^sfU-7;muP1I1YhFbQ0T3P4Kj90D1Uu7vF z0ExpmBJmuJ@&(yv1!k1@BO0qR^BJK+?#Hd$DRYmyiKLuE&EyNm@hMOrmae5M?#Zu>s~aqv@UqgII|Drwhz4 zn&lzvK+D2q+SCNwOQy!P*_`n(1uCp|W9BvX|5z0U|o0%S3e1pEptXBA$@dK6l1`az@n7&hksstB3YXbz))PO>? z$V-x+F^g+DWS@8m8TP6192BP84zGmV^VqX&0qbUAjV)VB#NTzT@2fO+w2N3$F@epuoM=%0Q#CvpH zhqquwn>KfW;qUISbL?o_=@*4;3IODkjt){`LgSz0KlhvQ#c|FZAIHtc@l8Ig?*YAw z3hAnbyj~TD-Nx(k9^|XTs3Vzl*8o#MtiKG<)WFI&|9F7n`DggdWnKd&>RYH3dg8zc zR&=`5Sc@L>_s`RQkC0w@6U7PG_<^sTpILx}@3q{|{p7Ts9T6Qa{UVOoQMof6x(d@4 zk%`}cwsE_C_p|&r@Rv}K_S53UK+hg7c{4$RdRnbU2Uqm*k(i1{4_aHjsSg6WQa-&e zd_Kt&zLkIRYcM<(zuvv9Ci#cmaoRs%4nGs8L+<~} zr?L2l-D&jzpy;)OfV~vox*g(Lo@adbe;teO*4yjihxP4tdMQ4e#MDcx|1fTzpbb)d zF>c=dpGzR*cax`30qH-y{=rU*bdhyFT4J=1|!kEgTC3j4_sx9@45 z9>p;F1f-H-Iw`&4QwNE~KYx0MJkQptXP@AZ14g-XaLq%27;5CF3u%2wjRRWkm%HNg zhqcVl_Tjqt=ZD?7_;R}$*X}pIdG|AdW|^0FKi`h>4yNtBZ2vkhPK3PpH4v-fNE(Gm z`UU5Zr>LwyI4;Y;ZW*AS00-Vc<@$D7ZAM@|A=ajdV)vME*%>&>lP z#UuOha@SjrNPP@I+B@X57*rT)3a zTouOS?x3JuP=tkTK{hvewzWSc3VD-z{M4uC(hhm8~)UuNA))Tal6L#emEgb z@bBP*v<|*b(Zrt=x(6<@n!q9$_ru8Tk>~d~9u!6eLEwQ>h_`YQyWtio7Rpfqow(6) zaSO-`)0R5ubl2cfp&V%tzXo$7`2s&;7xUvl3{tw>oe_Rqyd(y3;j?(U{E&?lvu7~y0D^E`eg2lWd3h;Wh;e~j0g`-3b zY=#QZ9g{ka0E0`Bw;Jp?hD+O0x%~!*oF6Z3%L2_m$rc`lEkR*Y#9>)xl1g!3dZnrtz0sS9&(Hb` z5g0Jk1dF-Ioeec)*Lh9WihZoQy6tFDV*A!{z}%RkY6d1kk+EmOyO`RBGo2CXd33H# zu*mRJj^X4`n-*!!%(tZ^mR*@?LdPlMhoR_Az+~A$e2msNT?XxvC^Ga_1zB+pbre4( zy>fhi%QsSstzS~(Q^Op-f_4FfdBc(3gavvKXMk~Kf z$pDSL)S~WsZZ3XWPp9puP{b058v^WT*q|o1V(OMifZT#Bhc=6=D@{^E9m10IVFK@H zhNk8e=pvEO_^tl~c^L7#T27wKth-Goi<1sY#KeBMkw0hrzB@dEL6uPycFJ?Xz;<*@ z-h^j#*%F#Ub)eM*9V_&%WWx%X(FVw62Fgr_$iyjsZdimXrNFFlP}6RZVDlSvDR2jm z-+O~a#pFo3rlwVbv3Pzd$o z9WXMKG<^uAgj%-co_Us1_EGWd4eOZQ>0n*eOxUyF5UQ~NTh@l%Yyu9eZXK22hsw6H zf7=Wjlusaeiv~vwlUi+`c0prX7ASK!s(HYR%Lc7rmoH@#C^f{z#};+@V$!-=h0=|^)zOMgOTh2bP7fsTwg-d zkQub#_ympYvSHb)V~}(Pw`)y)N+PE0f&$ih*QZU`5nJ3T?WvnJ&g&3rRnRRZ;EbG> zVYKEl;M=u|`#NIrioS^@1`=l-qc^;6U+_z+251+qb-Cnmut0^I_SB&KTmvV`KQb5Q zWJ|UKhaz&UM}AI-8M8QM!ty9vM8#knUbWrxO4IY(2bYSCIIkviW* zUd=M2GC4hsVHC|yoJB$E(OZBLi5ZC7K(~tRZJ^I1Vl!<&un1tt!+IOU4H<?$lM!cj=?bJ*-i4hZWu7OEOr z?VquIDaS0Ged4Lkz=o0rG5?Mq%)<|044?@pFlOu276?=j zIe4t>MdFfYQaVxVS=nJhn!owuSLVo?URNb1z&y+ypgt#FReZy<`FF98Bw=K4J!p4LzCaiN~r6Y9~q zXWjrrjJcM80R8LAp3$h9ni5BOFnj{ZvZ{0(Y)9M~7m>RX=yp9bfX(Cp&c?_$m}%Xg z<+(xa#7VrA&Dw2PmiFAonT4~IiQ)wHq9MQ{D5daYbG|!0k9T(gLRgAVO&{tEyG4$y zrD%!RA;CFEr_g0DC*+hM+)S$^ZRT`Lla$m_^(aR&Dd8I>2j3$3r*{k_k0B@UFh*_B zvPXjpLpv`~ay^7_XiXoF5S$ApRn;=E;U>l-P@QdqHXiHNayT`q{Ywq(sJr&)OwK9r8Xs_qtxhkut>($%sYy-5UBdwIV?0ar~jr~y2>6V>|=!uma(hTxx@;U@IK@E^3Wv%(mZk3>S zfqSl6iqVgcXMG>2Nf}lcnUee{^llo~pjvOA#u+3gXHh1rw+`?}Bn|1{gdBOq2n}7| zF^=~3ynVY?7`T|2jEeNkCJ5;;1_jGzwZ7ku^V7605@0|6ZdUCOH@ogD?zs*jg&N5r zT(eVh7;m&cRs!VK^c=$exPu_6J~(<{GQviqy3{fb_#Jbuxb%dY04T|9KF`m}Gz!7- z%qQ`*J_Hrut`%Q`!_rG+URT+S^0hC!-K_(X*nj|nSGMnT(R~7tyO!ln7TuTdKr%iK zTKs1waoJ%T!=<=XT*08Sl99be44LYfw{pI){Mm6E#q1?)Hqf%-o7Ul~ADC)65q_7w zugNt!1Ul0i+BM{CpF)}2kvqz9!6DRs$<0~yOxhp>?y1^ zaqR1!D_cIcR1@(}8s5`WmsPeOS&{7Ixg7K3sI3Knd<-q;0%hIqcgHYq;vUpBH^}*K zNgo@OO#^R=i1+t1rW7eIu9<-B^ALK048RQy*Pb3Gah!JhSOS8!cneKA27tR}!Z;D8 zK~jgHE3)Hn0)eQUUJ{bYL?kAEJy7$%5=7G1einxk?iko|!8vUv9|o0p>lM#={WUE$)poqX5D7XV>Jk~v(-an6m!!DZ0yi{pNp zz;zgunt%E3znrFxmkE~pSV9gZZZOA$zOXeBfvE^T0Kq(pRG<3OjeN?1Z1c{sGS0oQ zzW^>ZzJxZg_Z@bxQ>)^uZ@w!|+e>kd7=r!YY>ulG>O^cX(9&y7d4KCO2f zaC9OR!Q^@gN15}}GeC=;@-Siy=bzXe9Hdaee)3E*JMsgyDSV2^Kb@aV59pPpyIxNI zZw~k9hw%3+XN@ef;yA4i@BZ^?Dn9n|%FtdQ<#x-aCgkLSIwj)n`S1JK$>2Rycam^9@$Lr!#ee>A!RE zYd=H9^yKD5p3V{5LW%x(-kiKxyVtIt#<#C!R^}ZD0UP$XNQ>!iy{wYY^5hVnNGf}M za1^%OpMMvqR6m8Z+0EvqSUn(Q33D`_&hxDL4{&t*fQJevACZR;UpsQz#_>}=;CDJ8 zsaKGE00j6ju2O1JPhzV2xH2U#vX{gkJA$BGzR02a`%s>auYnw=+fk8kZ4aa+OV0Q& z*O|CR2f`(&iMMD<36+S=M#Z=r*V;yc0NBLccs+nBh{Ip_$q69mL9pf8vXAhs^ytnfI{s6Bn)%qJJyxbwU^g~ zqjc-Rj~fScp@j0;dk&BZcYFqyaqR_WR02JQCd6V4(3r+>YrjmC5Q-P(i zY?!i7ac#h3cA!Y-Y4CiS;!jr08NWaX!pOD^OAV#5LHv9P9;u!a#l*p$NQ(s&?P0u> z_>>k)5Re_Z0WumHo`7`_8}Dl-H(Ma<2833e!&OqmrV>v$*Yz_+-b4U#=%bV~PDjQ6 zt*D@Nj@zWSLbO8>$b&2e{f$cA1mo3y^gEe1vG`~hDJ}k!A1{&dBUr23QN4i~Dh7f_ zzDxi$Gb~gQ!Z=uom^tGQ%@gHR>`{8)wb)u6Ho4*?I9DTSV+TQPM!F~XHMAjD%)gRk zjC%<*Dkd59$Edfc!TF#0`u(V_hJlM8Gh##*gG4h7#=Afu_K;YjtmzVp(@SK;;J;MA zz3XMCVI~cnQaavdwA8#qt_Id*+K?4t@0c4~7E4cSJsogphE?Vx3a68$-EauFpxg!S z6Krp++^&9r+LR`z9eUQVwL@*VqBZmsBRbIZRZLzXfxODTTU3;|);NQ~y3%Q+(DG3~ z(c8%J042QK0|#^|U>^jYgD(}vORh~N(3ZB7xuOkY7skpA2oi3qA8%XMl376W(BYa9 zot0?{e2`_?WO~~hpMr0jy5}u10W-Bg&CuG9eyQ4+D#M4E01aAfiPResdPf}IR89wi zD!2-V?yFE$RR}v=J|+VSppNVI7w*FQZl& zw8Wa;u-fDze)DGHi3~Ntww5yBt-L)Z(42wD4Y+i8V~qv!rfGq4xfuSV4~e`*4SH@` zmJ1k6Q?+v3(>a#}4&KZTfH7vO@e*za3tKYGB(@dEt;jx3FFplf0ERLUEMb<^sZs4` zqT&?W1yoR`Ac#oNk9Q5^b~3%xIt=M`mav{13s6g}Sld+&)lSF3!$S{oS(~d{q~|%= z9ZPfQ!frW39YvjMdHX&04$Np_M>5DBaat*_WjdLm+J}GM09A0BESE3R@fw8TlBRo4TtyV3VT|P|5P_kxJ`9(9gF!`7o2&iw7}OJhur*9Tp|SrDK;Gs(qI0w*Fqd1w{V{M;}`#WFbk&PR{MJ zIthZqj!VBu^QLW^uHjC%&3%+<1NmUdofc`6j~oJRMiqGhTdSheIz+(nrdtbZtYaS7`bsb-X8x`iC}<#rY-lWHVkkELZW<0VUpZyT09Uf}Rr5MgH8iW8~Q zQar4tXHfA{4Z0o>%Vu(@%Jb%W)Clo&8}>6`pNCBQvaizx6W|ya?k7tns#{lx_$d~i zn-0b6tus2NGq3|~as#7&7d3c!(wf6A`LiI6EXQ6=*b+GYe%ubdWf`3HaihJNCD=Zh z73?VHw zc$d4{+ekl!yC6R>!Ig_2A-gQSmkWBT0s0dyCE+6OTRawc;Fq_tySrPjB60$PGZa&V z#89v?Lz(fRWHS98Y(U1!Ntd((7wtuL&)xXCoy6PeD!Eugp&FhX-eY9@lTVGX(Tfsz zFa-4l-MLPIw`qY0;Z42NQ7iiwH8LzP!eH;NEu*6sUCyTNTZzml0T#NW3}logcDuYt zEE+^mk?ag56H%NZ3%qN`&gj=&;>N1VwjD3S2*ft>R8Ydr2Iql14m{L@;nQeL_#x$? zP*kFFNbK7(rG(Cz_GodPpTcHmE)<0n&vNmek?w2`{6HtzB_&5SmeC2slzizgb9vJ1#5Iw6sT0rIgqa%c5d~L8xN6K zpNhluh-1?I{BA2|bWh^2A7D%v&MH@Op!{hS2Udz=n?J8(H?pH^ry=KgVrx z7}uNU@uk?xoa_#7{e=Ja%ik6s|MuO_$7wu6z!j1Ph5YlqTTY*(b19bwL~rA9BaTSV zX|evo3%9=e`LNzU;6G)RK%f(s!tcq_S|3)Mz$%D|@|X2jQ@n?JF&;(p?d}-iS9|&2 z5^fYRM2ZNMuhw@b$kOi5H(&{T9=9j|{F@2lv!0&~C^Olt&yN9V;%5FA#rNTI#n;nL zeD7oNz1+k|$jEI<&t;je$Qmme65&&H#p!|N5yLViVpRHwDw}?_UnGAa9H3wY;3Q%(+W$-Fte%m3;cA7fci` zU4={k;&N;GSOS=EXJPuz#UIb(y7)})ee<$J^*KSMR8sfl{VKZpUUnd=J4~ZI<#Drn zmOpn+vH!dv2>l{Ad>>F|{?l?uqp*VA9wHZTe0*S8E3|6m2tsLw&M;s;#o0i0 zhdi+bj>g&yk~(^l>m^*t5eE>i{qi z-;{j!xquJCvmw*Uadjnon2t+)pDA+2+8q!$ea98ZtE;23+Zi-#5}GL@Kb+8N#<{yW zDC|r;r82`^G0DLh7}M&8dmJ`yw{cF?fK)Ust& zSfFFtIst5>L+G z@+MFy$}3y?@lqB-4&SgH0}2TEh>TOLZvtg4%Jc^JL`@)eNT2oIA1Z1By0c8%Slas8 z4|PRY#-K0kL|ik1K#i}l^fJm4KTaDWaKgz8W7yb!IpKXI`gyk5Y?d6d8L3Q zNN-fEe6&au_r|r3@C#9f3M>*%pG`1X|_X5OL(MsOa&YCR#H6?SSXjBI3><$BwQ~GWnt-HBS|^s5 zY4r2)lecNQ9DUbj7bp>0aK&&YUU6H19rrXEJ$7#d!cD`#ml-zYx}La}qtT$6$Z?$U zC?aV-GJp)zN#%?hIfmG$*emB61UMM{7^6Tn0&bA*6Y+XD4k01iP#!T+t7uTF=ZtkK z5nH3g`aZD&oiV4Nk|}Pcz>>zP2VTC~l!^|ZC2~QWKOa%TDJxr3K|ewlAcfo21i`_KyYad&Gi|(1_sfrWNL-S^Qj1QE3@D@d zdzuqGYFM=;;S_JiNN3iR%mlos!i&>YLDBI8sl-|?GPJZnru&z@;)NWPk+!K=>8l~p z&Rg|N#OTt8%=^4hqn9tX>6%PAQMH|GgOV3vc@jl)px|>p{5kMBO1RnPKI!91X^+sS z%!FBg^hm|3A7qv&(JZm%?kSRT%t1A!41kfRoz5wuySndGXG7t&ElYze|>s*Dqw zYN*Qj)s+OHS`_%Ix^9^BC&E}OG`jTkIo+V0Hlm+^TC2!s_pU+(?bD7bDq)*b8;Rms zn#LzC)dGAMjIT~L+t8<4vux;`;ZrMca#Ge@T}1^Z;dzL`#Kn36iR0=wv=V%d&zs0D zCP5>~bS=y564=T;(;?7k$R)$#p)h^HLReHzb?{iw8`R>fL{qkAPNs!ER;9JT%WXC7 zGp~%zQ1q?0591BQ6(iC{;t39}maJXko90L_kAaY@0zb(6!8-lW*^FxMxL=1pa*!l6 z8OrWZ#j<@1z&~S3F63h^5bP{I_7o093qP5YBeA{$l}N+R8xrb%Tn@8_bv04AyFZNb znpZljD34}1l4E;;m5p;?KElWW004|(@!(jswhdxHhGQ;iXDnE$4SmZOO$^rdb(d+T z7y=BYMbe3TPohc}5^@833C@p5^i@l+XcL#GoEHHI128tvz09qA=nnzJQZ`lDQMq3H zccqUr1G>X#zwpU-H9;AU{5TRGg@8SiAGprb0K0;~e8(E5=k0p6-UsA|cq)cCSQ0!m zqBL=Fs88w~*@K2w);@iJL{>D^!!TrMrCfPNO=O$PF?*c1t978B8Y)Nyux}7~Jpj%T z*D}9VMv+U%MUfAi3K~OWu4CL>L>!mmu7mF>_bR=C{DjT0RwFb~1Bjzh%gSA3fokOC z)eJL=dYaqgtyR z2Xgr5u=_tv!NhIUoZZ~cdqG*sv6CStOH1k{x{8>?`T`GCN*K(PZHZzvXrVgBnRtIj zEx=I2jH-v-OBA&P72b;?gdg+3b-QjriQK_og1Tmy`~>Pm6$_c&I;1vyOp2{ zq7QgnYXd0AlQQhDAy*2L>n5XV8nGZf`uDhTB2nc=rgPv^=XSaY>1WeH#)GCS!Z#s5 zS@KSQb%i>yw`hUzrJn1SHM8-yy1M!)m1>nSP7Ohs@^Voj65^q6Liewx`j)9 zvpYNlEmtSu*mTV5HI2vUVxm$5X+z4(?PU{;F@+HmIVqX9FKx9FquMoi=c2V>A^urS z?sKC^ud)P7ma)m`-vo{M_f}qiIXQ^JVR{{4E$if>y{qn8a0%nzkK1CpyW1TcM&dBt zo}ptm9w0&n0N8Z_A{V3}Fyia_5H?%id~^-IkrxRXgP_^L0Tpi6tC!UV1QvYnP6y9% zP0KGndKE<<1BcQ-aw-5`j8J6o#rf$DB8c0!!Q_Vt`hK4FV_SH9$R1wp*0Jk=k7NuG zk?-Tb2P=u((7h-6*^K@&QG*T_M6a^v9g=x*j398n=6b!Kvl;yG-C8~indl$n=XO86 z$VJ7)dC*Dv+C8~a)1JKqM^(_v7Oo4fWuI$j_S+BaQ51t{PJ zsX_3}Y>Tq^^|(EchZoFGtIMfW0-?_r3@3rICHRpk_sY26A3W{{c!v#Gc)(F~Ie7}~ z!Q>$`4Si|i`$G`jf_n$a~nxFjf*CYR>;E)Pw_{i) zpE`2FZSg%AbdT~bWm%1D_m!PasQtrfF>daB;ZD(USlMy!Pok* zdv=1AvXXAcgG06j<|KumK6((^wot$z%(b1-{iSQMcZ=pbH1vSNNk8l(_BAMLUdM}_ z!o6R|hml$N)epb=<~kBlRgqE2|L~8FAIyDMrw5sjstHs0*F#d|C|jdl%t99~BSt@y z*MRfy($&3>K#3l#>zvPzSQQDdEmL?duoi7Uw5|k`8tGnR^{Z z?-mKU_-l*DX_$QWI0ZBel$uIn5=`r?CFC(0PFc^DmD1&#xs~by`BOCwoX^Y0@CS~w z0LeG2|HQc4{}%KD$t%3^EEhn}h}TFN!^5P)i)C9wo%+)4oWVmSN`FkxIC<4b@kw)P zB{Mil97e-z4=J`A>^0oU6%YbWOt3PPFbK!;D+Sl|4A?#fIYW5@LnRv=s(#qkJ*$`} zTd22k?KSUfCv)Q+yYEYozyLu7)*au7N8rcX+O`yE&ft<3Y{BoB!a!p`>sv3{{|8WD zQu04W=V(H2xr~+)k$|?g?B&N9fM@<;1)ezLkuMqo5n zb321tFEwMu7)Xs8SwD6IYV)Zw-_RQ;Jb+o4s(kTmC_`|+yo z7;fv_a6c}qvf&i!64KVnpRs9}yD-1KV?fG-$&#jclSk9Kteh184&|jtGnWa0T5J z2Hr%~yjn8@eG2vry8s72Rw^y1*mM7cXFXKI9c>9yS6zh1&sP+*Yyr2OHeQ(kMOZdrT`w{1eTSfzK zk!*1T?=69uyOGPY3=Ltol*qI3|UQS1p(e> z;ha=Bt*$L#>@;oOkx>(91?E`f6$MoFP(-3#BtfjyRSg|FrmadKDcCD3-bqrCbI>3S z9PIkEsE*6@^Cpl|Pl2I`CH%I^q*p_vhM^fg&7l_Z9pcOFu{L3F;h*# z)>O;~>Niid0EUCy`UR^8AW$r>5^)hSEN(yTdQn9PdZif_+<3d*NJ5mJfk!iQh$8^X zR5h01$*KZGo%U&;iwzdCruqBwX)|wvdQD3oNAzgbs6yH!$GYOjOEAdL`RVhR-a?$7 zfp^g8LdBPeGlllSMc+y#{=0Z56wqFv5rA|v9Mn4B;26P2ot&lQ!Yqos;zxBYV_1;X z_TxQ-?ir?m40AAO3==nM4pmJ$5aLo*X|C=<$D+;XfXmC)@vi3RI<4M;1%hYx5Qxtp z1xRyxOb8JzIL#P7BU5!+i`7$L@Tn?%BbP65+^RZV)AdzG#~X`wHsF*d8sq|XPv;~( z>8OlcvKIV*Ej7W#;)>3B4dEd=o-_eqYv@wbuPgv=R*o;5L3booX7OuUH1=GWBw%YR z6VE+R-p>x;G)(X8V?XYK zmQmRcbK8(B;tMo%k}nXOAQ@~}Ob*mN=WT=`$^P6i(fIf@ZM7~B z>|)RLa+G_||7;&1nJ#FI)w39^PmRm+rZM?NG8&$30OsFhb*LoQ~@O(&sBoe{iByNF7t z9`Lj!Js|QUPdJHcJS_@4^4u0-_gzz53 zBgtQj>Sz+tH2gG6h@3$>%Wy({#te8fh(!0Sw7r?O5jr+V5KY0)WzVMAfpv=l%anN0 z^d-=p6ijQG+1o&1ujSmr9@UX!08PmvM^h_ZXNi@U9Yz9kg7V&N=0*cq#APO$T8;ZL zmL6oGf=ZW`*LV!qkm!+7X*Wq`&NR@7VAA+wlIxUJ46;5-dZ@=3=BE)yt7E+HCB&^k zCk=yLqfj)G-9t~oW{N#E6}qSh(;oZL<)gAqCjI6iP-@Zu0!x13W{f!v15vh?gLqB< zN8!3kj#~~JQqe)#PU@CL7x0xIlJm*P&!J_89E5EI)5%Gik;pzM7l8xqb>jVV>-SoM z=kO~Xw0<)fLY#s@lTMsj31i5OSTvX!3ledD4-vcx5$b`D$**wZ+6hl#;q# zmQ5hW)V(VOaMtS*&1q_sVsd){G*9E@u*wV_j{$0>R<woivsu!DzhGXB&$WiY{x{|3uvwH^xuAUkW&u_aPj%hm0}&DZd-i5 zai}ne2Fhox(93cC=xCqTk84j$B^#0KukP9_{R6)b6gSgm_gvV&{P{2c_vead`g-y$ zK`;1+Z&Vx@|637T$HFNFdlZ*^o8U{^dhV}rqf|;B1nNg$-^gwZfSs(R@N@@I>&;-v z-EjgG@OT0V(dG`4>fU0tPtLVCS|MO-len21R>u+}F1O@2&S`L2PgD^i|pWgi( zK$!65JZ0GiD&&B1o+6ac!{*%Gd<4|w`Sehzc-xoTd#q|Vv)gG?eEiK=d^-Y)`3dG= zyn*DMYy*G_79Urg0}p=UWE_w1b{{>9nf^?iyW?KQzy@^-Xr}zu%}#z3X^!J!a^4ib zc>)#a=6e3y!?bY;6ZykC`CqekoxDT$Xgir57*>rVffEHT315@EQZb znatGY1^?g@#kksMAor4&_1W&c+1;z)U-*~5M+ddcc~qiYxlU!QY@{}d$7wvC4{k-d zm(X{ZdMhuX{>z__fArW*tUrfVm5m!X*eLs1DazAYZ|E)rcilhoX8~7o!QZBH_K~bF zhl&1icPOwh{uIHj@hJmY=-NvbmVIlAdoZxc*9ZbIAEUd4>0e<$!Q%I=0d9Bo_IZ7r zULW`CgQ8Q%!(k_%5f~vHp!DNdFOwDIBp4xL`Bf18bU%s-z@2q{m>?4;C{ags`^3>J zM@rm|fDoL8hx_$be*6VnN#DL1!ZoTl?P8j}aHuswT+j|-CjXjujA_u%@~z&3o7S1R z&(q`G;5|96L5HT52Ln>uz5mh!cjeydGTzzWM-^6s#_|I&yrwoC>zwGG`pT>N0UW`? z*-_3)DjE0z2!GyPi;JXy;j#^PjSIDDK9^eVWEMdn(`%DSp& zdGD($+5Y@^D^X&m5d7#{+@Sf_h}X;)khS2)fkmQBYXDrfecjUo+=qk7+cXyBIG3?T zpy`4V3xfy}t$W94)^dSxgTv*OGMd$l0P&}`H?VcvF~P_TyDVjCavnpEwP1mcvV3F# z(Lo6tn2S1)s)^epD}__@+j|{0=>2++ALhmlgRjK@*L5WRe?Jg4yE~rzc*%tY^$m0! zZZH#ugh4>|C}`fg!nrbPP?Z$c z#}dIH5xZrV*!xOq(PR|ZCLEHpx?V3kQbr3V9Aw_8t*VZqP4yeSMI$=~;RoRAgkjq; zq+{kdLejh^n70E8a79 z7A=aH84&Nd*J0UzGJp^r#9Z?+HtMLAD3O!r&NuKDT~%@z?%j27fc}JLt*m_7zY<3mq$_SIYquW*=1y|=raXE3~Gta z224}3Cw>s<3K(J65gaYx6`_tcF#C>dLVg%D>F+7DsX-2kdptV|;PGa?37N#E>E%nk zba5Uc;Z*~&Rjw;WoC%ctI_@?egC1Q|qRpR58FQ%@Js-^c5v$@2C^DIZGbO`M|l*7h!^f7H@%Q*#vq6KpzpX!RV zk~7;d%}ET(gHa6utIM~)TI~+A+F+HT_WCI(3HEZ6#Q!x6 zx=N%LQHEks?xp^q8JJOE7n#0tRJE(!E&_?G0b0u$u8w;}E`hF>#XJ#G`*Ar!A;ilM zk)WSCn<<5Cr!nBy!1*%o47e(Q&l8x&WX)HsUN~d%AihJN|9J}?H*bJb(9*eob~#zK zKx{$VvnE=-NeLF>jBwIEj5l?iVY-|#9+0eK9tof9^9HhAW<*|>xgxa$aazXB0n<_3 z;DM~?CcHV#rABx%2CFLmX+V*MPKg9_)T1s!L2F3UsSS$z)a(Ey00sTnZYXU}+e3;( zuwehJh?XHp=vvu3#Rnm|SDCp8Tmvl)?XIqjwxGweEr^H6(@q$YEvWjKliSU#8W_NF zNE0qd!dUSZ8kRZ#X&0WRhEx=vn@{Qr%W@rBmcN7E)RVrrN?c`{t+_AA%3`ZIv$cR( zSzwI0%jsqpHPiq}lTCt&OJc>#S}@I1=yXXCW}#d|i!f~e-lnu|Mv_&cmkF^N2L{QJE=px89`*HbbxFDXO0n1)a z@r-(?EYrIve%3o2GnMC4rLIoC2&NJC+nQQH;+V?M)Z_tH=f0w;U5!v~_ZE=BV_k;v z{;*!1H_;J5hiSzwq02a{0osW;4d*gR!NdTbol~k070^E#;1t$GKf4V9g#MTL0t}liR6hLWWlPd;^++cXXi7e)z^m1nlBGnGI7Q765z$*bK8)c` zBsOdY3N7wth*vw58O-}1WuXNAt{yYO=^AzY4D>R4OXZP6;Qx|crb7cNO?+4(;2bq1 zFnHXAOpK{|ak3f8azE{Kyb*7mc8S?sml8x(o^j9z(YdIt8V?i&^} zUSMS`Sa~^m0X0h$vg^@;!0r3Hz?~%lrDcfh!)_DLdLu!s;U_-sgO;G|L-oKko>x}^ z7$WE5kcqNZcRJo!r`6ac!+)cA-G42mQ-C7Un}ji6Ishc&Hv`b73J}xat3M1zBj(=d_;9) z8a*=M8|SfK7uMqdijPawyQ3LQK1V<0@w%uY#nX^p=~*!jpJ_CEU36XT4aK$ahAuS^ zqi@HGqwjhc#*$Lmek80_(2EbaFUq_mq(`eUg?twWiC^7{mESu|9%}%xM%?BMWzu>3!TnHiJHGfo zIirDk^WyKu@waU}w_o5U1McO1THZiZM{K;BV&i~O#m9};O6)rP^H-nv=NyYW*`bQj z1s#rs>byO|rGHa=Ij$ZY=@i}%zDxXnnc8Q6f6ot-e{;!?7xJ#Bqe68mt3Ql&ieG$# zWe%8=n6G!Y4n*U~sPKLuO($y!qvmN&s54|c%SN+q94+c|WWJ zZMy^co(b{-9a1@RsrzsNc&F5K@D|#witPY)h#c{hj|`w?@}y!V(V;`YVV%pD1qv+i zp%ChM1{n|ND`W>Y*4Ku<&!`Nf$&f)pu#C$0qBek+HBikf5mu8>y=NKhMb@mB_%*|u z=FT%`Ag?@=HHf)Tq{}#wsTJ(5x2EEUn2`ugpHW1-o$iB{c_%JV&*4%m#B^27I2`Xb z_;K*bFnNBcDR&@bXL0ga>6MUN;vB-ignmnj_!>i3PmyBg4P>&J-r6N4$Fonrwh}3G zb~UNd_w~?ny?G}3=}o5Ir>B>IQ+w}*3Jrf%_JbAldUC=A+g`^fy7)zzcx;ff~TS{YIR4q}uVG!j-NxC(lW21!u zzr~F=xD`r~9QTLF--FPUcDXeBD;Ba_b6!dPL~jGKKZ2;JpQjd9wmJ$M6C^-cm7LFx z{=c`82$wM=>a05pM@~0T3Ug6Cuf<`IuZai_LQvw30BJLHYO15}O}di57w?++@w)2_ zfAqm)R~wn!j=mW9?tUC)Q6}hm7!RT9Y6m%En5!<9OrTuYFc*AUdaZ5A%Y6ZesexIj zD3P{|pqynTlgkXA%lxnegti$7lca8>H|bnfm9u4w8k=}WI5(&)od^Sod)M}qq?${< zAV1mlU)Y~2IT#;#qmhHTwb z*kLbi)OCY_ZR(2hSn=bvC~FVwPAt%v7L167Ce3a0Y5>|gD)+}+h&QrOye8z()m1!> zEYw=LLG>|+pdpLOA;2*rQ!Q%f3@?l*8;e%*o{9r8`y5^z8uD6BF3X3x-lrf56c#hP z?1Jk=cIuYN6*X)MHqd;edee;Ew5IC~a|=4xG7@9RYwXE|Mu6+b3x7!-&mdTddO+?5V@~v`2AZ=WL39Zp3~fc zw?QjnOP`m#b5-9d#c?rQ9+)u_(Bx=X9wx2fK?55z-7xRUz3elgBV1hD zF2@6t;$IDTNzI3qdJQN~xPq79sxfN=t6fKO1S4N%26lXNSV!D}fzl1yd@?qtajSES z@=BMfSmrBVMpxhFMAMzF%NT6OOyw$EfZ@m zSd9#5{bV~JcAN#+&=?3AGYKVC>BMz_6ZH4#o&0~2^!wFT)8E%Iv+Jc5&=mcq_#$MQ zab2J5^!}Ot%gwDzO#)(baG7YI+65QuPjX|LhDQ7CG6(LJA}UR-R$WX04HJ~@4dN2L1ki&+UB8uY{ z0u6`aZgW1kwnlgBNC?Y+qmDGnY(8rErdi!Z@ggMK#mzmtX4B+scETViKRu$7(!8;M z`SW_KDR$5Z$q#N<@m7%axRdFW#{q?kJntlCPtz8SiVv<4a;I~Es5R-leY-{O8EI_y zQXl1GoKOuIZw}}E36eD9(~HxhaP^AcCH=2*yT}$Ssuiw+M#PWbVm2HY& zpEpicWAIngE?xYcyZ~FFrPBSW{8L%JG07j7rubr7c^wOJkq4bq|9?Nd^(>W#*qQC* zQD#CD{Hw3fp8!8Vz`we2Y9yE~t++Y;8>@$xVqbPRI^5-GvfEsKnuZ{k4(5 zckzxq7V2{oi1p!!p(#1GFTZt*+fCF?J{`Gu7`; z?jB!4^X_*+@#RyMM)x!8C5yDLHpmk>#_Cx#20!p?PnJ`B{;)obtLtd$WsKlQUPB29 zozVBOfcnufT;E;L={w`4l)eDw7oOY$gux%eupF%Z)?}9f=dAuQH~=KlD}K4Y9XIzo zCw3J0X0P>U5_zS_B2a5BWg)(#VnQ|uk+`bu@}cGb&3}--J3suiJH@r`|2O^=)t&G% zbpouR24r{L;?b?qV<Ze-QWAllqyWQ=~8po_5HYa zjob34Wm|=x`umD8XvX5ySz2v@`c_F<^UGoaqVkS~$L9n;U`EW&GHbI~P+N0;KdH-D z(1y|Z=yrTRUV*8H*}#_Jj_ZIm9FNA#ENTh0WiJ$ay&ndV7bP2G>*+9btdfzY_$8j^ zyt5{)(R(UxNuHuccji|xR4=>Z91CNm95(H_W4%DbkS*qa^EZ@0EgZ7pDUt&Ol)bd)#Yly0I;v#CAIC>f88BFl-om`}5 z@#sa4ypxZ^a++q&1$7XR=dQzt7<_dGjO^4JzsR(<0E<%ld&sUq{gGK~((bVq@~o`V zvFvgXBdEM&zR$RDS~Pfa;o|D*Ft-juD1rGK)z9%JF7d@A_1@|6CXmFT_zQ=R4uj>6 ziHQSz9U|C_BaBs?Mm`{>&+<3m4Um?hV!&KIab5Di*-L*__9RdS2sU*ImZA~2TSD`FX)3bo<)@L&021=$d z))2N4S5Zl)SaMSU|_oz)ww->hF3J+aYsAv|=Gy2X1|4qEL6mTa?Wp zQ8uMo9-z(eIL^&#oO_~_d~Lc+ArV1>4%dJfWcu~zGKK8KRmac|byZ*aY6l#dY+~<~ zRyqb-)10JilofQ~nR)MmcM;TjmErPO62PlnRSpbVPnwS_aC_66US#G#UkK&y%O5*M zf^|7a8zv213W$M@wPNPV)q_9jDgYOFvr5$#puJUdk_iWH$RW#i)^OM1BIjxG5oZWY z=CVg;;5ksXE7sV6XJLfR0+ubGxVRU&0{m0xOtOF{ut8FNu|vx)T8gX(7s9}tHxWo| zk*sD7u(bQo8rixHSI1*hxs^%pShea_<7}Ta6>n}}qRee%+I0y}t~D(A^^sg3$k&r@JR|av4C5V%P}E$7KvLsP<>Vusr(rDL>=)q zJ@Qf@#`ROv4xAAC$ZZUkH&4xK1ruw<_sGnCM0d$@sWV7Bq=0qYxUOm{<8*_zNHpLF zDp`$k=6VN5ZPnI<+IXJ%J(~(_ljJ5Pr5+RlWWCWoFCKj8?lkhO8J^6fXwh2Pk|@@{ zsH2;zZ!2i;u+Nr~)EdjhWyIUzk(}5KbGa^B*igOz^AK^3^j)V09k@SfZ*s|6wUFbF z3nYo8BNl-;h?)Im^$=zbjbZZCm;1{?zFvL_LK_*jYh)pUQQ4!oUss9EUth5JYja-? zSiVffIX22o;N%>O?K#LFOKfvO%gBD(y6o$DzuUSr@Vx6?`&Nz8#7rKt!D$8%XsFX` z{R{lSG7sqF$E$kr=PswWbA*)oSCrffS;;e8yh0foo!q?Rw?1f^H%&3)-a#sQdfvVE zH4-lqGuIm&mm%sR&+F~hBf*RD<_OWC$x}8WPp(Z9KH2Io^soQJ|B5{Ic=AsR-SW-( zFm4j_qXeVKz()HhvE^zz@{7g8xKYVzH}6mn-(F0Y@{LZ9>`o#BMP*11i2JT>igM03x>?pE7BrtQ&Ho=y*=FPwzw+q)@s zl{3tRlZ)Rxy9S2!R&IBm(_Ql5*RC8>tlhtQDQLI zAZj1W0wJ2Yjw{uZb7m9O59lJo`*@Z&iT`-OEcynHm=X``F)Y!KS0R)A(eiqi`D=Q` zxombws`H1Ne(2#7(iyqTrE7DI8a+SkWV#RN=Gb~7+0*g|@sGL1H*XKJF`VTEcx6wl z^V~f?Id%x$M6f6_o5-8uQ+L-Tavs|8;@Z*ZZG<0lEm1PMxCNavob*~2^zZ$1P{d8B zmB<1>H+wNA6|g6KGf!&}jA&y|;ljpH3vuMJ;Y08rJxVpXeQftunUej5yn6il)~~^^ z=KNjmToJi!X|;KD(s{!AMH>sQvOc)Qr%$P^a_@fj%>{D%=i+O~aQMgl+a%?Q>cax# zz<`=i2`L@!opZT!npG@p+i`0C>Iyge%FC+v8C+KQw7p5ySDd?y6dxs^nM_fdZg|eB zs>IVHV4tQPF)Xom=1zUm84V)cz!HB^)4T+EJ>@Wh$`{6S5aLvoy&d4~ygvqmiET@o ztGOP85UQhe&73@f1$yimm$(qIF9GNy$J?p9=y=&=gwvC94S<-;ov1y}keKhPj^4!4 zXK+!@k(ib#1(JNWQ-Cisi0nBt1+yUUsycgKB*wIkY6t=R<)W%Jy9`2{*+$2Kk}SSKY%!dGBcqdB(bW^aO?!tpk8L7 z%+LYO?5nUrY|kj|gT>9t^h3X{3^nfKazI!IxUI9pui5$}I%|$islA`~7*g3Q|lAWKTp&+a4NI z@8@`}S}V(%vK`p0??ci`Y|6f-z2ZOk!3JC|87TY_2v|AAU!qybewP@_{gq9#yP;3j44aRBZEvB9Jt5hz-rxwE4jE27T80XFpDXT4lu5?<&qm}seu zBCW7k@MRgghs|^hOm}9GbAZ~fXL>oyNjYLjLTb8WiC1-$Q9K#ImPZGlFDB`owBi)i z$bhhmFQ)p2^s|)2kW9Gyg$>hrP(^sVtq}o6Rkk@-SE&`n#;P#x1 zPYV&?4rj=Q^N>ShNGoS8t?LUxG7ErM&7n`CHzr#}S@Dv9gBlDPIbOITt=SBjH}|Oce&Ebn)(X6o9xtJD#r)ON=^-?yiiO0L z+_8{q)PZ!f<%VWQryUGHu|^jWFXX2w-ir9P82N*J&XeXRb`U?$M#HkPLEc??r7&L7?P+|M(>c^;si1D62 z^|=NsI}Q#y+O&74NU$ZI7Sa&R8Zq~xps9_8(d3nEV&$inP115JK50!>;9V{06{i)v z3cQ;wOK9&-ve||L5r`kT`RE1?bYT60!a7M;+}(w0Jt(_+({+De56K_4zuFeR_8^5s z$?d&~I|p??t{?ZCm*v04cZ8mUT`~O$$BBo(`?}5cGMDLDG|^vh=aQGNcMkr8>}DV* za>-#$o%;+aehevDbpImHe>}?~yE*z_iHd5Me|C*3$5Hdf5W=1@{7|%pnp!+7!hrM9 z!u79i*V| zyZ!WVkLc@+;?Wk@D{epf&HCtx(d1#qm)E~vI|h{3-%F6^xT3y19(P->FXu`nj)&>? z*0ZDOtSJ24iFvILNB|r!wBf#;!rN9Avc9Z?5>{C!oN2OHz@B#vl`oVW#w0=RT3p`9 z_6Qy$XrUJVxKDul+vz!Z_n5fb;C*=QvN7I@V`NKk^>(h_UAtP#WmUts1+5^nw+)>X zAxRlttiOO<*1wTA#b+-*?~QeJGakV_CL#TV5QX0>bnfc!vUV%i^e zC%>hJpGsQ2+_WvfvF}F-6jyJH-^fJC4WVr#6ldlP{-UGgbjeny&xRL>ejxwZ#`ox2 zk}h7()@j=7s$9M!UnyES2f~*(ZByux7JmKfFTS5!TXY5BM;y(W+ePIK+}=;8|5*HZ zmfLUVg#aQp>=81zBi^*DJUHxBag&E72%mpwpOSAGb85Cjrug~sr8 zEjN)s*Y66mZaFx*{A(=A*axJmDE_c6$LmEw%bFq9t75h<5E0(4@0?$Jio3wy+1Vjp zRzs&v$b4G}taq$#5UUwh`~*n0Qa2gpqv`C2s}`-e?5BjwL-(5;QQc>~h2ezuBlhc8ouqTY^B{;}ObhApT%F2)}`2@=}fD`pK^^t!kP z0ETo02f!Pl8-Y?-NiI(eO0H38h}5}sFEuVLyiIRd$h# z1k+fY!f<0V7=B&&-b7}C`ze-y>()hbL=t`unVGf&Y5fPYY;99htI*N|=Bo_Qz`1W~V!?6Ad0>5S@{P)bX0U{QR6ZkC-$+s8 zGF8Sf7sJ)n`4nWrddn12ej3!EY+4p)T(n`K0Mpl($6FArY5=dL66r;gLj!y!Jxa`% zH?=H1S{I~p-qcmabeHN#dW#B7NZbkN4tm|iB{VnOx094>K^-(ue$rpV8Pr49TBh{7 z%;(jE+hZ@yVr?00WwoW%zHGf2sydTOh?RC3FXiN;(?sDF=t6?N(}$b+oXR|LIKqSU z!yQ^wD66G0j+)4eVcEzY#SHP5KyQr+$1^Jbm2^sB9DCtIrFliR}5z zz<2BOO*NXfnA1siQtXv#(ji4h?rsXvuB)L$o8{#KNNYsm5zAVnqU4f%t*Tf|!+-pB z*~*^JjEYH*8l+5EA9BHs;0!BqMrwklNWASwdmv88*%+5a#2zAgd&g5BX?fC4LH>p9^PqTT+A|qy1 zL)$}0RH#V}NJ*ZdRV`#k+u|g{Uhox@QVu}L{P)wm&AGNUU@E-6-5cBF9)U&c~Y$$2EsjA58OLH zi;UT_&velF6Fc56i06p#?J?3Qnx>UwpXe)+8j+YZT|xy(U&v_P?+7sM=&h_%2m z1Z#1Y;?{P|Cp<2Lb#wq`XxcApI%QLFb;zZ|r=Xq%$v)-^yuFR3P#|TggqYOAoEkhc zNMO^yc?lNQavH^#T+~9&AZ{LWtRiViv})y-sHTczAsMH*LLnJW`dm2yIR}$G%H(Y0Hel)?wQ2k}hlfRx{9sy;Q*x zZ%v|1I`WpBAO>I&L)jxhg~P(M74mTGttyznf@Qp>$UOlz8;b*m-7(oxw#S$pGEQug zCwkfP*ju;Dktc;3qMFG0D(5h5qYD(0!2*$~9uk|stt;}ssRGuowli;^uLh64s}XWV z5povFhX1%ts-nL|%@6i-vmO2jNMKyR(Z9ju^>b*=bI-K#&+_|m^X})vdMdtu_w!2b z^C`Kt2&1EN4<^Et%hQ$9Ak42X;2T#O;FpuXZ)M#&QFuIvxRcf`>C7h1HXa{ znHEpuzg!kiKUHMJ2vW68THcM+dZFcU9{lK7Z259;im$YIJN;=|oe>OsEQ$D;#p(f^ ziypNi&+dBgE*i(&;M%pry0d?U7w=IGi|u1MfvRI{d@ZiWUa`;%^#FL+xIbHzW7P~ z3VsWi&DZ03^`K98-b?i1ujF6L50kt^h)zw@DPmK;cPc-?{Cso%lPuKT`EajFc}|Dx z5gV*{%#P-BaXBKb%wPF4x#CUeFQnHN?rH8tejU1c^+m>2B9{$IHTE&m_2T=#nL=~q z5$K5H+0lO!EV zJiSthtd@P4v@|9*zyW8N)9xd(ph9RyF52AfS%Cn7u5jMQ3m`uDP$|hNY}JOUp$8=Z zDnV3)38&0t;Ik|(`{P$vM=hQsp-@qJ=xN-58Q57s(~0{yudY^ypjXvr8nx1@S_vWJ zuHi(M-~C@{k>7Bd>c)SyYEglHkfzFiX|F;LWKBqmxsB3-9G)KJuvYF*113po8$7zb z!TDS0jOgf9q}BDIZjE@f)H5d0TEGGHP0u1Gi-@pCd`zjuMOCG5x~{5OEG@05qCm}2 z)J2>u&kLWHE)>#5U5~JVGBmo^dXuh2b~>}hg2+|ZIAL=N_xxt4Hf7h+I*s}J&e5f2 zmKnB2Zvl#sh8`sC*=&a%VnMuAPeID6V!ee$9=?^x=RM`f@Eg5cg|pe-X;2FzpX3Lq z*3hy@+h=r6)CA>-46bL2xa2lN&EgM&FkBjXXNv+-TSF#+xh;cp>Wz?bW30v16+8g7 zkZ_Rcb1sUwZD~8+2;*<*O_wDnNR#4fi|ju=RDuM82~alNJ#~tOd*lk}a9YID%>cOs z${j6Q4;%D*^Jlf_qeT6jo53!cHoK-}@Y#~}zv-$>QrPXT%5e)_2Cn$%Glf^#bY)|C zPkEj>K6}9JXlpwkXYS4nncYP*P}86%mnuSgI6&T@Li-w31J#B1CJ?pmDV2VaifKv< z@cv~0y{HADeYh%VO)XLo^)Se>kHcA@m#fE1MX9mRuSlg|K}LIJaG{m*=jYhz%x-BB#d1e3z^5BRdD1to4Bte7huI%mvs;ipy*c+Q8nPzvQiGDyc8O$`) zP$S__?w80{ghym#uGpJAxVm~><2@vh-j$h=@o@L$z7~QpHINOKyaAnKx)YRrQ#QP^ z+;OjfWbbpdLW3+f-LF!__{f$K7CqsjP1CV!s%~YlscPOo z*U(V`{lAQ(?|sYAkJ>}?hwD4$mKfQDjb%_+_bzYJ$>~m6YmElg0<;BGWRjNkp{By{ z+)X*K3%QNuYL=KH@kWh2EW3#Nc)fNw*ZMZ&u>0#(yr!@n%eI2Pbdsr?dy)#-yG%{5 zE%EI%gl!flRLM$XDUN|8t+)XEQZvSC!Gn+;U2gyiJjxR%IZd^~2}5Q}0HUz+3E1I8 znmi7SsU?7909rf+y$+EtayAzFwb10uh~PdS_fcw7mSq7wM+y=3YK7^-UF%MVGhnq9 zDm9p-oJd4uFG8_6odnOr90bW0UUXa-vP80Fa`wkR~GOZ1>5c6W^C_yr;Y{6AaWX>~ltpqR?qc^V%j-ow;zVpNJ;8OR2 z)z?aNh)R|ZMKn5TWQb*HQ`chfPSM9NXB-7zr??7?%<_hNk7Es&mG~aMJ)=@j(YcV| zy^^DV;-WFqy*Ai`GIeIXUUzj`Ori8RGm40=m~drK^b!9`|%0)-Mz&xX+AtUHri2$VK{&I-Bx|U z`i7X*$k)TgfAZ~%!_zPwW*Ff9xSKxqj<v%Z3PEfqJq8IwP z+^a0~<^l$+800fpXx$Je9-${(1p&~0vnWH5L4F!!%zY7i3NZiW4_|)!#W#K?ev=qE zkn_t2Xq3s&f01@;<4JxaciA6bU!2Yy{`RZI?Y{6XiJR?5NGm)jz@xmE`|G{`=P%XW z+I;&8oaM78EaBWQet8&APwy87i^pBOZkX3@kse>4kB4bo-OmH8U+FjWJ?Y=dGXPT_K)Ch@g3RtJO8_L`}}}b z^+-OdfAh8O_Tiu1(@$W8{p#Y;HL=%rd<1wAyo@lR-Uke?=%zXDBQP3=#P{cGaj_%0 zPn-24y8yu9_-1?m`TQP*zmVmAaudY?$df~r=P5tW!{1Kw{-2*@zu7tBg7b1UiG{rV z9~}bt`{Pyq@%KOf^a`=e4{f3TGyNi1&p1{O3a8J$iL&cJcJdKiPu$y@bMy&nSGC~m z;0`QJLRje9i}RUHTl?!EuArm&=H@kYqDtgkGo4QGvE_OLVpK{|PD-o_ex`yPTKWCw zAp6!M&>|U(kXHv9S7g{2Z*I<$Hslsi#G#OIyeLK`kyyd}iUC@cIHTn9-tYaP>Mfw3 zLUc1XHwl?((L(U!gSi8zO=sBUb7$lm-wljBC28M{L6i+Gy(r@*O!9aLe5MlGksm%o zHDZBmB-1##wW|g)j+ALZn*YJm&l<6)DVrdlQY)ew8->FB-#G z-sSQZPgbM0wP<=)Im)!iLL{f5Xft@9-lXo4uKi#T0&xH)RbBux9b%nKmv|z8 z*EBMCR8X2=YHcmcg1e#?xq`Nq5R*0lF34<=p=0H^RXb{H(5J;~F~JjCa42xK{=Gj^ zoJrI783M(mJY0rff~IUscK{^;NmGJ<(&N{gK+KV8K@+9x29YmiqV_n}{B^WkQ${Vg z#A+gk2%RzSW<8`v7Cmz`x~UR-ks*|_zR8;E8-{47ph+d}wZAN(w>6Y!Nz%bh3D}#e zSf(%<&{?6wY^j7;fI2fvUE#w~w7HjD#{Li_+3T)B2Lerd#B1K5mEr6tVBQBvuj%A8 z$gG;vB?bOu?r<8yXBvWq%-!V`)aOb<^~D88MW{D{x0k8N7*wNB+>lIL zukIJ2trvJ8Yh0x$5Il`c;!q}6Xk)HjKP@M>)1bs#nhJF$>>Vn+)yq*;(hrRqp?aIP zmB%3%EfRD9?W>rcb;-;t-jxh4&-6uJ6`dN$I)CJV4c(LCG87pJ+elO`Pt=qxXHK10 z5g^7Y$6u0Wv&zc$9dnb!`YeEPU28G`^+S*w7N^j5yg;7=a|rrm!GB9rN}QVjeD!FH zXQ8?wroYjsTW|%=5~XvJ*1SgN+-ZT0kqHzeTs({8GB+x=4L2REYG1_07-k_%ikpg7 zHev2Toby!PN*=l46_qeD6m?Z*PR9gRas0CdD>kL&7X{j#4U@!3IC_W{nX<=oP^5L# zku4JrMPAMVl(9Uzr&;z{_66C!Xcv7#Z>Jj6&`iie^J-wbl5Vkrq2(EhY$FR+Muf8Z zCZi^~@b)YUlPW6v^BD-Wklm75XP1(LorKri-W>oC*FKslf6}W3`e7M)ep>Rxc96~^ zG3&F>MtPGKns;OofV-w`Gm~(UrWYGiGUd~GiqeUW(>Ckx>$^6fU8S!(glxu1(J_mIPDNITb&w(Ra>Nqj3^BopWlJeHmrk z-$sN`qK|@R8eAZ#x(78wiB?+3HIwgnI-bLQ%DQDFvRV`%rCDA7FZ7xbub0AR zHzmD+d>4sjSV?df!A!WbHPdCKtPQ5I*h6WH+>fER5g4?k#gn9Y)y%Xx*1kmqVii5!b+U-2Z(#hAvS#eo8J+@TX5~{p`7q zbZ$qreOW`|b^U?HgX~V;L~bK@F>RZe{Cfz2|LmmA0A7kSFOz9*+j3kBF55k?*R6`o zJ?7BJJ)Ih?|E2WUepA%klK^-rFE5-+*G`2l)qC z&I+nLkMak{*UK2%>mpL3KZWkPpnrFM9UWc@;K88*r|Eccnq`0)98P*VzS}MSO$wa^ zj8V5OM{c~|Yv11&;4pnUpwsJ5KmVkKY16@}$8Em-`6u@?KmWA(^;b9;uAyRfpMUa(#!czR+hkox+bt1n?a0;}M^DqYnZ^ z9v1n+_1Xnc9O%kPZ92F^ zHLm;1a2Su*^ZvaXwe$M-AX<5KVwbWd%cM}|W4V5u*M_h70#^DUOG#E57=I7095sAH z*Rpz+_Zlury&c{!jIxY^Tz_={;a@NaxIbV< zfOwNPksp$64iI~M-9tfi z(D{U;9yfTJ-p-xe9+7>$2H96XvX|p-ka_dj#cyX4YE3dl@5e{Ktzu>Ec6$eppF@|f znE^kXy}<1Kc-YNtm$HHVvKB;Zik9`RPVO4|yxI(zQR3=Z zbUS1ceF?$@X3mBy_DA1pR-*BR0$7|ydf8@b@DeDrEKt+UEUX7*47L^2+gF#pHiC6U z#4TmKKx5;T8H)zBQdDU_$M9kzE*%%_C&%3z-}@)g9JwbqH?N+389c6B`E_&iGB*#l zg<*bYoQwW)QyI=723bY%vkepfCV(?K2DzCh3A~-MVrPi7!=1zaI)t+LJLrDqP0%Vz z38l1m5fyTokF^^VFO}G@xRM~jj?KrB~(85}h%>!?naH?g%uh3w4+Xj1}!jx(Rw8uWT4F&s+=&f&VO z2+KUw%R|l^N+4ZxZd9Mvn^Z-`=ngzXy=*1U%iypNezd^d#tj#N5*vM$b8j?ievW836d2;TnyY2`rxkfr-m0bgI!Tju308&D#is_AXgZCz2 z5ayUk-!<4^ZqW2gx_5z`qEUuwzVSBxIyz8T%_ni=y%`W3PEI~WWCZMgoSa-hnb2r! zDNQ;bnrZ~CH)hQ8QGlA5%*fo_+y|At)>H*6)F6Oxmz%C?dtNh%!eu9+D5GZX^!5fQ zb1<1s&=>1fr#>C?;d zXbKaxo3V}a%v^h zkU+Y`J@-}>C>=5zu3l|3A^-_@UtK~ykCy_rOK$*HdTuP0U{hN?#p7byxJ9#=JYfX} zv~2D=0H5MD8ZK)=$m*{HTcSO?D)pi)69y39#=WU(AVWY$Xsj8iz;ev;U9Qy88&r;F zlD-%~&cN(lS@BLA%JI9B&d?kw>#U(fa}4X9QQ4zko=G~A`URO>w&s$;Rj(#0*3d%m zu^bl|fXhsWR^rt$`L1;+c2u!WTs zBv`3*LsLJ&#I)oV-p7Y1O@`um&WM(%h@zoQ3noe+T_X-;c_!!9u0%u z8_pvySgT7Y9nxHoEno^;jclu3W2ltLN7|tcvnonfq+b%Y_Fz5Bh)c-M7;Qi(U675Q z%+uc@cW%Xs>FTybtg|)Hh@+_0&<1eDw7EhQ;?FBU8iOp(-`fm zk;yb_h7}T&O3Mz*me?7Or`Tc%iaUaJrh`jf1SQj=20IT+Xr9il#Ue_7g8PPmV*feS z0=2Rf-5sXT@`GC5uB2KC@ou!|xUiLh9yRaT35&8kEV?My<13Cw#Be1VW*K&$(@_Me zM}IVH$|q=wOsJ-7$&u16QB92O{!G3bB}F7&M*AsYfTMiJ&|WE1HzTNee7RR2#&%Uz zGNtHge0$sn3wcr&9bGbNqmQ#=aq>kiDqOmP8BqdLtAze_NxKWEDRd@T@f;0jNrr=K z4GSjVM9$IAgl0wX9vA}pQ|QUIkfCMr_TX+wP>Mo6rbs#jI9i5_B8KzuI0k?bz3AK| z>JHV($AcDSyY$^0aG}@hmPrdv$NT%B$!O*1HZ+gN;dFWa>vi}+XAtDWYsyC4+=RJa zR%OSlAt~gvkgF;PQ1WvyaOaBm$t(8-hwVc^O5tWS?C41R9A;R7X-XS}D?JPp9q>;< zsFR8ynD#Bw{lCb`w31&1>6q@L#{YAd^EOOLWhE~ueIY)m43mY8gM zd{h%TG`OBh_URuEQTElKvV^rHz@vORD&MXJ{V5Be_;-(vzBLAJJbUp&$VM1wK5@Ck z0!23{b&*NTXW0S?h7Vm5i^pSlEhyQF6V2h%@tvj%WG$nFi&tYHAqIxnQ5o4ecHoNt z%^^urydV}plex57kqvJIAr5hnK3qnrW{|?o2wVe~Q;VkQn8REgy|)2=m01OXW)eYw zDxzAH656tE?+{Sw7R!VZr7tcc?=6}JVLmyN(+}agoW+*5@~7c+n!uyf(Kk|$Zs02yD;lZXz9=q;m^82-y_>LWTb)5VCLZPe(_2`VLKp=&kgK-9PyTKragH8w5ieBR5xnCjWjq?Ht&uK%h8_ z?FMwmp5Ao-e)Dj0R0aOehwG1zfa_FuY|Qugn!d`%^U=o|nKQN! z-{J2>gl&6yRyh9s))RHh>UWp?2lD~wtxG)1hnSuYzCk=B5jM|W|2M_ZH=YLn{^|j7 ztXR1#I34|P$$s&F{Le3&Pnur-`|m0CoOm{4G| zlxD~6>=r?&X{%pWNC0q`Psq@4TUSNKlXUCRUkCR4!Mw~H-}KxR754*Oj@q(Z)hH(X zFu;bcWLtFO#dKclZZ zXgvZHHOXT?jd}~@3Fy?H)rAq2FxC-!w#CP(2v&#_`i-wskiXA?=Xx1c)!dDI)uwbr zhOAeU)N+6bjIno|tZ%eRp)V+5lcc3#B7r@{Gy#S~6+_o$0`k~T-4vE1?T7-phU=Q! zspH|~94BP+nSL!Mi@`!|S!j9=pe1}PDCv}>10{;eGEoP~G$+!+p_V(!)WBk-bJ=0N zL)puFvdWEfS=6Gdx#xCvf7A%{qI8?NPC*8{vy?@Bo_kQi;>D#DZqL-~bz5a<46f%; zNFlR# zH7I)~f?RU;wgw8h>-Ts!#in`Jl}F>e8vyCDAAP8W zLDM^-+b#t176qUhnw~*Bu7MD6OLOb_7z&~qv{^AF;jB#$k;aS=o_D#OMUC#;j*?rT z;Lz*HQ_$v&z{Fp7QU{zEU$tk;cymYm%=!zdO+ydiJPNOiU9udm?5Lp)zLf)p>r>u_ z4)Utbkeg~`JJ)Me6?6G6;QhLkfnsn}qd#v_bZtf;I01OXqN!kvkd!fp=fm z6}@ee=&hQ%m7R$!qE4NJ6IET&N}t&pDrgB)MeLl6u4-i2=s4HJiZ_v$n{Ftqh!AcXTe~V8y>L>dx@uNO48@lBS@UfrOD>2LTan-s?~hS`KU~`Fw7daVqh~LCT+hjon$c?T&7Te(})kojitxEV#e~}A%{)g zMK~b9N6lr-DV=&#G)z+tn%K}?ERJN?@WG#g{W98R&-BB{n&@?#kt2VIGzDmnFX$x_ zK%sPXE(D`Jk8_8+n8KvzXp#(eRk};kv9wiZJhE$G;o|axn9m>JzMLQucL0 z2iE=cH1kJvZPRhJ)BYGP%Oc=**|Z!1tn^LyF)}5wuyRPOe+213y>1|6PJP~?3&|Vd zykeYGtNDux47bc8@6h+ZC7cmZc4%pi45uK+jT$nB1TRP3>%j=fUvKECi#+FC)FdGy*8!cM zkFT#Y?>lsmsmw={v1($Qj4Jz**WKFMaPvxZ!_>$$8D91T5fcl@$WYa*&(j8zg6lLF zz>K$p04h7KkH=Ws3PD8MrdvN;+#4yffJtiDj= zB>v*sM*Ly@{s#U;rsQu$-QqQPsOvzMrk0V&P>APG82mZva?6QQ_DnZ8H@m^c_LY1X z%Jh2JG}uT8%Dqix zbT!x`>WWV1^SnTetk|AwsV>vYxQ~U8j?ucUXo<|7v9f-k7i;L0PbZr`JF2cM!wiqE z;i5-srEl9WZ>!|;KVDe$e+>ry*^Abmpjyao!6hjRck@YC-}M0A5k!2tc{jSU4*9`U z^}4s)uBs{_c8Bj2n07^!{cIoeH>B7vGfT!|*KRlv( z!vE2q-yLsVXuZ(N{`DE?`bI;9^iw$Y5s;!C{V2$vgCuG*r+M!JV~2Oe@k{ssXa;Zv z5b?*rgQA~%=DBbqI_|v01B9>VLKs=?CQ6h;h+;gSJ@E}nwr+RhZLB*f?n?JP&d0;8 z{MT)~XLzc+%{L(FFUxeuJbYY#~WX?g0#;1fi<6p%i`ivWkG^8qpn3xZt_W!${RGl7t8{w#ajmQ?G#oA7z4Z= z6*rsDB?B<9wIR3p6dw;_3zREzhl0P3hok0=xw#%*Tg{(i3IFyTnsGnURMvL|SQgih z;peNj0Nlcjc;QI*HWDEiCTP(0?oA|E21Gy!Hg^hm6N9Fp^^oFx*2`U{6m139fn2m< z%7I1PBf(l-Qc2eA({*iQnO`=>{b0ptxKPOLXWpR6^zMeeLM<}MyS21>FAqsSvsTfQ z*g)KK4lCg}IRj_{XWVYL2YQ1NdJXjLApGdBi(}8v5ld%UTQ(W)#YItzu>}K{i&~-v zL=misJ!i_Gnr4}6Ac?}EzaX=H{i3$7jb3jX+5leyt!V=;i%dC2%!4*nheAv$atJ4c zH7o-q#&X&a_6MVJYP_3*A`Rms-lO=-K2=WQS&r*k8G8 zb#I~X$jrwIpCaqlvhH{+;HgnjS!4=y9vnjxt3D?^Ue|!dW@0k-9DIQm+ zGd%=ehOX&34HqgdWV4WYMfI-<aBt`WtfcHT;R;S^Mj< z+$m%R6XEL=Zi3_#YsxB!E8D!`qKyQqYmF1SS+7<(8F$C<9;H*QXEEPJxrJ*&r~Ho( z4!i-Ne%f`e60N#}E8I&?a?4c_33Ymjt(^O==Kx z`%VF;gDQIFOp+K}uh+~;b-te7jzRj`fOO;|1)TscdrON+Cur1YE*xJW)dRenrLF7U zFx|6@^wgl_E`pS=L5>YMcBoTe&NRvdvL!2^2Q~FzWSHXS6r2sr6~pB)a~KoY7AolM zQaU?Q4v8XrPES z>s|#n6kuM%bVw59gd53mWit3`N+I7Gi7si=h|C%w5UP1YcpP7b*qGk)Gpx!(P$Vtk z81p3%+JkVv=4zEd=jm~*M&={aDqJGcWee10GiW^XNtoJnU@l>1B!9U=hJ@M5fE&-K zQfD(lHa=DkZQf!~a)Q9+?xTfi7>`|JpoQ%GnGR^GGpd-Hd2$AoxF+P4PA@Oh zQ^I_K>JM>F*-?&MMg|!hE)etwYOp9x!Ns4UJfxI(xG5!W{-r3&cZbigIr)q&(usw+ zJm0E+;7FU(F4xbGUYGjw<8^;@y^86-{a1MQ2XL$hq{HW5L4OF=pY7c3xzv;1PcO*I zy5829%D)~Sp26<13*DbxW6+cR)&wA&x!EtoLKd2Wiz33kY(1KkN!OVjVo03 z(!?7%;|$theV;IrDda8+`ZXH|_S*Uy5FyBp6B677zWI$*w|N-f}{ zza5^gn9?dl7S-2f5}d+23{Ow{gs^O3i59FOZbf8j~v36j`02X z-%oqKkSEx9~@glSROJB4reTRg}bIh(tEy*b4rHya+HPesuSM*`uuelBO}l8Bc{+b}{eKmT+_A+P+$4-5DP{DVN8HaoH0 z%xSE{JC$JGcYqHsP9Oa07Vh^bOp5s3pL9+JI)pTdgWq_>ZWTxBI~VfG2R_K#jQC$z z(%-$on(#r-cW*w8tL#r-{>e?qD>gun?;K<~9Zx^}C+Jw0uE;H~pH@avmMmO9H^#&-& zrLvW|FS{_%gU$49dwdD`?ShK;mO7ZQLihGZnD}iYVI75KVfK2Hy3P`1FtM^xZ6ETNg;YNi00aE2Pm z?K5vYf>gB3f)`duK%$J?hSEdcx$_ovU$8%X3Jmiu!lt1(YZ%dj`Hoe%7p=asPtg7E z74>O5nM^z>r_g6CbBST4o13?Kxsnita*a3~^s=KZ;NBoohH8uis zIez57Lz{5L?JMxq{Pm(KOG216*VRJoAVc+>rZ?J5Px{P!Z_&X|c{}%M)c_^c++)V- zsx4#&Y57MVU2Q6;$ghu#2poW8yP{QvrX?L~iJ=?%*6M(I3*!bXV-b1KU_yz^Bs6Pe zP1N*3Ld4`v8lcU8H%@PwO12wr@>+pE4Kk;eur|4)zElHdXxibUuN!}-1WoCOaDH5G zWU;U3GDyzD-y6g2vJ3iY`GKxsy_A($w;1L41;%)}?r742LvD(hhsO3D*vr;4VbQqW zSc@o}wt5Nh{B>EnoFnu{d+5YAvh;XHC1He`bv;W}&-r}0F8d!H@vi(hHY&jrvBb`q zX6)r$>pRL;ipLnPX0mIPH>+hgVe-8N=Vo0MrlN`9{8fKl4h`<8igCNQLA3>gjSDPX zyoEYLC6R=I-pq6@i#PbV+8IcK`d5U@R3QihJd7fYQx{rxRAa|ips7|(;$Tm*P z2Wb<3xpie5AMJr{>!NR2-Bu$BZ&O!>9T3fZs6_#gijr<~3LOWkVJ07)Z-Lyt>X?;5 z(O1)SwxSO#3&%GempznR`S%y%+#SkxxaxTBTXBAMNkjg(qizHxB#C&YE_qhV7BD=6 z8%dubyz7!iR07XIFPCzt5a^vS_LSniRNQg7XhVSbWNn$E;us0eAuQQa4qd;!T389X z2s%w$ph37}OY-Sl^|=~I%+yhe@;T^+gYbiyBaf%pmt2%`NYmt+={#g}w^GZw;iYrj_({gLDPq(RsZ5!^6_nlJ!vt?uE-mAG+QnsIrG;D0H2gKSJ%$%~I zaFOE@tu-7VzKv+2xL5Kf+!6CC({C0Wm+i-L+f3vVC>q=K1d2T-11fU|{8~&7biC?y zD`Um|$v&s*O{(5PH6!C`hE7=vn*K};KzI+i1(!ELvs*9MZO7epatyw_8v@D;R%Gtz zZM$BtTkvvoyy5+`00g68EWBO5VA7s$GW>!oEsQ*(!EQh*Dcc>yfN+REkFlUU-oKRyLG z+gOlGtryezGTbL%4=`=W#jRgF)u@r+rbMeWPiQS#kZ%cWka4|fAFfv>Q?%gxAaBuD zZOi-V93DF5KTE})!QG8w@)A*W40aw-I2hihkqO&zNB%H-QV7AShDu`|Lvf1*X^N$U z+ck1Rh=DcSktm^vUT$)qaD`*Sjp|wTaX&r{0VNt6QPp~Nm{W` z!x5seb05?QP~b&}M55LVHweU1{3M|Q1C-g(T2eB#of~G7o_2D|?2G*0rBR4^X zMUH)mS7>)#&_~O!G%XV(BG>p749bkyNGJw?v^8xv2``{6sr|>u@_C% z3q@g2<+Xm9W9V2$N57@?@}jgD#AlWzoKVQ~5Z0$0XeH|=_@qOWJb>U8*MLl?S+K+u zprPXW%h2j8|Ict`$E{wKHP_J=JM3TQX5xl9*luoO^J*{01br1Wbp*EWuIZYZ9A(Gv zu7FbP6>h)_&I%A7ungOT0ZS^=x-3;sUFUHx@ip!0bBz}<2oo; zjTK1$J8;=#vy_K92Z*Yfl4;daohQfxfbK3k8)hj3r_#rs;i9b5Ktqqiw6quFH!c8kg# zRpg<)6CDe_SV7~94O)hdFYkSMh^teAaQS%l+UZ&h;MwXn4|@=!L;GBQH&)}gcXmH2 zcX&O@yDi}4zsh5vyIJvJ*JS$oJp2vp`jZUpNvl}`xQ~bOpT&nCo;Myv`yM^_zp)#` zUpf|b$fXbZz%Q~%ZpQt2bnNllO!apwT~E0hxzvG(5rX&Cvo zjumKvT>Ke))R-6m!`(l)Dj%f{0w}(9O~pXKWgC44eLfy$96Xi2_G7}qmhn_-f7B}c z?s&ai~G~Ur*m4?bhr7&t*^P# z$PJx)9;M=0JjmuPNJ-zj5{#e!!`qWAiOc4T$F4zsOncC!>wk;@ar7$vAZR&I>H4NyxmI>o(9W z-ZwUsK*S1&@7`>pGOIieG{f-i=lSx1@(@3<@eD3XLKkNDSj`!vfY>InSIhE&bjwH~-Hc4kaaS z<8i;~erUVd9WNL%w}Qgc1sN0+mASw6H7Uv`Bdh8V|QJD{ntXYkII)PJaCczezfdereSSNH&-1W-1z)8q!mzzlTk{F2)(_7=pGZXxl7_QH{2%@ znN-D~k+n)FncQDnr7Q7?$Pr$Y(3D6yQ=;wWCICvP!|7R`-9hxK>|l)y5%v|5ucG#%>^+tSS!e&Up0Rj#bmmYnE3xe8~^QUoA7Fmr{=XU7z+HxsE z-0|P|I-S1cp}4uZ>Sb9NOuT+SsK5c(Yg!grT{aTJ%pzS!Gsd66-lPEF!L#B%UajbZ z+}xbCh6X@KNh|o~=JnZ6YQU~sY!pkHOplUpdJA|j;kxZJO!{dR4_d4%c5G!#{d**oWs?k=VYENno)-UhJsmY+oES#UMCmyG{AFmjQer2z=gMVknSeBuahW?AzS45MoRVBhm|WDHRJ-a$ zM5Vmsqg+GnQgU?vQ4?HUT{G9kLd~Iqz6CRlDG^mCTXWM;{SQ^Dpm)@AWzI?kvIBTX zXaQ&cIw$})loU_77&=$H#yPDqFjjR&_6IJ^ie6SUDyfQYZUOF0kx_#jLp2hTDV*#^ zrHa;_$?z96dr+vTJ7xbCy;-&kiV4i+U0oFWIf0rtl2lc+JpiuPd&g!A@D8B{nQ>QO!hvv2L&d&o~oa z#=^m-7tg8tm_&n|ag5`Kji>1<0O(n=i?rIOfl?$F(>m|o8-SnGlBfJ4Tgbx{N+_Df z)P1H+QcEa6L&80uQ4*=CN|`_GV*VH@FUr!CTx31I^;hMnEm=!uR_rL`0AgMaoD$)> z3{|EG+-syXFtoIeQhHl3Rb+ap!xC%2VNOSJXdi5#K|xP97gLGbSs5mven$9yc#1=0 zz&^tMR;V58tFATl5-%$M;I(AdBl!O1^*IO_$uW%Nac-~E>ophxDbkto^ty(e(|Qzs z7!?~r18Kv;SOF8Ps|LB(^$d$iGjAlPm2ApfOPdy3)MbGjH*M;P(iw~_xa*0!;q(AH z;*$;=M5Sp~%DnYgA^DyOTJ(J+oam`sL-T@aA)9yqA?nU+a$MGOeiyVtV}?_0KubXN zkUml6Eo@F^d?DLZn+k#)qAIwN2f@{-;LDjzI5 z7~4CX3<#uKaty#X$=XQgQ=_wg8cgcd&hg%Ym#fq{js4L_(F2f&w0Uim3 zb&;3@f+JWkdn|MOz{zL(dl~(<&>ov!MD|zoe9XpfXLS7=0=AAn|j_ z4$-1fT@EveoCqz_0=Qm21yv};q`TVGppv5PY^iNj_VPl^h^x7MZ_@R$uhNLg*%Gc> zRFvjQbt9XKn1C;Zwy{7l)7Xp?8?bFxJZUIn;FHU0QOH^$dGfgnzr@i&Ui3nW#Fm*l z_(jn;80b;YxBitxgHINsX=rjTOTtUYZ*l?TczKTcFf~q6D(CPyUbT>g#3dB5E)3F& zWkdUzB2(VPl~K^3#cDnjq(X6oK8gc;BNtC&7-?KozLs(-aQW6!LQdkN-Y_|!mI9a1 zoS+Z|F1KC=-!-%9Nq8}aDU5O-E>}(%&q`OML+9*VcH)ZMYU)Bts$NkDM-EO}5Ln7L zkpnQ3?>R)x6In?O0rqRv23c*^&_SzMfSNQF%M?Xn6gU4U;-5*l3N*>(9E;+eq?Qu$!*D&v`e$))>5TfTzh>p6E% zk3de@X^06k*H3LSXP#okY9zzQmED1&(>ifv%8Ks?2|q$w>q4b84|oLL&Bxg%)GUzkHaYsaQ+QUu>ElT~!>Bgf>2lD2#h4 z7o~O<^>~~hS;WcS5Zzhan4RcA8q-Cx;U?7ogx%7X31b3ubAm>AKeK(zl z-CKFe5JFfe)^cm%x*E*_=OHvgY@!5Eg`7S4d7=1RW}=+JAC;&9FSneKghRNJiD*%9 zn_i;MXf2YW^qezQp|vQ^V<~Y2E0KluZ}8c;+h$Qg2&K4+&PGoo_oJGiYMi|5QL|(yPnPTC z5TvaMpb3-j#)RwZB3@|J@W)$f`E>I23Xb8H1)s9@wM-oGrMght$vH?t z!Ghx#{OuhFg!|wQnMQUvuF-V4IW>?6>1i8BaiU&FGaH?Mu~tfr9A`Zvbn6=7UhD{V zUbHKix&jv(VV3I_8v|D2qNl8F(uryi2^kJomJGIHWs4%S^vAvW@1Up$^<-u#=tn5q z3dyJS7@ag*D=vn3DtwYnFRrHaoU*sc0brRIN6`cWU!%>)zOASw4LWvMcTYU1)0CD{ z%ih{l+=M>&K^g-_&5}UigLs~5fV?`IZRtR|##Sd7RcM-__*Ms4B5+{+nIfD-1rt(& zAk^U+7GD5e+_iA{O!4I{ZEDwDpl2y1e)vz|twS+gQgW*FO)U0PY-I!7Hnmx@V-+Pz z?A{$ztHde)$QoZ66SWDu*`op(he(8zYe(UNRyt2mG7c9zQd3bn#xA|Oa zn8j-?U5TB)Su?;xcTR4 zH|(E|@-Cw|9w%3gJSfYV#5GsPz2R$=H{Z#$yWIi)c$|DH^t(5ye9u-1@Z?op$?nq$ zFK~VxAb=xp;M7SRd8C-!ZGJPJU%^O*Ki|nx-$w4N7hgg6>?<2LxaGKif2;2enI3#4 z8R;+G77+xC>LRJJhUU-n_;49eW_&nayatP_XP8TUYz6->zEOn?B8pZEn3Ak z_Y;a3F%bR(`1nQ6KD>Qeb(z+9YoX`sQOmXW@;|&pOw_Q^53#$t`rY%hI0m=qQ*>fH z!(KVT#5tYLFDa)QN>lOMk8(*KOa-Agv87!{2PR*t2{e`E19hoBQ#3lE3@C^8+`Zbiu&0jSYbl&EIPC zNjd+-r&|=(%19p%N*QU3WKy5K3XVKPd>Yi?o@M^|65ZjtoA5nPW288{%dKbs@SbuH z!cFd5I#C?Q7CG0w$^be3x#F&muj#1U3hS3nD$z7J#Tt~|4jTw;y2Ws~xCMlQ>FKbE zRJ3C7x4`Eob8UE$XM5O0n$JXU>=G+xXGXptx78G(f1$qH{C0Sd$ByD;^KZ!$?yund zlSL{Y1}j%iVm*?!8>n=;dF7|D>@spatYbHi(R?wHUDPf~)DG-Vc-!T25&NKgMym)!bzh(ub(ZM3Gyb zl0%C>5&PE^{iFnI;z(0$dUJE0{MEurq(H$_LOg4ovT2iP-kNJ{ASWb_5IGZaM)|AG zOaE{Mt0`+vq9=rVF5;ye*$WOZf4u<9&HC>9<2i`tomE3bf+6tTre$XH|}<_0!Z6<*Y~Jd+#%Ole+7;lU5IN zZ&7ubVj~>p{xZ1gX(03mLd{(z(H13@?%iqSE#T)fjvXk`%C^n0Sl!%s2)ivB%kbZ6 z)&P>IlHo%fcY3|)%ZBEsOqtUZ3}gi{?@6rd>QUjwB@Pd#Q1{i(s(_h5%IW z0D1!`MKm-aQyH4N0Le_I>t1hRT3L9^YQEIPaHup*UAGH`3LzvScxlbtKUH7L+@_^B zoua@{WlWc;#4ADxP+6B5edR${c2|*?6sb?Ve5g^lx&{7@&j{I zk5nS9+qAw1$;O81IpXOVn7i7t>yIPUku>p5_9b#lPcMxIulBQ+@}D3uc}>oHoe z*z*&;2253^Y=b;HO(|7iDjR5c<)}4OD@YuguA~BY1P*##wqwq{MSw#ltmVwg?52C8 zO3wI_9&*;%D^Ul|&fTLA3DiU$gSKrKw1+|pklYnCV-vC&v{BVm`S!Fw-iPPF(ZRH` zY2ked&YMh`;Hs_|G+>sY6jiOnbzs0*Wve(rWV*{y#m?d`)GFW<=vc~UH!iESOH<)c zXILlHjYC?f;z^Wp(HYR;G0Zh9N}(8#&a8zbUePggPUy1$O<1N_XMt_Vo`y<6%BC)< z#895Z1aLPuO2Gx1h?+9865=L6b_H~w{JaG-Qc;1ECX=#`5VVYF$1F_mo&y$+Y8yVH zWmnv%3B{r`tzrIIFVI&M5S<|qT6VNhB7lkuE5d-SeO2j}2rrhKq0%B}ITLp)AEnD=hgE+OLp@QNgUXg}x(m2q#Sf=y0}{ z&ISUuS_@4@s+SvHr%U`LQ(9J58y_D-Ex+sxpl;=p-DPamL_9Cvde2COMhlr`)B{S~ zKaDFgWH8-J%d1T#tC*5_%LW9~IJO~L=6}=2aREN}28V)dBk{e>bPXQD3qw<@W=8n5 z1QkX4Wy#$*OWax*|{v`G;nUv1kHM2@elgH}qcUa@D~E|6xz! z^7Bs$g8VTmTPefLoY4i1j|ypBe*Vd^y#TKHWC$Wjcs>P~Pg{DjuB&EwVe206N&VTm6@8EW@y z;F*IbdD@KuX#DQYFqGf=?#*`f=i&f9AKjhhj1R0$7@|3FbAP#Y<1=`fv|&9U z<<=C{wzl0oS!{pT{3=Y_rFklQl@1u)G#xjTx9Lp#D$Lvk)E>Y?{~f>lO9o}%hDq$>EYRZ8kw6CVtwrHck%-sc*N!-dxQi1-fhF(<&E8@hV4+iYhOWt zvOHgJa5#oP3ZcmJ^opI@mkSJT4}El6gz^aLTFXfOLVVIltj*Zr5}!o3SWS{f^P+b~ zkk(-TEkWN(@j6*fJmH@K^(%!skyWLrGJQ=6KnVhIlSAi}U`MYt^zx#pf?tnB z0?^2uHS!D}*GF?qFA~&oIJ^kEyQbBXR#(*&!2~flz`=4W=Epf)gE&^(@k&{6qljP5!>|$} zLL#K3jVVp;H%`_gvzE^DyUJHHT}Rtj%#|Sn(quHxXCSdb^)CT@GpSBfH!@nx=H$;& zZ&Kl)V$PbhlAtL&bX?`qsv*HFCbnl%>`0B%l;}aD()Z&jkZQTy&h_2lAx|cwa?*9^aWNKGLK>4O6ARse0L(#QP)j?jWVlw<$JP)iFD8Q#( zWrgpSvf~p{83CEvwv{PZ>ykim6dG~0`mFc{w3)j)f2G&zf{*wGMFiNB8JX9J)frV? zgS=wqO*|M=LDi6{Z_=a?>d6~M4i*q*RmtJOdRuF7sVYk_kP}Yq52&h;HCfOZnLy%z z?la{Yk<_5d5N^cmb!BjK`WT2H^(BmWT&zA5>ru)c)z%pSk2ITKA}7KH1S>fw^rI7o zohXes3zilja9Ivp*0EU;iOJ%(C6f}Qg#u**uBQwTHtR<GGB2!pEjYSyPqa8j?s7gjT@P5`L9t;enyZiOe`@QZQGb`J6l8 z#kbLZ(5fiF!%2R=ziJzqJajd!FOnS&8CK?YNr6Fm3I^L_7Pqjb&Je}NE`Ql}3+ClG zXg_(x7O{D73z7{@qF~>kEqw`3f_>2RTxKytwSZ4LDu+v{h-77#nbMOOHhUx4a#4n~ zGMc*;WjE#85)hLHq|S*{evs#{Z05>@HBut5!soG+~U;yrC|f`Xiy7RyTX(INuM^qa1b)Kb<1hyJlD5Q=Oq z*Z$m(`={yZ=e%`Y{p&Z<GKp%R%H9<(DoJh}qy$(MF^$@5` z(wAC*^I+EK7-ZB9 zdFe$GxTx9~8q^QhTl!P$x7Gh{smBFm5xM1w>PtF^@;!_I zA?DxmSYT+J1M9r(eR-pcYZ`odK8^R=O>5+4g}-pV-rs&ZJP%^K`Roy8u3iM->HX_v zyQ#P0n|s4QUr#?kJVF9}2_O8w`|Laoo1+}(P+$oLWhcMh&A0tbc5wMwZ78zxjP-o+ zTTo=k+dZJ(WyXMJxb9C*vR9s@H{04&$q0RPFaUH*ob(S21~M#i9{eMt(ez^3Er+9& z2uR|=kBo=Ra(U0tqNy1L1`xRJTd zXd~|TAnznE=38x@u9p18+&1XH#>qdP{KfG2D5shVDt!eMTc)d(#dLZ*zV`w^qx^db z`7h{ZmI3(re;LrR`|IQDba3DEdcTKu_vVwYe*4WIG4J9q;>yD&7|M8$sIA#}CB)DX zDJw`b0n0wj%ru}WxA8SUU}=Hz%X!?tg?7sseYCO46!Zq z^Y+duhLZ2JNSs?R8FmaYtzbncN;BKnoX&+d2Uw~^nTqFr8snj?L7<<|EGvb*ENnO|XO~tAd>#Rc8mo-`>RETsnIrzfIfQ&&3(UM1h zhz$A=p2#LpLYrZ4Z4MKAj9?Vh6$qJAhf6IGb_S?*IQ^nn)+Ntu1=amFsWDxW6?m^n@lvG zI3eDuYl?<;X+?HGcExR-b1w!kAw!uXjf&KwY0#;&UW#(koWoGmb=@-o<|0WAx>>Sd z7)uV(j_|t_K&VTQqj4ie+T97J8r~-lGkYnz!`haH<#>{>qrYL>R)S-yY<2r&Fex%L zdEyI&tFll&C`{#Tj0L3`FGjs;n!cs2c#%1+I+Ww6VS0Ci{eq9M33E+E!ZtdkVbP0Y6A!Q!`~)anT}J?>tceUufz@cP#!<@JIV@S?WnXk z1_giorBBD8jZ(CrpQP2k$f=7>Rc3S;XZ`b{l?|a{koy!|o&wqEl3K>$73uW`T06uS z;?e-2Inyzl@LG;t_ytbtj>$L|l|az^koiwub7VI}!=mL@RsGUxQe}AD5XkFgi5;1B zaz=_#vtaXnb$AsEr9^sJahyWCqeL@wxUP_XnH%CUWK@CnlU|3sH!*GaquyEAUcO0B z3nyvYwc^**-2E+Q&NP#kc0J0bxX!~#rY6mb2Jm*Sl)_(cdf7lT%pfyf)zno_&Zs|` z0`OJ!Jt$J>oQ|!39Yj>t<1RtbIbmnFl|(c>1>KKBLJV8CmR0#~Zc?li-fu_Mt|`=D z{K4P+`g-0^FX!#1ihn5o_K&CI1q8eY^cNNBm46xE0jY7|O`TrQbWO;t@}fw!=7KQ;44>M*acIAnmgSsX>t6K~+!++I)P?)WktTpR#wUHVa(*}_}1|=-phB{ zO*@CW_@8(f7r)@S-xkv0 z@f)u!m-Bz)v$F0%Dhx{Y@4W2-#H@TLoKDb>19s~-JIZKn3OPLGMYaoYOSXCp>CD6_ zzkgnUToIK;lim3=jQiN4xWF#;VZZdHaQL5S#MR2UJSZ>!@f4`;X=E13Bb>+6ThKwd zn`jZ!c(?h+|9#n;ZPgh7qT}gReq%f*6tbPUYvwfz#PaYYXWape#nekatsDtxU(Z=J)MgYb1E(vYHtSs~ePx(pjaQ zUUgk$*t!#3%JmHq{NOwdhiVlS`O!Y3hN}{lQb0K;7OacHxWeYeO?GEsp_)tH(h7aE zE$VdmppD>!>k}(DHks3uumGQSrolzFByV7mGy7l$8V2AZXV@X;kNdVOIr)H)<3+Da z#F$~Z)IhFpdc#ytey{@3W>YuJ!4jG|`pWh8Q&8-W0&jveE2UVzcN>cuSP9Q5Pwtb~ zP*N!}D&Bz{v#Dh&XL>#MGZqC6+Pp&Dpm~ZG8bb_-L`NSb?S|_?R|?1yPG!J?MSe}< zAVa>{oKr!|6BpP|>RSa9=t+J6i3&ec)Z8wz0;{T`;E?&npWNO8LSZzsmMvfaf-?Hk zy7_sN21*zlsxC8zm}K0(xy)dS$KUl_hc7cJT<=W9L*so+wP-C%k)_aBBY{VzbUcYe z8zlT|n(+zLi~gjF5(Vqtp$3G%45F=*-F27Y*a|ZuG&GsU648p^W)xAW`(8{kf!BN* ztD(%|jOw|@F(~e5s0zq|^r0UsrzoB0K($;h zfC>htkKlX&eYvl+3yuYKw{r<$mO_%M^Pbct1-CDViLufYPK&&;{DK54s5F zV21KXcG=UONrxKIOkHz`c5TD*5Fy*4HgKFWTzB^oxkGC_2>a)TQn?vDD?6svNl?-tyCsFIVVj9*znH~#3f6plu|}Q z)4UHUe%BX8#*^NUrzd^hQZ@vRC1t}1mt|IxFQc~#h*=>=A@gs5lhzwpaUfX!A}L)0 zV`j;la3H^q&3qEtGo4M!J1%bmF-kgnl1du+I8&|CAKlt zxKfPg_bP8wBm2QohNiRva%Palpp9zo=|hu!5m&m*{wkdU&)xfMpZ%U&w{L!4#&6C=_xkK^=((xl&wqa#GOqE?%2fUA4lK* zKa`e6HXd>eP6g~dy-X5|g+%{H0lRn*qFN185ml-_x#1?nvSGe--`6!X?u=h`N8ZUB=VXSvDW0Vh?n<+pW~?_}Uq;Xz*X zu6K5i2=sLDCK&^zgsks6cX`-(Uip&?k^K-~U-7%iqg7y-gzq9Es zoW7Vo>~4GOSI5c->+1z%&geJ&q|y7Zk<~Yk#-C!G z{ZI0$GI}NH3)Xr2=k=5Xgj>s052&8ZJunpPHsj0d{`h|MTLb8vw_f58q_lTMw0Bc_ z4oUs8D*Mgr&;Q#VEpxwiza#fMo;LTGKhTrhVeqDh%h|8qCZ?; z?nkGA86RnjP2q_0bnN=NG&HkW^KJdb(U^_RhNF7Ba2JiuxV+BPUf{#Or1 zA@ka16zU@8R*O@~y30qOhC{bmpXu>VAU`?#8ZQ6wYttK;0+fO4<2>w#Z#;=**>sss zt*5!!tmw?g@B}e|14xl@az?!IY?F0GEot$qzVR(#h2eVW1%6ip2IWe;;iT8aaj417 zNxkR3hjX&F@MJfAXP`&M{C2M@gx5l;hjOe*8l`U~lIG&%Af(qwOhoUOW+K57SyGPB z<8FOj^z3l1Q{O!Zy?S~F9H9Ps*)*)TESL3aiPl0M0R;|=%8I+WzWi6EkVHgH3CvN+ z6nlWc{N+Ldwd-xy)0>C+;Z5MnQJ9vdr90PBNIfxbi}D7xDspnuDb$SiGi%tlcRqu= z-u6iOQ*GIbG6D(#ToD=$O>a{7O+!V_aW7Y!29zrlD8%LEt&qZI1brW_Rvi~9FH(e{ zFy#+F14+>0tC=b@&F6zqvE-UO?{Ybd1_BHeU|{)9dL8;y^a~)zP#(g@0gfNc+o4Bo z{aI2s$Qwa;aea^IS*S?~**?yAX)9a>Q7>x$SU7p!5Sk#As!zf2M!pMIc0{K{ZP0H< zQy|{Y8v*{IL{mI~olV^~J&(USooZo9LxERJ_0=d~&hQ@UjP*9990p8V7*gl531p<5 z9uhpQL*@7SU1zkcFY1C>H50VA>Hte~z97xZ$uqQVPkVV9$;w9Bv;@m7QJ>c2lG*AX zc@rnP#qDCLKdOdBm60ooQ9v+~u$KXhxs$j8$W#%Cjd>uEO86PqYeG4hn6#ijNV8UtX1+6Cu!W0{#P%N@FyB(!7! zS|)eWyL4&+@P$_F6SVuhL0bdk`@m9*c$BO(u7*m2L^ZQ$e@N3){6^j_7nz4iU4E z2PnCtCMm&y+GE3jkN56p1+}g&kUXZ%H>l6axh~EJ!CQ-@m|DsP%0$6|OR@!TH*ID6 z;rIJ6H8ZDU|nO;ug>uEfUN~jNcznaS)T-MJUR9&CpN zF>xQK#8MZd~vB_ z=h$cp?j(KMsWhVn_a;{ZBnf7<6Ni%b%=2>`BU^xefFg>+dbJrS&@&vV7eSFsv(HUC zHL!*;>zTgR$WSf|dDqaoZhF+W zFbB`^;Mk{w{+gU=IE|@g(Ph;lKGlJlid*i*g#cVYqrVA1Q?;3bn?*0Z*jS!fZ(YB4 z;7Bd&l4BHi$Oa3r8q-d-XaSTEMI41oDFKtLWr*>KqgJ;EJRt}ppsBr))2k>u#X*+Z3QL+az0;Q_c8D>t~Z&uQqPy+ zQJX-KJLqXgoKsP;nX#;7h{$2n{F;8+0vtgcI4`si3#d1jnW}*iYjz44I(ph`N})+y zvJcnGj8dszW&ubpvkL^Bl{z!5_|BH)n`bpk#L77hqQg@R$7CPt`<~0ikRcBwsbCx9 zj*Fz&)tDBsJ1ghHQQD6`s*!jkEm!f*l@JEWNv0K9Pyy6;t?Ub?Y<{We;{V%WniRhB z=Y_+pqV``i#bNbOLln@ZD2c&UaM`OGqoa zr{~7HS(U-T@EA@6X0HyAgowg+<6SZZs^zm^PrgHSP*uob`0)Ibje)IEBj}Um-ZoZdDbn|Y86CDrcKY;h4)~}I?l+2%i zcb0RFf&*uE9|6CUZze|bM_B>i@;%-i6~CTdTpIRpnQ_tW<`1L9-23;p^21)M&)w+v zD5M0OdizI+YF@U!paEK(I2~6l%{+$q4KYau42`w^F@n)nX9vFY=f~^*_;eU=KgS_` zkf{13*S|WS-`~oHbNjVimOBjBA7hiHqc4(=?pk}pry=UwV4wHj3?{#ov4&#g?;Mi_ z_K%GW>E?NOJ1S2rG#VXxEC*YzjN@+e?ewD?_`kclbA6;+dBf|;{r$rpTZ^_Dr+lqT zna0g;;$jRUk9zmyL;7zEE?f7|xi2RVeuP|P7@4gM%@1HAewcp#3HqwP`}wB>WLfu4 z_3`Z(S1qRU(a{x$J#sOHeo+r8l#EHFX->`CyVfNZ$(f$hfq<$F6`l4wW3= z+?*8im+N9dwY%e^YIRwtq;Q7t<_24iyKcL#;SfK?kzkYS3|WHtAkO58baON8_j(<4 zH%w)tmjNZV%;T7#A)uL+$dKqgZ*JVu^cJ=FWfU1Es4_|hcX77&fR z4I0=fIR2GhtIPglELn|2)`sPT^k7vJ9AcFCesDz-YGENSx-tl5Fs?n6+by{{7x){~ zW2$Q!OsazR+gio9kp1XBN+(da=(VK!?gb28OXyis?Mw(Rya}qj45e9S159O zUyhL=w(qz^B}u?mjRB{4F4>b73dq!BZMTT!$|rAF22X}66aW-`rh^2-9WG<7b3Z}Y z=haF!dZtSA8(^p*0GvY8qS0lB+?V`9h8e_Pwyo@*8BWopR7;M|Os|ZFW@=$`nrk!W zQ_0%NNTsFheHe|7i#lf|*Rn@BSVk676M|)L`htAxwBQI*usxCX`2>QFM~N&|mTi6t zr3W?gfFEREE1ec-MjAJfc?35E9TO`68A4GVrRG*;C;*JeC5iQpC42-fZ}6u@uqc?y zFQyQ@O$qf`e&IFpTgt)HbHmBgco;(}7H(}<(+&_*$7NBMP$SRm5TVBw6$eZ?w**c~ zn`x>_2#T!%WdqH{Rf&lK4>^@#v>@+3rIS{Q|XZU+}ST4VR&t zMTg*FK!sy~%b`+R>?TE)8B3g%x#=D%JIvqy;soSjZfKNjXPd*p@+R7lEk- zs#sYf-RA_oMcJWlkS5zs86|MtFa`O{b6q$c5^6q=NIFE(s)iB)QQZ_J-#rT#rkIOF zOr<*Jq zC0!*i)r&0)F=S>!*o8S#w9?*HKsdQ@ntLjpfVtZ<;p#$K(MllF6&zG2RTOn?3O-F2 zWxR3($>gKycQ2oQ7dmT=L*dy2iw(7?`KMwHCn~E*k*$pLOV=h-B%zf9nK_ZMPo6?U zEm(a%UeIGfx?5A&JEy>dRmDQQQ3q9qwc{p;q(%|Fo*DEwL0RE5(nPjRD3Jm44KwuM zRCowrcyGn2%}=vXs%9moDJojB3n^c3Lr251bk6i$-SgS&6B-f5>)S@$Oe!gU7utGd zn$;~U#;35m_y~2)2j>DUY*Ay-(%=N`Ves{r91Se!dfOt`(>{fE)3z{`VNmV$7|3fI zr;VADxOg}XN{DQl5~l}kR>8eAjcW*|a~)-@1L%&=i5b^$ENjahq$b1;sDd z^LPy-W1zEAP(Jz|2Y#I9#;G}%ePZ*E1G4{_5CrN=oPvPfug~C@b!e1wkpdtJo=}91 zwqC7s*!=D{e+1O>AODB{RhIMS-p3M(vEeArPhMa&|xG^@2At z1J9SuAYb)K0jk-b)@O?V+?RjxotXFu3EXZywYvMYKmPHL5rORFk)LIq$V|K2{F`_z zV}dsMpOp7c*i*H#6;C6hV7sJ!TJF4fA|-J3-JAV3_rrLcSSe*XR9+vRph2lL z&ba=~E7sxWcf&~*>S;?*A#DEDwZ(4SgY*0g9}`^Mj?@d^@c#X(Cp(4W*PY8gf#YVw zZhY7~kzCaNz3$&{mGSt=^LD=slM@Rp6QaK+;auX7n2?0GRpSxgqCkdE4)=^V64#a zS1-fPt-f!^*URWynB`_oknPkxMfZX8h*G*|KmEoZUy*%&JF%M$z*lQ1bEDdG_V#g5LWh+rm@ zS!9w!W@pWk^8h!4yMr_{c-i8yJazY5J@*fk%*!;TN9JKZq2f>KOXeQcs|tl3ji^k@ zbjkEmMH1HyfI^qMoO{lXG8huQ9^K6OJvP9?(2wc?DLZkLoDNq!X`^z;#hG zNoUaH{t1D_4#hL3w_6qiJBY7$l$VK|)Z3eqJ&>sUb0pE=H?K&LAUb6BUxSxDehwp4G?QXNFWTB56( zbI}b@_!Owb?T<~HnVciU_ZRGb&cd>DrpR?dd$Y_#?&m-Hz!))Cm5>~Z68lYNxl=z` zma6LpkOda%Sfq)?3PiRHIlXSgtSbqJX!WU2=`6~wXi{=0p-x_dXtJhHyp*r1p*)o! zqx5%ykQTL0dbdGMs>1MX7RA9j&#Y!5jm}v&M6?bg_mo)3{%X;2VQ+yP0Lr%ICfDhp zP~A#Sppv$VC<$Mo`+*bIbJuX$ny#cy#zpILE>A%zi)Cwa4!t%Si@`PZ0Cv=}ptM_; z@Q>6A%8ab$8d$PG94!BjieiH&tnENO$!&*=Y7?r)bm+!Cq9w?=4ZVuQVtN^LXw>NH z8HweuxhFX+Y&&^t>fpQ=Z{PPlOYGfnnQA=V`ZnZ7rY0ROE^u%LG7K#%Pxij$w?t}# z4*RH%C1-e6bX0DOYW$__2_0qBLQ=Mb93_JUW@ty5mmOpL;othp=##$wMsxRQX$l$S zk7mfr41M7=58lk(cV(7$jBi@h^^2~P)yd$IaJeuHvI^B^*;Qg}af|Et+RE1I?1huGS@pE`nIkgKzPViOVs(1;Yu_f1P zj&7Zq_?j7`e)^{|8WDq{&dS?yH?!#CUUcHd z)iKog5{`~b9C!-@$W1Djee&{`5amnoo?1Ydf0)cl|56k>Zi3U5LoAmu2MaC(gOARP z^6=gCx~aJ9Xo2oA;z|~b@97sa+@O_{`&IokUOND&Yt=npnIZQvmlFOHibxZR@EYx{?}<8Q%9Tg$ZMlVs+Sm9QGzS&4>H zU)wYQoiMjnOoasIm<0PwT$PgWyv~y?R=(vI-aWyRBcLqQ!40P~u3CPvCs&k1h`H$H zL1pH1TI4)!w4Gg6V9rwY5mG>wUuBs3$tHHyK>TWsBo}Hg(AKQkeV-8vvOE!m(SG}= zSJ8x;U=NQ<*P!rCkTqXJ|FHda(skC}Hehzw;k zrA7&j_P#e(&NFsnuj;8yS;_2VCrasT8_*NCbe6^C@(g0_Z30Gflvj!hgR;&H1AQZG zq^$F@F&WL7SR)0RpC->}m%D;iWd)caD5(-s`f4PLl4TH(tn%eq31DmPK)&vm%|UF( zpeN0)t4$%RC4r^AojvjlLg-T9q8fv0Dy$DaDh>4l@H)??E6BMmc z1uMRvy;DawTHQLY?W#l7A9(X)T(^xb+c(h)csk##YP{IT?{?DRT#3`QC`ZavLe*#3 zgCJR_nLF3D{A;uu*fYucyT(MyFIK+;z}KVXlGj{!xueIs4&Tn zg^H&yG5t|KeDXHYY_U|szT?p)G-p7QTJ8^HKiD_Rb5M$4}z;lnQ`vfQ7_Up#eEkCH);<4=0JpWOg!t?Vl zhXeMX%XXZe#?!?vvryZVJwm%xEl9=pklU|={b0WWi^MqjiQzxhvcfr@!UUf-my;WH z;0N5H;@7&pV{b(QR!wDR`|@|fir8y0)8QVXepT$)BM=&!oy=qVwdIhsQ|=yU9am*M zJxqgJF@Y|*N}>o_w9xvKINEkRU4vhhZsDySDYN8Rw%LMCxWoh%{S)IrOVTQhY7 zb0!%U2+QvTces^@V8XXhGiU+EV}_yi`pI(+qiL?>AY%5CWm=Ss+l?DtEvK+yIA5re zHV}GbeJ;Y*S)A%e+ZZ4+-!U!URi#v{z_>{R;>$3N+`GKa200e0HN^DECP9_@M!aIZoZbn8vqFM zk*VxV+T1YvS||(EV4Etb7OMxHGpXAu!8T>R05zx+>6cKikgFXZCrfatP5x3|CAeN{ zBj22w=wouNx{`Q;ekIhOEWy zx=IbO=+9t=vACL387Nm1uzoHKR)%TT7rLAk$iNez`H=dmvaGBU9e%zMT()Kza;E!* z)2#+Y*K#xX64v%E%=zl4>5!hnfqaDJP0{PKGR+ToKVrR7?RuKD&-BmMT&g8 z&MPi{X`G-YP!vqnUi7j|yo#7PmUQ72I2UQfcfQ;Uf;Jjaq)YU2RTw_KuSvN}bW6~H zMI`)S2k)2^dfB8<76wwmRIZZO_2l0fv=90Go@v-YwM+AydId`Cf;Fos#>{l3$qP_( zf7CH?R^nuHBTuXZoS{dTCR+?5k$oM-8*)0AHTS9k;_~_&yu7`59oi1MD%3C*8u$v3 zn-Ji}$Bt@}cTBK6cZq_-wxBXqR9mj)m~9h%zELM|-2eo3Jzm$OuMi9=2)r3*+6w@0 z7&95Vfo02M+?`@kxo!cejo=UHm3D)uxQX+aCjv{66wL~v(STG2}`##R5?W&raT8!Ui zIXIUiDkG2cJ~SugkN{42x~yf%^%AH!e%`rs1hw9h%g5rI^?@Gc$Oit6j;68)%L&69&pAY9Lc4qo~L;(TiW-bqqs` z@ON^KWlRqdGEGq6llX$>ey3N(o2(0hIn2ciHM34XnawH<^8Q(#vpVB#1cw9mrN|9= zOiNiTu*h{{mY{_S{Mm9;l_h~vmNewzp;jd`C%l>VL9MxyjWXjoG}D5M>tzysP>-Gs z-G-@5jRyhi>HID{iEJz-^8n42Of?f9u1)B{M9kK?rIScxHLm1=WVe$^!M)+Sg1k`; z2|=dUw;k8sq^OyBAjZw1tbo=OvWW=zvli2sGhPesc@VjDhG|ITamLN)GZbKw6OUTl zzsTyZ2XXv=SfTq?9GoJ>^fHjP82SB=FQ302 zWmEm|-w$Y(Jvi;yae+9%yc|Mhw4qgFZ1T#S&8f_AvMJPlXnK*R~C4 zs&9>+>+dJ9AgG;2MJpw|xH!fOCrqrk1We7-M1T1>fkf$iG0%a! zh(RS*HbOr6Hli{ZFeSFD9k5dNz5n&U{9j#Tb5csKyXSX6Y(+N;-dsTQB%ne&`6JAr-SkG^GXCYS^#MOyxeW10L+2ih!A>`;S=Ig1 z_rR2mGk2?_;%eN`=(Cqt-p6roX9?a8qz@Lhb{gOMJKeYq54(e1UCmMz-cB>SiOAp2 zng9{}O`Z^M=$}Ad*g=op$p;Gb_paFh|M+WQ4H@d^)cLc=1bAj!syiH;tdjdQIPzlg z0aK)&cJoyO6p)xL?WBl^3W^k;=-Dni(=`nG)}8X1c4ZeVNO9|O(dLDfZzDHD0T*cM z;ZmXgp&Xg1e(q5RetD$&J@-ML~4TkGH(Wqtts&df)!IPlr)`u{V3cbETR9{C!ySSv}YQjO# z%PcFraYd%CLq_2G4MV09;){s}x&=g}tsqpzg2BEKzHTedql7AN2QgZX(Y-!cms>>^ ziks&~CfNs?1OR`*3kpG@nqsv$^VhZ@N2C{x%ByV)9nZvqfw$)@px?yI1UL}g^|I`8 zs^NU>0$}FKkZU*{ILsG^L_sYHOOu-;t< zf`DmHeeE1PQwJq>MN>+X*=55t_qk{*Fo)y zxlL!bHnlV8a;Nmyg2}tXFkBCU$N0!hhrm4rwSo5&>glY!8#;oOk3eOPi(buJP)oAHz)aAc*CmztmdUZoptwHg#SZUYG&*S5}q`_UiKA z=`|xCgrN_<_3$3t0qi8}_b-7P+8 z#l3gSerGufDsFp|f21YF;yy%$V zX&2bMbNHPMXY)YEA~*738R$6k78RzeWCQtB^K^lx!H?*^sDXS5iZL`(s1_1n6f9YZ zAl(^ctsFjdle@6EiwsCb?v!7Ys*0@Qye(6kXL7V;RM^ysBO{(QOex|U*Tp+E41e#@ zAs^5L6}mMO9+W4XZV_wIQk&}m;;FYxDLR9n^BN#sPIkwO5M{2MHcg33^(P>E#2PxR z&LDFX7`?v&`v5&6=UOfcc3$7it@oWQME0O<{!c9c!X_-3&u6|5 zm;qoG>C9aunbfeVnw)v3B3(}?17s{<90+^S72U8NC$eV3m*JK?73fE13W+6O zym&=9JEV(>C6H+u!4v%IgNl2>+wF{lm*E#*h4$>g00VV(aX~3Oo%;_=3sg5XYKSxJ z!3JH~@F!mAj6+!L`WfGO;(gRS6Ls4FmzCJRZ5d9u3ycd@C1(!#IDyz775#<}6xF;g z+qPhK1C17at-e4doj>&yT(bWLyiP;HmBn9zy1qJozlx8Sss z)-#-mtP~kgu8w-JN}eb5W3*w~7wl!>GZ9tbwFmQ?q2p=6bY%elEy+&`xLR;(SBS_` zQ_It@=TMcRb2#o8Fvd16C3w3IfgU{rzKT1)BhCg$JTsFHQi`J*Ao)?ywsPHN-Zgp2 z0PJOrR+eN5)c`uEHnVJS)3&W40vl$-ZomOVyUYSfCGKibHk2-1Vg@rb?91uI90CDs zC&LUDde%YLs**FRyQYx)ahk~Y*w>Xr6V#@%tkFt@W*Fr-9`1HARRL2i6}>#~m*-1KsgqtqA7<#a3<4H=X1ffB#skm61Fni4^EB};MpUB=6r1HPTq z5%3i0@`W1qf97hUGn1_b2<+>*vrv97uxS$hVxihDlO`{i#KE^DWpYMljL@75K+X#1 zTgG$|2{xb%+9xIB+atpDvSj@*6nNZa*>6k&O7J-Jr;t7nqr6}q*qC)P18yq8u=*e_ zOE`3pQkF2~SF$RE>JS>T&*R}TY|(8lX91G$OLy23J=-RcTBJKI4TDGUl(Vl*;m>kloH+adx7W#VozQ^sVX!#vxhV#9=1)NnB zoNYG{Dv09~I`XyPth(3RmC0Y^^G~T{u>ZvutM6RZRhden;lh8tw{M5CrPbwFZTkJ- zxXp1DbkK4G+sl7ieV)Il!+pNE@mK#|-gPT~@VSW|LoRQw!_&~D5Zb?n;7#KP^08<O{^#ju8JPS=f(4#b*Vo z<<9cgpUbah-E8D5;PuT41xJbT0q?q<8^<(NI3tiCrs)NUB@4HnJ*R+-vmsO0AT zS^mPh0w=#(-BaNCzL_ph@{iwIl#zVz7Yz^R7j7p0cD%cjS^g)x4nMD6Bvyx7%CCL+ zMLynV-yKiA%&$TM(1dZI7k}+|=f9VgJ8q|o{6WHcG5KWuEk3~NK|W=EBLPJiuvu5U zD_6i4JV)DmxU2#W?)yNIU3Q+jtXIcr+6M}P&T-yvx1*B-pOnhN{RBTj?LZ|D7DR## z68Ump&3`!_Zx6RGWL0kEk8~H<^>48C9IX6zz%N}6*s%5xmw)WvMVqe9mesj`>yQVU zSrW5xI}3ZyIzX%DMKqX}>`S+>S_kyyYA+l8Nu~?-6*p1vB-h)KBA49WoRALIC5@vhN%%qzFv@pvaMZVsNZH8^>(mp$NZ9$-AtphLRz3;n9-1C)h6CKB}zj z;y7j8J+E)$OiYo;{OI;%EyQyY2ZCC6eD5@11ma^PI96>w*Al-cFy=E|m9Z%-L(bW< z7HDiHgS+2lmFv}v>FNHr|M7T1RZ0k7qy*??p}VW{6>BQA>JK=JtjjoW>`UBT z+g9X(uGd)gY9Nb(D}L+ssrQ5u;#tz;%_#S_tz{mvy82T7x-nJ9b)o2la%RZQvY1ya zKGaG?%R+&6^VaKm*40#P*NI_Gny%|ALy;x%M%a!9<(X#tS90O5W_LQzWsP=fK~`z(c0o8v3}Vk`j9Rs#RMQ> zk`QmuT5w%U44OnNcm@Tq+a>Bvh#H&)(y)oH5*$(g6gvW6>1#NSyx=btbGK7c8E6DF&zte8Kf6!l$t&-KUqq1{l!~yxYb$8`ptGSgb z#P{AtwivGZc@z#P5n3h-m_vSRWit=lP%`_$&Pt9NX2q1z4OfxaN-h2H14u|E>g28w zxNN$tVW3E$J}PT)4RFFV>dvpA08DR-x#X@FUDJ|dw>eI4^Z^T5HY|C1ISeN))PjyC z=cbwm4;YfKQBj#^QO_>(7HXxmNx8sMO~g54fZ^R`1}+$Y0J&RFQx4unZp8YB8eOVQ zZ6HN?z2C-lrxMw2L!aeJQyu&woI71=%xASJb8dRo_*iX9tk3I7<_<(_QOMz+aatLi z`1}2|joRy>8Oj7`$Kf{APzn^h6MV5FIBd17-dZQ(Bj5oKm?+Vh`mvSjZHk%-gbU_^ z7Pa$_$S~=R8l+g6(PJihE}C5Y4LZp0Wr}N-Rs4FuLurjX^rK#Hazk6~xPLrOiZVef zlkEI_1iqQ25t=NqN6uOsg?OZfLZ1=&rEp!m|8(WtPXt-*9kH~-k<`m&!x?lR8t7=r zhG5!6mF*agrviy@1{-JsEtp8yN^@50-8k$)Y;pb37VH}!M55==43SzireHYjqioK; zJSHaMUD4S#?)T$;bVy>W&zX1R&Vz`G0x)jc=VSW&q5wjh>X%C-B5IF`zVnM$ga`29 z->Arlp9S|e6}t=`K_roA!W}z$x^a1#hBNq)W9xw}S;<(`{&IiuubhdGUDFT6TxL+d zXvo=@Fsk29I<4jSY8daN4vI&&6lm&CAS&mgpx9I`1=km4@6IJ+&6(U}c>CBNPC=KU zGYkR^2})-INiT*1=(icNPg&P@YEgnL`*bi{iT_7qpPSZ7F*IRS^)_%@*M)- zMGaCUmNJI(q4$s>skkh6+V?ws&Z+^9lc6|=?aoIBd6l;yE1}sG2RPi%iYV{W(-SSsA=TT zGpE;Zq4;r@<*4d{p(`73>}|a-k{e44_eq`#;L+O@;lh&45f~SyG_y!p4?%%M8_kAC z#g{;)uA%_z)O#KZiC0QOHO9zvBL`qp(^iaQI6x|(;oVVj)+Jj#RtNe(8O_OA=Ax7- zo6~P(m0r!{D(yCkN|md5&8&ri`^!LPDNA|lwO*INA}2rW7_m6WzFV`>{gP2a1`<+t z*T?JJ%U}xY$bD=T1U3lxMzp$zh#CY^irfouxAV;;0DLSd5Pa&Lw zQO$!)!rajdEy0 z&_*BuYdD-`Qex$h8?X?Ku$z0qi4jEK{$AqYngrtZwttetM72ref!nH~ue4r!ahDu9 zS>_a6uSd^ml8g6~lhYXSNr?r(A9SuBgUX5=dsRaa#QS3y5|ihUVfi??B!g;Co^{dn zzXk8{+9i}+?Rd0maLQ+Y***FzznhMhPT_(^lbe_s-X-nf7=V-CI=kNt5w9d1^yr+25} zX?pAaP&}XWbeX@}{LN1Oqun6_Ifn=IiWv62Hc-OAf5KQlcVqo7G_rp`4o4sn#(No? zZ9IZ^0j@t9Iwe@>Q3_3i*%T2%i@XLPG2f@S; z!_hJxuZA=Jb^dgp|6m^$e>3b=>>~QcKYGJ2EOrGhg-#Rwrhx_H41$FJu=@RQIF9%#f0`~kxZBUa=`RwDZ*Eo}{%Jd1Fh9QS51<#3 z$$SyRZzub2uZE+=pG-0!lig`9yVX~SzekzOZiYR7X~S{L_%g|&2?8rY^8zEO9G)KX zakZ2D0NSFnTSM-x6;KjhC@iH9zZ}O6$PHvp?jMhXUC{D(H`vuyf1EB)aa9)4&aq6&E1px686!5ol*sAvN_cTgT_P& zyak1S>ou0CGswkzBU+-UijVWDVr8y~vjGtqIXk1<@2!Qn?^R2*M*UH-7T%l+vi??3CJ9)hf z84Jo-_I5KGJ+VPvu;Zr-)LE2G+puC}*sYzB#N5p0yIxy~sac#9dfj+AqI4{qc$VR~}3z2RJQa#idq1&RfK(>g}30`QUH4^sF zTf`LbNb@BM-#8HMbx4TwFSt|588qmErB-q8200t3T2R(u>pLw0r{MZs3{Ytxrb(y~ zcg#A2va1YbuKm%iCi0{NI_iDav?-B_&fkGOl4U-u46y=A8L>##Ajtz>1cz5n6F;$^=eVk%G$v@w3k6J z(Q>WcpVVrT=Q+n=a)K#ALt7M_{)Y=co(EkSwLD`&=vX~{XHqq`RrLCsSZ7Er>Wukl z-j1#-1bs;j(MpFPOl?s8rbGt^R3NLAJ8U)Z#f9=6u`+eh{?V@Ie%GS+b(8_!wKrF;!igF?cHS_|ytB zm{MxPb{ql*MUEn#i4laVYD+m7)0?9LF(5UP*j_KA*AkKOC=;SS8u>b-#v&AWi!B{O zqC!fC9zz^DeN)sxsB#1XF1=hqvOYb-+!kL!#GFaGQ13bRqldSkKu>L`&L8#`5KCfh zA$K)vI;P7&)2Cj}YsO24_}n}zA+{y2_Q$|g56z22 z*)p)NB8Anl3VW?{Or+zOpA6N;k3%WK4u6)6HHj z09jy?CmCbq>+eicv0s1IKFXp&%9XO|j8(4YwbtqK4njy{k4;q83HbGOl8$leMTVYgyXs(2{J`$$;_)i#x9& zr^NyVk2Y{sgS<%SNh65Y>zH>`?ps10Kp{cda-XJfCOPVX3APMHSjt#t>|<$Wy2!wT z$0=GQ69N%MR>_djFSzuLTG%DXBH`~8rj>Vqc~aR5gxR64*)>_l(gK6aA*hK=EqYd& znXDk674&GY&J*6Yd5J7_?# zzTyKCDDJxKQaPjD7Vs_T{8gNqmqS?DBV_J(H8JLgEVj6RS-~&>ILrD2woFZ{&!zah z1yTeQ{7ATX(sL(Ak8ex6(a(pV%m69|CjK6u#@+aS-fs}2*9l@{O{_Q@aYsu|H;f0! zqLJ&TYV~ymXG)zM16M<^dN~E0_eP6lk(7{i|JfHPe8<__sgVb!c(^-H` zfoaia0JuR|#l$YLmW{D81?10Zazc31LLSNB8p|)G;lS5e^IV4QpYfb93iyf`INuf zWO9CP1)j|$y!$w7+O+tgD-h9H@IO?*T>3 z$Um1ZQ45e{8fc!eHta%@CzHX=5||$iY5Zf?S(&jrq6^5w_$;I6Mvn5{o}1BnD%0-b{S*K!&Jy@6cDvu2HpJ`|*7em8tnctk?piOZU^2BtU`=RhHl zqwlSwu+hDg@A}7n$-<7aB2JcP=yyrz4UFM!!1)8k3T`AE)w_jBo-^7U_(dgpkVrue z5e{a=y$zpSc89Xz$noBJ0pR?{M`ShXped1w=UMR1@b+m8*YgxZL4;}t_dwZn1O~d5 zi0(YzU399*sAUoxQ0hRY)4hG=2ECPemjCv78Cm@GRqcO|I?LCt_620ZqOf-3Dz#Vj zPp!fN{5;ymSlr8c({E55zS~WkpIrgbxB0Gyw-4j(c-G1;j-&8vhs%DDZSb_R+8FM~ zhPRK?Y1C5e0rldaQ5!#4wg6X%+@A({33>LH`f$ER^?iT4lRGqe_@-^%vU=(dtFNZ}2d%2T{_xBD{s19>KS*4$`gS!?gN#+K%_A_Kf=YC_CD!oUa65mlNHGm?aOa06jkZ zS*DR4olw_)^B^BooD?0*2x@Q*2XKx(>f%-d;CIIlzie#qW>ow!IFqY{eKYudT)@UP@Shr{aSeJH&C=xWG8 z!C*jIP=@&YW^f(cD{+0?)LgxpN#h+qz{!ubvIYdojQplurI|9o=W%&-u#o&k6Rh0D z>}F-h@^EkK)_w|OQo5plzn>JyfY19}+kjS@Xa^)j7R%A0=ue(-!*LKyhhDzRsb78m zO~3lXgx-ch`3e3!94^rESbaOn#CPL;V~0^cs87Zr`pO<=xI@`JC23CQ?Bf^g$h9Bg=l@pvU`2Yhn(sL_wIa2*ac=3Drqozwkp zwfzkH*td2r5B9mC1h8E#9@~BY`Qd{g0e&(3loCV8e3AcT=_G#d4oQ8ro047)Uj_Oc zuGC<6AADRH&*{`(j@TOqF#X9G>}4q&Eiyz_!e-vRVE>g%6XVvumF?GZC;iI$wghU7 zg9~|CV+F5EO3TmH9QlCaOS1E_UfVC?EadFta74yNssEe0U{GjOBQM1J%M+@@vg+M+ z(1MfdU0kd&HdVk-z4h8|EY6}tKRuI=YAOxd0g!J^l$_jE@c=j_IXCB_0^W8Nu_D~t zeaIQW6{cV|#QCUz%PTMyrJn-MrV{U+!nT-Lz=+#L!>xs1Tn4oyWx!l@50~+=naeV& zutOn%9>8l=85~^&ZH<4iJ6_-fYbs!ndJ7_`X1`{6>f5!O$-TiD{hdp()(?= z44F8JNPON3C4PjQ^d;{-03hpf`j8uU-dRVlYq{q}JTI;PBjL&dsJ=Q`G2$vJVQQ90z1M~<>)IghCns`10LVG3@QcKQKF0mpetGFr?mrao| z%n}tmpLT?_ESG(J_zS7#x})7O;;iWMlF9A^=D2A=Y)W6;`WW88v@K;af}`e)Few-m za0T#bps>vZFr$ZI)Op5z3^A??M!QV-kp?GtN3aQqL#v>33Ks%l&el^$N46uxNtaPxU z(7}2b?pkK0)(MifC{6%7!DXK%{H=!8FZDP;bl~0>`(-CU3GDhQU7)Kb8MjC80=Wc` znXCzRd+gtj0a(GU=9(Z;gNl7`Y=$Oa4XCPPk{Mc*152~R}O^!goxl{#CQ6d~xumZ4XIgw)j*LnOnCA*g}x_QU~SE5 z)6j|}z7G_3xy{se$2t?Jhj93~xy*GT;Eof)n?ID*rj`hihg{cT32||{NdQ}wNW60; z^Psr|wW;A8u>4#E8!Mp+M&1Kps_t^unVP$9=bCo`tzquw|7kesfiIyp1vU~bxQaZ; ze~N^Mwjf$)vRqu^k7xA@Q2jH{ix`21r-6i1s&IM4Wp<`#^C6< z5wVbyw#jJ?xG2z*Y_uTjXC?vh|BoInQbL=C$qf6a={TM}zeR~O3}?6%6@%8F{K-#Ow@E%x-A^i@8(eRAyLKtc+Kn zJC`P4aSXT@RBICBJ#0NGNhWW>wEiNT92a&^l;9=krLj9^fix zNZuQ->IlENCPevtfAnt&9yN9eXKt*#LXQl?eq%1STsEdIr_{^>g9bc!mnR7|{a}_5 zxJy$hns#`2He>yBxhLEg>`BPYV9%fU04L)DWhpI3gAI+KVTx zY!hvT70sd7!J~{!-8IZ>wgd|RfK@PFCPonB^fOFI=b1GSF6T^9bd7|mA;3Y|v1??v z0iQ!n5=@W17OYK;UZ>Qt5=b~u9w6^KBKrVsK$5>RHYO!#xaZIiZzp5U7!7cOzKWyQ zqY;TP)ka+A1O(2}=!AzY3oPR6q5a9POsg4V$jh4`@#Vix3335)C9j|(MmFxQS_$Y@bujxNR>T$KP&V=; z3`t(%dgOqvdBy}_2`nsG8=XxNs>jalnw-F>MCt4bsBm*Yf1b8}ha|P7MQJgb_5VW!UbqMfsifW!Ng`671l)Ibh9kU#YA}v=Yqeg@*hoB#~0ADdB zBmTs8%kpIl7AFvENZd|P$jyUi=75SJhuk5B;Dgl+>Rkz;^99a4GiVkx1dJL&J5+{E z$gtYk5_cz7Eu?AM$mnlCB>j&oh|5o-Xn!YvFuj?jIF=F&H>hkYKpOnkI(ZI?@p?Re_ysZ_-teS^ z47<;Qob?S9N=mD&a0M>OzHqP%ixgTyL}GZuX~s8r-HYEzxaJ^0jz{9pm+^dt2uLlf zMecN^WN}&jPnUjNy*l=X-MiJl{-^)C`U4=b^0coX#F^a01B}0uXCLo}13tGOVog9> zu7j6%@P=+yFHfWV&+j&JAmShVmw35HWZlc)eqM|Fg2(wy{5Y@s;{jM%8L7`b9K$1e zzL3AkYJCO&gWnJ5@hsPO{iw1sc&dHCcIcOwe`WPK!eIPJgs}31fsgau5ky4x<3P}E zbt|7B78LP!*6C-lEb7pluBMo^{T|=$dg{#*9u~O*L{vog@Y%^tveiFW6^?JLVCKs1 zeNKYMKky;oVlMKR?}o$b!+*IStY*vW-R1C3K`u8IJ!Ua~-)dVQhDniBmvJw{?Jea` z>Bk5At-*US_BMKzHvq{^Vo}7)8`Hs)y3DJ?-WB7B|Bt~tOz6*?*O7R!)7fDLG1;ER z?ZrLs_0UE0CP4h6|7L+|;d5QT)Ne2MAO4#>|ChtA-yEm$Z49`BL{=|T?_r*TjT+2R>~oxkqFcbzO7k>;n5yNW{@mgXRJa)E0(PUF-Fe>*oL*sIIyF@d^(l z?m^2UCZNUGJX1Ii`h!xTjfe9{bj(4=5{#3B`!-QNmFeQHXIaN|+0#*(%$&$bMb+$( z3b3RYH5W%?ga9BxZN+&cJye_pXbzbwLU9TP@k&`GX2!)Ujb!44vX@}ztFsW#k<(Cf zh5)djs_ZB$>=%+Vsf&`CP(!sE=-Z;z?3cG-`E|MT@>MqV|t*BnYi( zqw7Ocdz-a2!>>TvqnAy=Yxb%DCp)QWnUMN2>Q!;RsnKh=h})yM3-FnLs;1gBpeW=` z&TE@Cbz__V6YJhxuTY7kKw*KeYGkHzPgRuAZ)CNUEDA9@aL%fj$O#&gAe(*D`@|n)Rt-aT zFrn=Ax`tQ|AFRh9M^b}|n}zR}QL~(!U>*6$7gL+E%b0N|BJ{wa#lWG^Fj?o|X1RXU zCq;+kkcVKZo^kFxlo!c3iyIfgYdLIkUcw8Ai5wT)-t40^X9DS7D)UTod@JMtC30@* z`N;(yYLd`p?vKhSR6#zB^UbzBdJGCLd1L5sj*t;m(IvL>kYE6}D}z4*!lgn*OagPX9c$hP-5a1~o3lty-Rc5t!X^1=!qJxqM zJ!M?7L=rwgkb*90mY!X9Eu-hTY)HZ*bKvc35AHN#&5>;yoJs-xO&L{@>~Fdx1%To&-of+>+g z2X1;3l=#gRX8m%(C4BN3xaQEjieCK^E`Ojne2_WAl> zVHz+8GQLL_Spo8KrZwA*5*q~8oWeFLlUul`0NW}!pW#<^K(Q)2RwYEDw;!Mz9$O-m zWd-d}mo74pF>^bEJM4Uk1&{}Ev=SfMKbo4!c3RQx0@4JBMxxo)CM=ynB_T1ZSl#xX z#zo!+Dj%smK(9knoVKAD&y^P)5*Z8v@AV}h6T+GfVK4G!ZAjP9B3@RNp(4)}k*L0z z%m0yQNdq4g=#e8brKocO4wE%nBuEk#;9%GkOxN{jbDPd0!^qN$pU`L+kSid-@0fNa zlw5;Z!Ik0ldKYvg0O`EGQ|77u5l2@PWm&X^C*44`i3=uiO}9aN1amS*02$LjIEt_o z3-px%2~9yu`>Hftmr$0WnC|W4u#F1QwcLyoI*WP+$QD(HD+{9fxPGF6(~?u~!Ek;!=v)NkU!L*PeGXy-4m)xg zBK097j!(f7wX+%|umg2VRZGxWY`jbc@)}Gq;X(5T?S&n+i=D7U)u0kjm9}h`_VIha z$rH8HWw(Waa%Vzm)+ zn#c=?o5?`J#!_fkwNIFtj0wc3EDs&tj+2mHrWIRPThPLYF%!_!D4A_!0Ynp&Nht)GN9=@J`|QDBW(eaB#Im(_N}bR3V_uT=lhHW7o)7r8GJuML)_1Q zImzT;pxHazzkBo`1lwblaP4OVhPcGyHZx3T7F$4OuVcbHo559!tgQh{O9zBUZzR4- z!rk1K7j96}X-L?zQbhbnE{B#N!m;{eX8)5?_d&nzo=QOVc0tU49=B*{Tceg$^45uW49D{{9_<(P zzcfEi;JDe3+wCrtu$N>vc71zP#c}XR42TRqIS_!%3piymjpNb#4-On)U-C^Yo!V?kD+v(Ta&kVovY&j}LNrBkw3X%n_m)yIr&p z=c)IH{&2zEmZ?|0SiL+bUgdJK69OPNNR?ra!tckA1b|7ue>^?^@K3vUSl1Sj<54WC zj4~WCOL~i*k=uVc_V@Di!_#LctW-NQen96uCeZZr>JeOgR)?iC#d5~4WaD@HoBZ87 z2VdB1E{~(t=y7n1@aI8}X5+WEnY!oB4!GmwaXC6n3LtkZKyKa`PO@L0+&*pLN9REv zd_4VXvJW}YGEUGf5C9T&Rn#3674YGabaZ5{*LEpeiyCyYl--bh(q08OFso6-m14QQ z1c{$0?^og|vIuC5LTId}DZRvuAn0oDl3{Ba@g0IIf~W z7_g#Qes+76TF53wk@Icu#XItv!Q1Qg*<*0q7Ob=M6}O|O*C-PSob~#h7xgI7Z*!fx zcBP-TY^p>^D3kNx)iwYIAwd6wmZl^q$_x{zgfb}9Ss1Re$fvcLOmB`ee0>nLkev0i zL(N6P6j>qEz8**i8VQytnXC?;v#4y4cs;o4v^f*Yyv+?sGuH*k9820OeZ{50D^R)t zCvQLixSwQ=WYS&O=@QR(C(<3S6?ZC*7@KfYdz&+H6j zD@b(SE(=F>Ek_p@;TAcKx<(S8YJE8A-SwgYG9-N;pKh1mD6@=yL>|}Qr7#5_fN^mM zRLv6WF*e(JO3nn)D@nzuSEav#>M4Eun1LZHQ;byVSy__`iZZkl?`>q}DGX|vr!eK7-jEX}QBur>OU)Gw9 z?APR0V72IFnO&6rjnOg{5?$ONxG4Tqs5{V`#6|y11~dg*(5rF-?wW|PEV5WZN}Fg? z^C@wVv9WgfLr|oF_5!Q;itn>?aGSCQ?{i}F^)Rp!xFY6l55S4CEVF`Ar6C7|d~TxM zWdU6(k+32cZezm%XHc|x!z5afK(*Wjz3{2ZzFwDsqPKLHOqt6NexgBB{F}Vxdb^Ki zd@p|nW{m4E5p~K#hqWR%#<9*gixvkd6M_4FeNf7r|C*BG}-1eG5 zb2exu=X_^KYa>%%Qj6|S{bL_WEqPNlrlIK1`6Ox~Q?lbGvZ()2OyvL2yY#b6xGW2C zU1`!@7hh{z_W9T@;EjqhLFPTHFI2LVaHIa*hvqoRsB_NcXw(9Hi46katdm;-*np`K4pI2#NFYc$2_5w!z|hqr?{38`i% zgzzgzqAMhru)@xtCl2E2%&%j1np(R@)Mwa(0PB z0}ec96197$8pw$8u@ta<2BM`MuRNQ3bQU>uJ(yZ*dUH5?6c(zsHG5qFkAjkH$B_TH z+6R%FL``vq-^pl!!j>U^zS3IO855_-69%1y4l1xz1P`L45W8y8DngFBZyrF=vTkS> ziK1UB6hVVQ4)=`9{#Sk4wvi~28u7C$0~U*g6W{30kpYrsSW(KmSKdfY zUX~Y_X~{AP_*&3D(9eXLCeX|svsPS!21eO+rey)fSv162B=mo5oPYEt@R%v4eACA! zGGMG*C&%s^HGrH;A|bIG-u8hyLfnits}g6)aq72kd~ddd2Sv`4DYStgH&}u>GU@;7 z@&+wx#WJ^^1{emihe#|;shhH4 zzJV0G`Y-McWFYk6)DH;->>rUcTm3L@9w2ga@fA*2;(cyK?O&|EvxO&pZhhh0V~w7#A0`|Wi6O{h*@m*2kX zKm6b455GMBR=lxC)VyU1$33Sw7hRpr78iT*5QnqR^P%S zJKSG}vztX9{%LEaEXMvWjEKUIR)10j-RIx*7a5<;%`9W_IGwD>yb61)z@=}!>Zz}J<>112yq6^I+~=5!O4axM2iuG}Bx zldq8S-Q8$kz3sw&H5~6h{I}!a6dK&R(g}(7yD;Sz{q6@HCp#4O`H!nVOsC=T!766R zm>yrOzJWt}bM;O$vg1qC*!r8*Z^OV^!I9m7CgyR#^nAF1?t?t!m;DpUw12p4-=R>C zIUByhR{&((U3RN4hkhGPz}LMz;}ibWj(LI*T??-yVdVyGqgL%1XRUL zWdQnx$LHsv$L8f`l-pMC$IbFLEAwLY5>Iuv+D`9>!{RY*EnhcVzgNp%ay~ot5#85h z9}M&k3a&vVNeBO%@eGj6#i26(_R6q%VdWwz`UrF1+2ln+Fm7^_O)bO^O_Y>2+C7Un zs!bFXvHMs9xG;61GOI1Z;<(81$w1v)>OM{&L0 zj=s0NGaM^!FjMq;g_IgaB-%3Gm0iPh+jDt62XQNVi}Q7Hh&Z6}Fg;w)8djV1-v^9R ztf2jX32v`L%rB=v*FGN3Xtk+ICWJZS=y2B|(wa~vm6;Tp(HpKTN1EtjA}`78wLY4* z5)~b*6R+39&Xu+$+!Ck!qw?-BC{YRGuRN*#Ld!Ho=_KFl!rSR<~Fey4;oX-{aZsJRw!_GKs*!u z*iZh*aW+V@GWYv>{iuN&`mp3SiDJar|AJ-f=hTbV8ahy_w-h*UeJuhVlv$$X%(_0# zLY#lYjOYvB9ujgL&1vrfs!5hHu0P87cYu?!h@6I+E!b#ls(D0NQTg0CH%04#tmA39O!_w|LMWz2RmYOYi(G~*JkVluxp9a@S!>V)@aZ}SOU|ms++o?9kbSLd z1NEYW2jHi7U3D_ODO(XL>UD*h-u1W!ap1ZFR|{(rgo1qyXcW0Px<|;xk{qaAFr!T#z*6vZg4gzgp;46*=IBu){F%~{HNDTY-5z2j!HUJQ`PLG zoFZk1ri7!&LNv!f%UFS;gDP(Db`|*`AF@diSIFV0kc6si7zekg>wuEme5yiH!RO<2 zl%fqvlv0!aXn;i&NiB^?+@UBlcD7=ak(?=A!-Ve|0XmbsVX_89W9}+uRU*TFJcrB7 zaxfDWh_l5Ev~kT6u!@W}vN!!s6>2KmO~R!LUqu81q{JOu7WEDO;;*t=^8(M%)J?%V z3}Q!jU1npJIxOyWvj@9p$B@zm_yo-Va=J3OWUhP5q&8(khsvT}R)XF(K`j}9FTg&@ zt=2A7)E_ZzQ#C0Qzs3^GAe3zOOE${rZ2!Na# zDQYT@7ZsnX$yq8aa8pTO+m&3B4~QALp245Le#N2J27EcVm??EpYlG&3>`4h>lbxqd z5+}8j*GT%o;j+Cj8)m8mJ{xL~_?7tL6;a z27R$K2sZhsXaR}uGTwC-Fn~t<$eNs3c;m-OH5B*NAB@ApDs?gi)us6u%lkv6|hz%e(#S$FiH-T2rnN;v?=%9O;#CuVL63Uhw*L!$W zE3J3?L@oxYjGMHfj zGeB96F$B;Jkocz82tcBbnx}CFCMqiH91o*Ye?;J8M4q@ za<>B(QT`kvFocJR%h}*ZF1gvjPAxSkWp2@Kvg9DBO~Ke>RA1cXDr-QvaJ`;GTwyS~ zIT*8ToN7=e_*G*04YWjX=j~juH6>^gXvRz)j0IeD!&DgCusmeiLQ5ds*D)&~$XQ%> zEG9AqNU~T|@*tE3n?u_Rbt>2>)oXqLO496kV* zUCU&N){LJnTdxHGP78L8eGN>FGGMtV2wSi3-1WS%TD{j}9Yf7#2Qf1HMM)IknBfQJLvDz|SIw@P0TUS=0+v|biqJ8seX z4`jup$clc08}5cJy0SO@<9P0O$=_IAWIx+E%s>-2om=!le>o$ezB*mxk7RUSO>!dj zmz&k=>3A3bR=Qbz(}U+1Hhas#HlDQh{^cxeux$T#&-zBk!p=1NbeG$oK*WFv zK`PUEs1ZQRSzsfR)wZ#g`dwwZzDL4O0Z2BizM!Wt3K{@=F zcW!Jhk21~0kGJpi`{^(ptyG1wU%ZIO&6G(5YeA`l*vpPuw>HYw6h}lT4oFbv6gx&Ji8_xi4vcGM*{&>8p4N*UXVc-+BsjySs-XhB!&9zuvf@XCX*ne%#(bmqY^Qo51hf@cE4~2 z5+^45_%&Jq7^IOvlVYArobTzxbxw55>jCRbYKYJ#v?@H>Bd<~XPY#ARNc2+4_fkZO zk;OtOj5)v1cBUw}q$ab#U9K8{C}=(508;}b29gSnK8|h6&Lm{rXVQYQ0huSKb(Qy_ zm5r~UO~v}o{(8ys2D97djLC^~R7`TIj*Xh2 zrJeRi(K@?}}kYf0F;Lme`ao*R4*P~FN? zFz@|?2hY?pKZ*oFaHJY3?nT`(4N)Y53K^j;!JlY1UVVZ(lWds`Pl zRk!Q1&A4dbI^pytXfGwGV(&d^X9Gc3%74gIA3ddnjC+$Xxa_=Y0~(?d2D(;iG%HhH zp5s*QjSr(kvP7#Rw|g$A`&+Sc?TD_JEiVJ_9fR*Rx4 z$gP{}_$^+6lBI#lylT3pqnzD>dbdKEon?`SS<#{4Hf*`B>#c(j=u$941QF}ysbWVb2gx>PhgObyDjGSJNF^mMv_QY0Wol#`(*fH8m>}!QIEO_oP)}#N zFkM!1mLy4v_P$y#$6Cews=ZBqI18EjO-5sZ{m)+)uaB!|HtVAnO=2)Lob&*W0^nD2 zP-l!VK;JBK9z$CJ@~D~XC}bqms1Tooo%udsQRH1)dK)=UT*ESG+Z!&}42S!m4=+og z;cn9fn^I9{RYKT#Zxc+;LgrSJ2w-o0jSXlI*;9xm#dVtNS_0lMXG(%Ox#+CWZc12% zAlxkjX0n0`ad%_Hags|UYY349;z4&EBn~ZI*HACml=21n42r>t&KD&&1T9yEdqNbHy~kJzoJaR8MekK25k zTpUXV*vi4Aj;UPFx{A<$$7vrLebIDN)55tNZ#_?hocrJryLK)jGCVo{z-p9eJ_-hP zLWOK0m@I)n*Rryt{KnQZsDLSlSo?sf*?PEu3`ScpN-H90%4;x#6VrCUsbDN9uLQu5 zyI#pL$vs4y{?0>8T8pcrwH;BOfWD3i^=<0MPYSsM3lP{8oM%=6KLnHRKlTT|$>gB= zq-`p0ViAjy5wch0M0trWesYVJ{3<{pC9t?-piqHm5V282$2I6CTZWYRA(O+nVOgQm z7-lfk$QgVm!B-y`Rmy@HAiVeoIJ*e*W<8gHxG^bK&1DrnH%;9#VBulVXAtkGVj$KP zKts;*Qry6Qlg}UUouVag&{gQH1&XQSl2f;chm&J>GOYBnn>LZU0C*O;CGN4;juk*2 z#b;)YkOi9w5^q#(u5G`us7tQh3z54#Sy z#?uCj1AfTF|H!#9>UO-{dEpLFZ!r&zY#tYn>@#i+^C9O|DP%u2j7c#HD1{D$yfK|U z3lurvL^PS1p`AP+jRw_Wu3Tp^Ws)Ri0L1wB%?VjWXyf`cp z=rinIqd`sxc`G^u#657*a}wEB1?$X2%^DeSu1r^THysC0!3I{8j?pwI$LsY<4rX%4 zV)En+vWyLPR?UJp=w)mx?#Q?wz!m09WH*wvP8kJHvKwB>NL-X~XW&ja19AQkEw|D2 zzXiym=z%=DMi02EX}}O{rv3hMfVKpo0`A6pi01^n#OujcHEvcf4_nl=zZ$plFOIAP zq6n6opx=b+-}bv>e|d`vI%1pE>s}_Z#fZrNxC3FuKOjrB`iK6w`ub}Np!sp!PfmIQ zn1ZKibo>SumGHfsU8^5uS^z8ZYB;~~;FdR*D&o8Z6-;*ycrt>Y?TnKA@b(c6ZlEDa z;0S(uJOE+v`vEYHH@ytjkMg*~PVTh%*-{c&NAUpsZ;N@=kX@c6u zBXr{&Bf||eDI~&O{l^n(-rrjq3CAk(W;`5RS>AmgQ?C9v-CFR0oJJ_6&%6bIOQ0w3 zXNYo0F5MB4kgbMJV<8eC38Fm^hV-5;zGQf|!3tq=z zC35{^(2szH0yYlB^5ZZbWZL~`@9%~sx(jaqc%0x+0=979qm#yypV+TpaY6O~y&N32 z>@=W>=7d@AaZLT2-?&RRt3g8J)$m;ZZPOndBr5!bodZ_Vp_d4G&KAXq1v4-YXjlAULEMrr=`#VW!n*~7$}yB+Q88$W@57!Lgi zXwPO?$!b#|)MhJ5ymx^glLg2S!Apsfox>6!hsBQ90Xw3*UO^L# zoXk+=w=AOxEX8fF)E{M);9@xS4nu&JjgE?gevvs7iBh;Le!V{H%NewP(+BlA1NoCtVdv7qOL=4&Zop~f$67MV6HfJ zG7A%qrk>=^1m#l}IJm+pbfrO#kb(|-Ry&KVOSJBysZbd@#0QsY%V-BJ3!+JbWI<}4#S87B0rd_1B8M#;Lj*(kSel8On?DdXxh*Q>z+D;9K@KEh{(0vC*g8N$?Ns{$@7&qrpcJY z!+E7@P}JOUB8Qy2ifVPDTXyv17HxuL1g?W#mjBC*J8&EIdMzXL$##XhLpLtJpmiD4 z0*dV19O9WCS_uUWW!__?kbK3?14^wj;J>I^I_YMx zTm?|T^`pB0!d-`4GdEyueT}?A9)v+DGbRK%9i}_8v4k^_!?7lJBXpu8lb&j2ydS)r zhrAt^;2%e?)qxgKrZNc>LZIKuxYa#|Qk88k{t(wjLWy5apN^cF1rt!&n5<%c4o#1W zQw(TXavM^E(j=697x`1jK-Zi`>UW`lJ(^;^J zZiH{&rk8lYKZAUa#4bHojtlVU5lV2Gg>i#p|mIBYI+MDr8&%h`~=#6bY_R|5YRggCNuOdJK4RaSa+SPwxFQ z$nUouxeg((UF2n3QmJ&QW0U7KbN;Q@3Ko#d6;~eO@^=9>4yJHh0ETi?X3RJ)yi&K%<5NkOjmBjtEVRS*^&b@{q3yW!m9% zngVF0;PWh079phrdOuq45ppR9%_;5Vhv^mypjrv<3Py58Y$IKf0xB*g>EsqTaHt75 z=DW$;t}9uo(C)L|Ko%=g+755ebGy3b5iBW`ia~hd#z>?}C7&5HCZ|qQvKxO26<0ai z*bCT>kGpB_g>D-7YS$+Bqh5q$2>EftTfZF9FU^vk!&|+GG7<%IcjMhTU`dJuchQ?5 z(IbahOYvB&gaxU~gj`}=d&_aevoL~&TRG$d8+yiZw*vq<&;sOWgJI#$1%5O)27CRr zjV*~d1E_VR+S?XLBWQmyEm`Ph$x*T7Sl{HCzEB1^uY^ix(0aiqFoTv0r z0wEQA*Ooh%292$ZMUNl9?asFKAms!{tc8PaB!Db{dU@_YWZEbLrvlP`81VF6hHJI3I zR<=R%y5>(ft1lYycdLvd%KPEqK~qNJMMDFA-KS$RxF-q5pG8}s5&xU~N#!_0*I~w- zTRtlD%0!l6lAOt5dx=3>@;fc#FaqC&f!uOQ@Vlh|i_Gb=;B<(sJgG%3^NfkXL2K>} zvW)wR2)PCo*n$Z{uW*O51?M%3d1ChqfSgv2)sE>|Vgx;iZ^Tg~WziX@pMgc2)A>8! z76QGQn)i8&JWvDK6f*ALF@>H4R`^LGrIf~%2kexUDLWSR+}%xwlOj2g29oK|GOM18 z1_Xg^PGG#v<%mk0NA-YLm9-ZwNWoh)U6n|#W}37V1DwR%$`-W($=TQ#rm~F$LLrn? zG@Sbh?KoM9EG~CK%IexZ1%ap%f&mGtmqvkIg&c8=xs1X1sBbg{6L$jk;SBN?d`1b3 zwQI?@s39<=)Al)YC1;Q+lrKo}koYfQ#^=8yM*n_OnM?F$AD~ZOv=Rq?QI!oYrN)&yu`n=w5hLlhx(Pm`y*Q1aAeTwRT zn*WO>QitZrxO2DwYcc{}@fWKxEho6+wb-BRB*ZW7`xANx0I2~5I*>{7zwceZ?W9ZO zza58-?PYP?Hdeq}@30@X@BZ@4{r7(50bVQ37p*)B*@0ZHF%tnY^Qu2i@)*zU`yVDe zCU7IuX*fobHx>Z(s((ZlX;^)8+3nsbSIhVM`By*4fPtz7k24SU>**qoJgOSiYuS{B zBe=LmWpj~-1*Jw1r@>>!l2fvyyjZ;gsYjE|%tBkw&eIi_6G1WaQLi#gb(}t z$Ns~AKY-@s!!Oh7%V8%Ti-RpWqLFW*5_JIeOK;IC7XNd4vHEJ5j`yIG8Q%}*&rX1K z?S}(6nD!%(udBjx#XMj@WMuR*xRxxa-H*%k5%Lo}fb9D+>c=aWMFSEr;~i<-yX-GW#nbe*&p&U zML+aU!|uiEMH3n7Y@p$WakXN@dD2_kMp^x0aDr#=T-gLK<&lzdT>CArB@1CRhdp-{iLU%g9 z+YJ$K)jaM_C+YmqpFQ~)?SBiR9jr$83=%75r{4fXhJT6{s3lPX7;aYeI{hs`q>Y`~ z&n&Y|-6a^7Zv^Zs*QtFB<7yRx$Y-Aw{TWlt4nQIr5NUAqWgx2Ld--JI}rK@_g%1uG~@w0<9w zzraZ$Z9zDA*@fKY0vy)lbnE7-CS@YWyk2Vq9Z*PFf?mLjlz_H|*$CI`i(;5eBklq@ zmpDM&<)Z7T7!ZNPOrFUo6CS2lS|9i|)CIBxX#k?sldz;!&rPezL&ZIRZlSjsdx30=T$eDOl zy-5kt6b6ngkTo588a`Du=p>Y5T3P=()&#V%tZ6=mLs^XyDO*ZX+7Dn*29tC4L zq7!mz-YalMb)Iwm;&#-F=!r_83lw6}LU%b&TzAK++T>N!(ggBp$fg7HH+|Uk`l$G> z4pcQ%IgX zL~5gq@(@i~qN9oyTA)8_eQK*`uK=iI7AVU@@DMIOBpkVO6zkVp0Nx@yJY zNGMBt7Hcsmb53*28)V?9>8eEAa*%qfi^^mzRZ3&LX`Q40>7zWfaZYXC6vF> z-zak^0T1@1K9FN?$iKjTVXs&AJs5XuH~K&m(^cT*=BB@QUUx@kV3S}+@KX?lvL;at zh_QKc@_iJeQPlvPa(b%i9OSgBa{0%EHUp~o`V&aiP>w`)94*#WWs?wLl!5fJKH}{0 zwi@X}R0VP)G=CQfSAgofhO()#K$*BfhKU&GrC1@&#SC0mkeCX6j|?{PG=eaY72S|y zxhU#Z=m<45xnw4|;<6plf=V6%EQ#D^XDBg`ndGj^%7eVk~wE# zNxN-01N4BhRFY>pL5eQt-7!`NE5#KOUpax7n{i6Ej86~Y!)9-U9!*N+B59=tB|z}0 zS?nfZ38DfzZ({mMlXPHXJvX*_$fBX36E4Sv02Xkn!ZEOqM;u8h!oyyZV?ffLm(8Wp~VqBuhJMoqo5Vb7X%h3i_ zo@@6hs8s<9$29y%(fgXdgdjOHX1HXpw`eowft!WO09LJQaCCE^H+9yrc2UHmVoTi&NuH4=^4Ym_|Kvd=b(0yEwuP*)Uau3a<#v!}sUIK{ zshF!-3|vQXAmKL6$uMUppWM)1EMLqsoX9h=I$umXQb};bl&7&gM_vM;&PVl34P=w?Vr69*P!jqv93_8|VCJVA1V+z?oxQ6E=1^9~v zD& z{Wj52KpCI}f;rZ9l9_oHhCAS<)72FS`m=2Xj2dSmWk z5n%~;H*a%yAfs2a?F~2N2XMzuF1VUMz@N;Q@E+BxYHEHI=&ssu$dM8$se;}xw}(FN z%H7?lgUT{jLK6*Y)n_DfuY z_Ry9&)lq-Zad{p5Rf3v@nx&x7qIIeIDp^{|M~mUxeSaG0eB?Cd0Laz&)C$6*G@rd7 zB6x)gE*ONh?5>loj@y3aICBb49X2RJMA%fEN4``ME%rc-sH=($ZHnwntuR5rX|=5R zhq|d#uIXeFFzfRDqzLu2t!v==iOr+mE|?dzWKn|Km5tTL@ZIp9Cn1P4D4Wl;FnE?4 z`GQ<=nLXu}ctT)0qeEmki(}{Hm5_wt=BO3x8|0$cUkSCsWWFJ@L95uf93D*4#h%DS z8O|g_XL4L?&LK`(2~{Xa-3=cU2!bB)k^{tr6I(3gcewvqLWq7LiAqsR>R2Fc>AR5i zljjj^XMNG#w8+Y?PU!jr3ehHM748baa~ zJ;{4~km>wnpLM~(7@&nsLjd2hC^>yM_s645#W8?h{KpF@s&V^wm*kJrIe9JL0azwW z_~KV!SGX0}m7d$^`#nczAv{oSy#(7Mp*($XBuy zPsZ#p^fPFyjP=`5&gq@E(P?MCw!9qkJuQEc`Q&EKgbY-;-s}Fu&|e_Ml)M22IdpGM z78hrpZFO7r-{0Zeyqhl2KEnV1%Q#$qeg0aG>itGy(?yfb0Fi9g73xpP&mc@PxUeU= z3)yS#F4$+>TsyfhYPUX_QHuF_b2;O(uO>|Hzlh~+^3&7ZL9X&@kQV@^tU;a+|20n^ z?eG6l&ht_3e)mC^_59c1P!vUxgPyrzq^3oosZ&31?pxmni>_+h*G##{u^*elACUpxsF#&Nzr|XhCv;X(oi<3m!bL=Y3dH4FvxH- z)8(W%6gj8GD~LtZW?16B2u-4gEfkeJeuU(A9ux$Fo^MJQZzdfC08v1$zbhUGNjQ$I zK~Xeigl}YHNAoU?$q}2~$y#K%y+~>3PAPZ*CXdhqE+j=+)#ec$O@|CY&Vz~P=V5e` zPmQ>-oGdpB)mpTmAdG5XW@x1;W7=SLu&N`vz6&%a?;?;CSJ*S-8JC9T)flByKx<@D zQd&@#s#;W4P8rw*SCc`@iPBoyVV)tF%ea^L=0*-{9dA-@@3k5o7?q3D)CrLtM$)Ql z=?&f7Y@F5<()JWhesH68*Rtf7-2O)=W`*)AhY$oFp&%b8WgLzP>Kuwj zqSTBzPCa0xLz^GJKh5=bpoFFBhWL7RRH7OQ8fo_ITn|u}qIsGE0R_kzv`Pg9>+7ZC zOI~QQEF&O$$7Cc?I`hFSEEz2njw$yMfMIIXTbTg(=;W8lOt21{1EzFco70>QYnn|5 zH9uC_QKhJ^1gwO9{ez?Vkv*g1X&(S*AYj4z*Vc_wlR9V0e&g8K@mfwZ<}7oLu(hcq zWTqsJvdp0UCgsMlFw+QmI3WMzkC@pRl| zCq9xACS`IqL{0`*MgWi$Lp-^bmJEF3{R*8p7xS3p6!lq@% z4XY2gn0w5tc+piv0(6mEg2Xuk)WQO22@xVLfUaOhrC5}-Aa=^xgYG!`%3(@Wyh)%U zBRD!8u4E!{VZ&pK#GpuX^2$sotp=$qdx8@_w}YTwU{(bkt1*jMOj1RMJ4&54J%14IRR^h z7S@VhDOScjIgXpAgyY5t`+KMKB)d4HTLKhT0ZxIEA|aatRf`%qXYzV%y3m@4&(?BL zSE9u|D3(-|s5HnpcI`Zn6mA1Ko~OtDqmH8EJkmPqXj{bVuPSDqJr5?KWo=OHDJdkM z2lpUF69Ec_kVvrgl#$R2nKCuVfK0glxGN@{yn|o>l?{D z-I}0G#vAb7X%XQJAk-t~ppKV7yE2e;h5QA#OT~GO01qNBJ!eez3gA*gpO|W`STD1~ z(m=J0fjyUv3QbAyhSG5)PH}fpf`x)sg7M?6g#-!}9|i%q32?HU`hPsrL5_}?f_!kL zFE%Ht!oj1xwF^zDOx7le4G@FAnR17 z#=pidD0@$yaXED1(*$2;ozXeEKsU1rEi+6{eRy#9Y1rck7O`MjWK3SfmpCWEypXb( z-n5w<@}^ERj@f-B$Ql9V&h1nY;=K%n-UY47lgZ|*MJ}5MMc1;cd;K0cXt}$^?mueL zKqn{yuJu3xLo>saA?N8%Rlwu`tHl{5C&xoQvJ5*qZBZX52adu9CZQYS*cCsSA&Tc_ zr~ZCksCL9lD){7jcebX@OdEvaY7A-CLxf&Q`S!*~j*VRFL~b^S`794bv*I&r0V zzV`E-lTMOl%i>-8QNc}?7A6u!IRs8A;dCS``pdDw9Xv^ryy1$QG-R7_48n-=yOeol=8L10$siRNAqfn~ zn!t11vh>-yc_sx;5)!O*vq8gBnwGNIc%`}*o)&->ws|o6Z-@E(zf$Xdt6EK#*aJY3 z?G7MDKr!%drW|4Amu`WMaP%AJUSh>la(W!^4Gv}kb$l{U+xO;Y=4rQ6D8o+*D6t?A zqhlo-$8_1cCo+$yXb1~j>Am8T(NA)akO0jS>PGeFz!;kVcEJEYebdezK3_!;Za(FM) zAy4)-xeSebvHrce?`?=kS)(Z|{^n1(vv2k9JQ~8hM|ZX49sb{Na57Ke%K5X--79jO z5P$@N9C7$xB;QB~VW?Da;CdZ0SvJ7J91f#A!U+mp`r_n_w8BJxxC7kf$FaZvZ~x_g z|2WLTQ5)e*!WF?*Pf8=yDG2@5XUqT*W zUL%wq4JKzZJ|vIhF_Ev5oVJ5~52MbjTw`26W zjD!5|15BV5u#)Zca8kIGeA!8sD4tXw!7NMwo36juW_TD6ScWTD)C>Q6=4*jc$XqQ! zPA@SYc(Py_?L)j9eoeHk#p1Q{+a^tull+TZ@0CoW{bhF^k2|q*_x9L-06CJ}plSV5 zwj_HSUHVrs(<~BMIC3&_R%Szvo4t@@FQVk*(KK>kl^Ho~16JfS3#O@ZCL{?EBV|?A zOwM_yU)7ovRbE7rRoGdq0f?Wu{O07jGGK5&iyX*a6lGg7V8{b6)nej~JvBW+80`MuT z#s6UOU8^?&0F4B?92;0M7a2Ap*k7mxfW4;9IhnEs$;jJmA`qwHqQ8PpY)Wq`ei;y! zg3Gv9C>Jmm&`s#iqq(CZ*GfSmz_37{nw5od7+V7gk0ackcKI6Dt82Qn;26j&qI6AO zGB2uU9uz;kWC}zNcR=f0l~ab#x=LyRbyg-9w^eB5JjB~)T2`~GsYwlhIL!kOvyuC6 z7%AmTgWv&SAdFLfZ6%)OX>L6tH7FzW24SULERZ~me*|mMfNkqbJ%(5*-0vUeF(kro zTNa+3RJjaj9%i>T=chgg#FgkR9bBrV%^OJLP~F3;cC~1QjR;Hy2_Yb73Z|#=b)+@} zHv)Bs+CHZiOM#a23J}tQ*&|+twgycWQL$R8qw<6;6X9!dwgHo8LrvgY=liHpi#i3t zLS~o0ZAu6bU0-D5^u)m}EDtSrTCmE<3s5Q|nzN&D`Op`-|mi4Mdud1rSrvraxFoF5 z2IS2mY|?`huEUDvy?grzNnCddS4CY#ickwM1A47YM6iTM)(G9T^rQh=%50D&RmVfN zG*HKM?A@9^4gD;s%pC%mEbvR<9$2L4&U5yQOVE}Kr*hGS+GJHk{AWRFv~u1tS&9U^ z-Y<~V2e<&OE{~uPuR!)G1~M*nl2W$GjKwTH6tt{mBjaTJ8u4S=*%{|O&7*@2H8od@ zkC(%H9me$HV)4*B;F7#<%5Xm-TV==y`+ zcwzD!F-J_tQ=BSXpNjpuIvbwos@TF$HNHN zOa)3#5aos^*2v z@QQ*ZgseK8<^^whD>@QjdN@6?N(HK6)3N%RNg?TYp7T1fX=ZC6dzRc7Q-H1bQ(E!j zGAh#tbqMhM6pSJ=v5k8s2qLns=<%YXxD#@C18^C1DH<-;EFgfqsw15qCic0&9rVoB^ndF;?5j+>|P}%1N&@6)J^h5E%jhl;#OIYF+fLVB~Ec#YYRvUOpctp z96orWHHdDcoC*N3hwh&l5c>=)Ft1b%%#x3GfVdyYvzk@pd=VKPBwfecnb7lH2$ST~ zhKt@d$m!lfqisRCYz$#5G~)C}V20deQBg zl%ch#OQ56Zj%^yo;a?zl-V$Cqq#y=`TAHP_#XX>xQhy5Z9$TCqFN!pfAVoeT_dfLZ z?r{p#gfdEY*s1AY6Lr?KESEWHxF$Csi<+5hf;Ny;&d;u4@SKPHrCD3bnGqw@L!MJ_ z*4#$gwSmoF!CLaSmD>K(cdwJzECK`6Hg}q~*LOIc;aT!09%5`ycY; z)wtP>)A{k!zd$O+nxxLe)Jm_f_-dH9Pu4PP7t482wH2fDV>9z%A=)A_hJ&dFpB`av zm@`nuG)~y_hsji(XR$dMtVBSPVS-SUy?4_Fm2#8qxV3lT-pz3y_wq}-r{r$hj|Zcc zvUy7W==QefvMYlbQ2rT?PGSgk&i!~mLHdOkAhIP=P`sgVZt`%+X^<~_r@v>JM()Ph z@Q9z1ujWs`^*aFkBoaA&?FCO9%w*UBt)Y@WZ!X*9J8{G2I}0em|9=Ik*F#9=ATQ^% z>yPj&-~W48d6E16_E6|CcR8Q$A(Ets1l(-$GTB{z1rf?}X#+#~ zS=P&Z7?PKVhh0AcSt0M_;Pj8$+7-Vgpy>ft@tiEMe*Vr#Pstp5DS^G*ntZAL@&W2# z;!*ux9wnfkkx0a7;3W4K0~oMxAb3)XqFDX`fEinJ|I4S}#8>o6SgRj5KCC6F@yo@M zm)Yy-o26fKPXd7Z@>#A4NdQ=I2pGpWvV~009Yzmw)GP6bHPyE$JBR%DbV$CL3}KO* zx7O$#8dz^FoB^~X@}a;7@zKAtIUk;0qyFx6W z6Bz?wZ*DBOT87)Y;#_NhSP@>vZuqG537&CS6!pM!R!s%r{9(3`H@9}@m@ zN6XplcxoU)FQqB%iLC}{!MI~3CTV~^i6|qTlu#$D^9ZI|Cf_sVHK{pGrryFcgQRWu|}m$|px$l2>&EpEEnTA!L9l zF1a|?k)~y)O<97DHKj(rff|54o&k^zr0$sEuS=a!P@A$g@6FBqq{9-}^1BQ3OdvO@ z3f5Sah)7>>3Yu`%C|FQZV!$$=OJz>JURDFA(=)l_)0Z!~()4Ktl&?W3~Z4G2G zb2G5@2CRV*T3{7-b>Gw`t24$u%#$PM>>!`P*%_r_K>;>G0^_8XQb_djN^%mhXm6-X zCI$#9)GcC4=1E!MOKQNmXj8Le@}0|wDpl`>(P9%J$#C?;vKywHZj=yG#e^iNV|2`_ zl+hMV8ew4{Y!wO@qr51l z7mwFluA>bS3H)kxDI?#(y@35fiBcA(`o|}OOQWJWi|RLF(YY?<{I8e~O!pjXlgqwD zx3Q{X*63LM406RAFk{Kd(*Z*gswJ?JtZVHbM~ao%t2}JaP)gHx4hvCL-8Rs2zMiGl zT~-qmG3j-#AtGp1Wikf4rkqJ0VSDnRF&xzlv0PC;hzV*$UqCECC5b#5pG#1iG(uP= zp0OP*8_3jC86m7?ROq*6Hp6?jO+o#IJDs;C;!p#b{*)12W!C|2<`u94)#N$oujn~0 z4%|_JIiGQah>p}k&V-osg6}}CkpE(}>L4|aCa*Fhz`+_g0X+)tFkKi~fH;h>zugTg z$rbr`7DPF1`!x6C-@8OhrS4O(HympqX z@Pggk6(3BVb}cJgt&6lW?Ldge7*-vX8B@}f(H2QGqjpq$4Qt-w0mz);j#ZPcGZsaM zur(sJu+?Dz&1l9Mm)IJ#;>`1uyt&b$ds{UXbATRPflb`dMz$C}Q_fECsAwZnvggUG z|5SCBazY-;bLaW8U37#hAd1K{KaHx+DC4L@P*-3;lOn4kWnQ`vo>f#b;|K_WJ;Z4* z0c1-cnn0__P@*%L%XsbsMWikTuj=)N8KzCu&}ikczQxw>n*~m?>yNV_i@^WILck^! ztmCPyBJy^8Lf08(CA+!S!SiLSqJXE*tF3iNmmH*l9px`pWiv5BfX9e5N82djOb50wdP-GxD>7Mc5yspQ%TPd>g8O4v zXs;qIL32V30V^bw_>#;HB1fi!pmcR)mYK=piD5dR5hL;RNkga{06g+Byl6!(v3wH| zSngW;I`X#3N;=F;LQBW1w#jMw*{XnAKz)VVZ5Gfa0NXhuT?06D-ry<`nW^DdH{PU7 zt`)#Snv{k1OuEz#!wqAQSKjDoBgZivv?{Vy1EN8-_63wDXJ{GewwCbuUdO?_#z{Au z^0y{{dNSJFVa!lxZy5UukHd}^;>yw|@*qt)LpvsVe7x+{9nYb7#{>`_8-Z4@md}r{ z0-8hjhTFMq&dVz}<DKyD;aWS+!gYmIcbB3sU*r`j)3X%!TpS3U8QcdrN0KCk($v~r>)oh0`)@~ z$#{8@%}hdH`83zx^q=H!_RhLv79Key-7wTX`NMiD7&!rpmi@aW7c3@nQjs_Nemu)*<_KMY zAYsK>t-Y%abgEgvaHrf>$j>*C8}aCo8nc5o$N(CZ)`;`DcSan7N!G&g(Xq=^X+&<( z6s*+%SsrqK{8y3VphlwS`U3s@-r;5Bje+JXqR#BtX{yZ7u~2yR(Ho>7izat->B~zI zhi5FMp4U1o0Zl_5M*pE7cfDgggHfI_fomYYgj6>R8h?~Y?BcVul+&-~V`rnGCUl@s zJR5}9&MSLvRKON>I!*?FuVbt zmUDUTmJn~E6M*Kd1A&3xHEYFV3=ID5;H-Z*{EP(BZ8Ci{2!knLb~x=PgUjp>O3#2} z`ZfWQ;mLuF=HWppKdb|6aVLMlSl@MW*F~)PXvd+zg-?4`xK3&ENC_xv;pitV`3|qz8A6;hvCz$MFHo zD)t|N9g&0GvXb079v&PB*Z2*~Yw{xb{s5Yl_STCWHOrlvYdy!c;(vV$c$q;rIlPGj zhL}e_4W7GZGo9Cu^YKxBBsmT92hjNRCri@f5Md6N}nJ4*I|5<)CTr~i% z{AD^lEQk<*$HI#rro;2|+~1A*b@!m>T3s28O6FngpKYP^BQBV`nFY+Hx3xyecz&2b z5GX+uw&KtEQMP4yP2cqohkl&pw_fggd5~`|TX}T(zyBE>w%hme&$~~*9VYvu#^gsa zFdx}$W~p*gu5#gt9nj{y4NLkBNp08sY9a{uU5HJn9|of-z)AcHKrenf!%h%Qzi@#sk#NL$!8&@=nh8!<`b@49Pd2{_Eb|l)RI- z)8TyV=L5cEve42%IBSPZ!T!}L`DVEMn!N2Frn3P*EeI?lLzQ?LDrj0+1+@{NQ#0jG z3h(mLp)!w$;JXPff-h5_gsw5hOMs7pD4PAe~g?ZKQi% z3ws!@(g;>e7V|U9KU23YxrAn4vf~-*JM7xt+o#J6_hY0eFsuAhiQEenSS^V-XkjW2 z?{023`ff7h6$m4Ab7MB$XBK26jwDNc20nx}Yk(+-c-J~C4rxcZw@rVra65E@rLo&^ zxfrwt7dcHdb84$WiD2b=rc4gzQShx1L&uEHtep z`D4e4QiI`#`y^a_udNG+I$Q@y-ZQgC6@*<0OcCHu%3NM!WEB1hm>d}x@75IdBILZxbB*in?Ki}p1P|dy|ND~)&*{0E~17R@VYIbn@%V+4Hnb} z@*?~oKDAXhGFNF!*}I!)AXrZM5%2R_BZ$bb*!-?{ay{VEj>KLr{acriZol4eTyTc0 zE-Sp4quPvD9g0(gfY35Ug=xrbT%lW`b~VFAOi*Hnn=T?MeKdZt8UPN$QUi;ts%8jR z8HDOt{sC=CsWSCJC4nluN;;ifkZ;5gD5-+%_Q8#(85e{W{Ei$g%nP={@1PRR+rkQ) zlec-=Mb;KPC2JuLLdV<`3$O<4UlsQy;vKmuQ+D#7jOrr2&Dxc1R>)CFo{;Z%mLmXN zmej9&v6Z#5Mp=X#z7E=`ENj1PI%aj6ruY5^WohR%Vdhumh!Xj@ib&wqSv9_@+GQCt z`-SE5vTP!ySNFzFi%m&cm!P0n%Itxv{FMbFCLM?JHho$R;jIZ^7j*h=mf?ysrQ#hQ zbbiUWEC4QvJsimbPgKDB|hR;@>Iw{a`ntc z-~lWGK3l=`*8w00XxNmYPv_Y)+Z0_^)w~O?$Qz0lv~$kK%wwh8S^BsloWO-fS*P&{q&O)bw|N+h>M4M+KOH!j<_+ZDgPp z5*1X0ga|MMaA+%DkRg@}`=Ge@)Ozr!%_v!gV~tKmStB!EyJvi;ARR3xUW#bDBxi$Xn)uMIic@015*=+MuAomJgLa`e^gyxP!q{dP*}1@ z-?Uf2!9*g#CUG0K?jjHhqz^Ywo?Qg2W>ouL5T$ZrgR6t~+u?lPxrLX?`BJbWzVGML zFz#H*xCYph8yG&is6DOgvgP#E3AIf%fE(P<9sxheac~r%KjyExs$Ew#@C3|r3MN$C zt}EA91rrJ0In02ZA1xmsbFVyuX<6r0%S3@@+tl)QSpj`alL&cFFW&+&HHxl>$%7+G zIsK~`qSKA%^2WKn$$z`_qLKimz7}#GN zd?GUrRWw+0;D!uh*)-3R{ig@Vhw0i+A& zHZ57jc-YEz3+_ZnXxM*ooCS9mQ=*1bCr_}QB$&HJ(ex<(zb(Aur|!o4d%3pUuRiKy zzZoq_g0Bq95^*)D8EHT9HWKU6mbwFtF>lr`a%ijzi2UPTeYVRFN>h^UQ zN{jxSfYRhqUgL}8m6vQVuW*qx`NOZr>16I5x0kiDhpm_<932SCr?3b=hWYJ#dB)2e z{<^KVZcO#{0kSiCcTdTaxb<=evaNcqg9UjYFY|(*JQ-OIJg>i1cdZx657Woxg~_|$ zjqhDa^Hz)A_Tljv#2tPQ{gipSJjiRX>KS+alU9nq?N5*L+iP}_HG{Z$EG0;R8D5(}LC0xsxI2>PJq>H6p0FnW&8REqH56obD+w6OgkwG; z{z)k?U~F(6vW(6HO0kfkaX%;PaeO$K`l~}1{0Dgl$s74<^ZkGyH-t1l_Lp7qT@S1~ z)R6x32f0UE0)8X2%1E_*H-13LQOoPE4P`M_%$t5YPvfsDGm_}nOa6Ws-rqv{#T4uR z)E_RmFZ}7BF3UG~4b4J%BRj|ey)ocUB@nU?DnBg`DRa(XTAT|B7{kel;TR^7>6jSu z6wB-8%JH9@6mac(quVqO(+51Tf(cHR5(j_#7<@#>JLCmarjsG2Ihj31KFi-FZ+`b@ zLjV+hFbjJLZ&6;MBQM(b?@e_7Kb{vIqDv8?LQ2LEd42YI5V9&^*B~L#g)5>zBh#y|Z_#;%Z zBTvCmLjLPjg+FVd=fF(|WznPDvO19wfu$xd;;X&c(Tdr-Vct&Lhg=&FHhb80{{q!!_oP9FvwzwIW10r zqN0qz3qHygWxOeLmu4KfT`7ySgpc&%qY`G+v3i_qjO@g%Pno&FCuF(|(zTq9Q;)Dq zp*c)dhwF+$E0@Vx_hZ4|A{kOfZW9NAHE8O#=_vgO_d~}E@s(++x}sj&WUSMm6(q2d z<{YaH;^)}$3K9)mTJ}k}wvu3i(jSBOldMrGQ3>U8R++69+31FTDd;Rfi93c=a2Lqx zNWZdbt~S76q)g$uZ0c%O)+y_YTqQ1=Ha8It#c6K5iMDULc6eFxX`3^@xjl>31RM>N zhH-iIrZ)Yq3_o|nWmoXN>uDL02DFQd>=3=2p<3OrG&MX{@s(q8^ z5^+bEBsG9+pb5-CV~^JW;D!lTe2->>osB;t0Nf5PP|Ct3@li_n)E%H=%sK6tGGuMA z^oMss)Fzj;NkwE&d0y958zX_(K05i52GUb8kHAx6RB6NUtJhbq(ZWtI3ulz$%K$=& zW}|G9iZ{qRXiX4+$%a(c1Y#0Yt9maEtqueJLc5a*VV5nUhR`s_^0G^w6`BC0UP6H^ zNhR=Ylr$bNNOTZ8;S{P3D%1o}7Cuh)uBP9hZ(f!m>mX1JD@?7GkTfrK0@HjB*A zIrkHpOxeg5v_QJ56yQKgL3#8<3B*>#`oIf+bO~jliXc5VH%{Y0=4@56FhVG?2O~7~ zG7(Hjf&n(YsB-fa-#?&m++k%VpsRz|vh7FQZ%=LmHsG3jQ+75Ov?jn)u$akGdnpSL zbCWvb#oyVASx7zum(*Y(ci1(&A=uDSO(fo;y9{bSMT@F1RnQ+KV%PE9s1sgq zjR80U6@`LZr)@UIS~Q(((O;7E7FQw9Sp)}tIgZz&?z)Oxl2w_#s>MGdoh_HsmlGyS z`IuNR9I}!s1}v|D7i9^~MEW8-PicXCE(_SqdV>bILE{0S1@a>)2V8(i>v)kxWGWn4 zoos7zK+v4CJmqSEErUwZ0L$8OIqWOy06!`{Im!rCOWFGsO3Cm~@cH5LMQS7Ox+ecB zAQ+MJSUx!O$X^`rBg;V&7BSFYFaj1|me4^%Y0PanyQl^hk*5EEo+57}3Kc#P>WpOC zQrUE+paXPiM@NBk?L9aHjBEsh3o#yIQ+BMou@^68zeu*oiNM^D0XhsOHJup0_}`Ef z5Gf~nDuxclNc#NaLQx>0RF}~lgIxCD1YbaEL~!Y0GyCuYilapY0%CGsYJt`EC3-V) z)?yTw9%a&kwvZN^CyZ#oI6>caDP-mn5$8EQ#&+)S-4#;V-pQa>O)VKlOLK(yX>^1e z62h^%UPua3$2#h5rdSQ+HRa^1g(xgBfi_B%ea921)Y#scKW82tJ%&v02$5SL&r>xq zeD-GW7m(Xz?GUZ}SQ9yq^NNC|fFlY`qO$9Jmc%J9L21kJ&M(8$!!S2`x?Mau;1jnM0p%SEIa^xQ@>YcrmvHXV}ZQ8nE!I=<|4VJd1WIhq>n`!_;5?ZOI-s%h@pmI$KJH!!)vY6TH z*W++FjY|-+`ZjKD&za6)Xs#46pP0E_Ys(Miw$wJf$iCs&H-yXbaPZ zIY~T6wLD1CWxUx3*j*!Jl+yHEp#akksbMC4U)Z{tD zY|Be3K}A|r6wV&K5E1BmnOzabp*KjG_~FsjIZ~j4I4$ro0P9o|yRb6n#;nuJrjI#l04b2U!e_>4u@$OP)jSX?k zX;H$(ix#t#(5T3u!$^}D`K+lk#8E*E z<7}jaAgYLmby`p_4K~&A67=|s+hjAGKLWfHlTz_YDnAY489D}uh0+NB_h$p}Kz>y2 z=ZA6o>9^w}a4{R_`_vA=f?qoyRO~<{Ft+`KZxD?*NizN68iM{P9nYP=@@d;2q zaB0Zgo2T9K_5c$4=`y2iHi8podlpCcY;=g+02>;*ISU#+x`MyHu&3nI|T|Ec$S@n;ahV6zYuBlhs(pKe}N{{`C&NUCWpx=g2|tQ zt>bPuj)qvz+?UOHB!=fwuJLZ#4D&hpW*Sct6kM+V7D|ivmtBG@T$EVEf?h3F%Xmk6 zvE(ZYcCz!}&5&%M&SQj^cE5Z4^k2^aMhywXrW`vDB*~IdKHj)>OLB%&C%c8%ofI%= z9`|cVG|gMN#;fTAo_>9!M$^zgRj{oX=+(&s#{SfAL5TPKkI)fM-e6OZIcy%r{*&$# z&tJ_(!c6AK1fDENblL6Bp%xDTR|p2p48NvwF)QyN%l<68!k!zG*SO+%%TWIWdnp~QBQ)7Bsr=akxE9Y-BhkII}=Lk<>@nQjxt$=uv1>;iDFvSa-B&CLg$4w>mNj=+i3 zYmxA=AgYfQo-E56F5@V|1#qx5V^qj8-JYrTQKTV4xgansE*^bqhZ#e&FB8yeS88Tr zh%^GLvw(bzx2(~MimUtcw&WcAN>~X!6j3%r#r>tI(NKUM)MJhA5xJHgS|Z<1eBjSb zL`8+Q&>*7}zV}kpD9@R4!^7*upl*vc!Wtqw?%b;dTDCsV5Cr+849wJ#bd#21yAt8? z^^90ZJ3Fx3Mrb`gIQEx99+Z=60Dm~`r7EJ!E}zuNC!ukhf&u5YIg>JFq9gxB3eXTm zw7jrct3}>ql$nOVr^66hwrF82E`k`LzfopOurkQ z0|%G%>m;0H{EJz)ZXC+Hl;5{*qcK5tB1WtcI9+N731jNY)l=A+lsRSV!b@4RwvGhA z8>dQz=jEE)(&^UL5FMb*&BN^4w$W8taKEB)h^>L_7EQ?&{J|xqWh<6LkCG+}BB-r$ z&TPuuKDyB~){yB^4Q-XldGSSjfZ5KtvwY`hgP>|3NyF*w7f@iMtvG-JwM7ZFfq33Y zZ?eb%4vpxSNuXyf@+#$o(|IUE9fJISfs|#hY(6vaiLAtMsx}xJZ{Hp`37|NaF*wO zbnHq<3)3FCVEh`)Q_8A^%nYJkXMh_Jm2wP=z4 zq1;}Og9lq%1kk>!i8Xl!Q9wHPLY20x5I2OS684Vn38Z9%4f5#Kv^FiQv~lDLcVEdhr;1GR62_F`{CUp0aHB{PC~9jLl)>YOvBOP*2;a>kb4O?Fz= zm30NLZy`_@D}e)GxN`^;sZkf^R$pJOs&n*$(t9jbsts}%?9C4Iy-VcP$XiFsX%{!z z)=UPz!XpBF9>H`!*ho%Iayh?wTAmjK9^`Oy))5HxQmPJV%?J~K`WRIwYllOi(IR&$ zkVEj|3Jd+>M9EpePD+>Ra3)R;PpS8dgjx%=0zMA&QpaW0FwA#_1g`3=V}vZ2p3hE1 z4MI$e=647vIgT@C2netxmBgwg6GN=DtRS%z;R=^?*NqpEigs_pel0zd;u0a*1UtZB=HsABw@$V(W5vTGO){LAoh*f}61>V;WE%gycV zB`R^e((lI878&YAXsu$F$a_PK?|w+OcLo-Fv{ebTV}uILf$xlHG@lPEcr@D6xURT( zj*i)?Dr?!_$(_ACI6<0H<{8%pZOY3Ul;{9pe>cD@mIdH^L}Fs1ZM{e#ehYQN`zDGK zWpCF0aA&*Ie?wu9SBzH>DRWcrW#+&@0S-#XOfI-C=$BcWMN~Zl*e>d=kzV)nM6Q}*~)7;|HjbBH~878~*G zW${rd&~%+z1E4*koIRFfU+z@=cW!K2u{zcG3EK4cu69>Ioq^uQk_5Fz&>N3r_f(z= zR2V7aD?9~iQON|N+}d%R4ub|Z;L z(4t7wHjM;ICyz(1o05CPJm$f$1T*hZ2z(cJuFJUai|T~*wpm%rvqPk!4UHzvFE$&mCMnzjqKeuFuvVk(9#QFaI8*4P zFcp-CCPEU10Syak2SUjb*hEUKm5$>%7%Sxw<2m?Dm<7fjIUfH8%r`f3o!&c)WmP~a zBS!t|cHta5Jp)NY7-mtIynyzmf}#c)_O=M@+pdsYUPfC*6y$V%%5DXUAjVw#qMMol zM1MVt|MAlAyx>rW6F7Dxq4H!+TG_K0L5Ah3(44j|GMe z0TWc+B6Q%)gB#5|ro(%1idpz$$_rT$5GR^d7Y)mmIx*0!t|VlSDCs$NS1_El1q-^* zo?WU%xr5TwEAcV3Ni~${b;UomDcigy8xLsBWcJqFFW_rs&ZudaV_hN~cOwM=np~Ao zDR4QXWI|BzV&fZYA$L)6k{3)ZF6$2?FblHM?#UwoiyB)Bc{^6C%*l{t5@vAg&*Yi3 zWUc2hM+V{r`qJds&Li1#o00M~RyY+&A}nOFl4U|KHFPqK_D#!84*PKd$`uHJDZ&n# z7z`w}qD~6Wpeo~e*+#hCowTJGfW_?k>ptvUmNNrGRiwRo0kCCh0fw^3j^LTFP$-Ca z!&{fC1;HWkK^*BTIt<=cO6SqZKn6XyO9F_tU^e6haSw}<6ZwxQzFGtE9A&E08M?ya zwP?|yQlNoNr~w>ZhT<*k&meVM5Amg{&S&H_b#E6hE#II$0N_pTx_7DMzZ({7cu zr9E6g0gyAL*P44A>|o*q(#paMR|AMlGgGP@al>YI=L;xEo0JQ1XU&YI5?6upmzpki zCj~MVat<~H*Uxr4BocdCN-8sWTz5;D=a=57+19Y(ECFfOO@YhIF zSxg1s2%xwUANVvRr^`mpcT1<`$chdQSaTRYo{s&|a0Q72+5eLf95O6km(6Ze=A8gR z^Txor?f`fil5fX{exBrij+;-v&8l`K|Ng^t5$`7X^9JBdIcqMn`RCU?AQlOVnEj{! z`|#;E`PV*q4I&{kZh=F?A^B=NSR$h5f4&TduWyZN&8L6fK29eGs(ag?w~xtJKMb<1 z@1CBYF7tB(8x!MqpMEe@cG-^xN*!Z!)uh;2ADPipD8M?SA*^pa1bP9*5*B zDBw&FhvE4vxfl|Nm-+m)mCO0vpYQL-?Pwr5r{t&Ous!$N9rTI3-qQKwWFcHfaJ-BU zw_oT;9jCjzO-~MQjLZjik+D^OOB=61q zO~2X9!v{kUhWRrB=ZEP6HL#(-6ZdsC?|dO`v)3_-P*z8*Eh_gele|=NI@y>M)@1Gl zoG1tR1eO)bJk!yte>=q|mB@NPR?Dc9+eeqHZTizn4(#Q@l1wcR`Lw*bUr{e-Af>)(Zru9wx|HJoQzvZJgkJkk2Mlkdas`vJdWA4cyaxCamZ z_0!PFjXCNxGx{hC@?-(R;j(eMV7&#`1<&mH&5)D^VIMTohFK@hALV$u>!0kr@{|LA z8xAc;>k^X`1WHp)UUn>xZTR8LcrAVi*A`6WK?aMi;^;ack^-?YSAhHbXO;^rYgkDd zkRg?*lIV$Ps<7nja4H8b9mn^h<(JBlB&1k$kW{gJ6^zmA4Se99R~*hSE$Yo zR@HSGYeR=?(wemrb#cB>-6u84Bc0f8-?t`JQF2S;(U;}BtU>0JXZO|-X2XJ78rKvf zU7E_`j3|I@9n7UFY9_moDY$bKT_wKU=TqfvSv8#a->7<3A#b6e!q04dF>8X}Vd7bb zzhs9&%tN#9QFk~s$+D6une5hREU*i3U?NfQ))9EM%zN#tTMd}nBJa?_)EZ z9|JRg%gx;S7}lf#A0Aad9~`Y$fd<1!9IpUHK)Sy$mb!${RWuXeguo=cX013kZPi4k zWDKt(hqg5+@`_vgH?Ehw$!m#TsHz`?64opQ7ZMd5!`i*f>n1BHcw6z_IO!}TYPh*h zGY`=A;$U;BFVv{-vyA)X1|_P!zR{p7xUSl?WPLKY5O%zTMtp?W#IZDrNdY~`>)&t> zhjeX0@Sn-CQWY8U&;L3*P;JX|*ITPcyUBRQXZXGKZLLKr+W?odfP{3qS0Vtz zv}(JmF6p!l*)5>(RWaD<;i98W9jWc#+}QjC&?pQxRhQ3MGNT zDy`%?C5u!joTF}=ln)sXK$4fxv0UEG%`bMes&R(V2S*g@UStJv3MPiMgC%GunH|u$ zhicIp`VD@K?h2SHRJlGl*|4Ha880h;jqp`XGB6-8Hd%2bxqJiyXRl zW=9^rXkzS4cZ((&>scoroehiJuBpDN5;9=~v+xN5wJHie-mT;43{s|RahzuIO`DE2jXu~> zVDZ?KTNg;sRWBz2pEAAEC~LEpJFrX$xZ^j|~AXR<3YTkVRLD`$^SV^U@sxLVSUaKCn%?ScA5Mhe#d4 zVQ+FFn_bE{sZ3e7Kx$3Z|5a>n?My2*v;9s^`LZYs!w2mvq39V}lS@kdua(>Ktg0ER zH(hiTpdc1&)m)piX)<2o3-BB_%BRoX`@n-m3G=izV|9nBLowju@kOo!!lbNx#*Le`|I5VDb)o(`5M#E|D_yiIEXzH^=( z=jo%THj>vwC)H--{#;HmUIHQf8~kC=^X6?(DT%}*DyZDFDiea{x@GQ~05Y7H=pEyA zv`SH_B~IQeJCi+hizv7X^(z-EOS17FY)g zcr?p1IIXa@=5Xe&A00JJ+sfOBH3@mg$@*Q2A5a>j5>^J-uI&;4#Ei- zIWs~yNdW?04qXD2XI4P_k4`5GMwHFo&?sHcAIL)uPFF8&AP-3~cK~67i5!y@+#_?X z2B3=Lo+7zPMdhbZsq6@`upQ5%XR4_=;c25A9k0rY5&1#i60mp?8R$n@kdEl3Ey1hH zjWq%6QJz(Tdy2C*!|q|!olwFcv%S+qK6s>nM9oYi?0XzQtHbSbD>(}oRdNKe4#C5) zba@*gv9gQM`%bbfMhAYVWim79AQUu6LDAfBM%ixyYE_f~^Ha`WB7MW#pjIhF&*^YJ zOz332bD#o=##lQ!Y;%JsU0LQlDK6)e)1Zg9T1A=|Eig<^dvHRfe44Kmv~u*mFk77* zUXx>m%CVYK+VX=^r?r#ACoMbfTi4N6oK13U1IneW&a0do_ZOH_3ZXr?i`VZ(F%lrC zl;ucp>Bq^PKZCa=E3KH&6lamSP44Aj zwe&Od*04m(qf9qLmXrKwE(oTLK6yPIw%|gvN=^ph0PI6@zk53N;J*2wWH0t7l7*;7 z%fLVOXYq#3^WF`rpPW z@00$LYeb=fuN@Y_m%VrHLLtavuMA9k1xNIUPygox)XaFcvV(Vty#ZN& z?+jhzI9!&0f+p953;p>BT+cFKh)%?*K;>EgF~BOUAK}|+J`4-t&s+I`AlU(MV%(n1 zx);x5v{ipkMIRnAKY?xuNvihdBivF2`#9`;5`;$KX*=nC20_! zY*w8>@p|lBGiayb`FF!X&X`aCBG2#NxwfVm@QGvN*#NwCwtyl-Xz1SZe!P30wP_mO zMsI}^H`DZ9$%qb@({>(@XZad7i}qK^iS-s1P=HvD($Jvj-r3rA2S1GY$``${CXeJf z9?UBkv;bb$c|;c5_P}T8t3)Z5f5EmKia`-W3kQ`sn1~}o*RVx78I>cNQ(&R$5gCm(}+aJNZYo4QF^ z2K(46T*D|O(2T(GX2(l$3z?!_OL+#a0V3M#oY$-n34|IoIi>*6U-a&PkY;6cJ=v1cQ|K3yh6giXeTR-KQs>nvcu@Hq_pmm;R9t(S<_owmESvgk2U9*gJgJ@gO!=Zk#UYWgNoOx$0Zr`N}I;` zg9Zf~bZH7H7tVXyc(3j1ocUz&mK<9Rx>_Ub8#s!sMcYaUMa6>vlwtd+sCmn^^|hGq z5$b!-!Xm%V#oSpbuRx!YRR!IE??q~5F2*>+#Kkw`4G7{XQk55IKXfCVFlHNW1CdY~DL82wWA;t%nUNHAJ+S z#vXx@9WR@XnH)S4sYX48lk-omv|P(_icq~Q?x(S;4BtL^Ugl+atVUKv@KVb5T*pn{y9ox(htw&i}wtv;cQ7A*4<=C~@P?YanlV4mC$lpV9~ zpyl(?jhC#dbT)CQ8h{R-=h_Dyt{c!H5cNsZWjdZW9Rp%bh;0ojbnQi09rXUoh~VZb zp^U~@RwD4xmfE15C$hJOWE*l)G|cDcLKtf!6FQA-vX2Vp5QnH{Wt>GxP0re!IrDbP z2~ef-xtVn_lya+TRtMY~q3V6_Y|57ABElg@l9A<{M_Yr;>5RWXVsaw8DCYPULvJ^jF0Od%NI+TAy8)~FhjO?Wvqv1jg*dur0;Df3pYusu2Ot*nX@c&LsFqAn_y zBo6rH6!ei8C9#y|ppF~yoEnRx9N)@oU!RT6AL|KH9RzsmU^VL`Yqfth2KS^|X+f<}8C}mjWcDWgPEg zzjG`bXiDz5u)6bbY&awpufM@*JT-<{h238s#X$!5;pP|M{&`B zsfHc`SQa}DZWczJ%lUP3N~ z!?NkFQ50yh=itwM@}kHKPK~V@A)ubfIy)ids)0_|lmaz{P&GgapMgsbOOhMbS_*U* zq5I5GOS78Lie@Xzfq^Ex!SHLZySo@PWz8Hv=jk##awzDbWf93!iPgNAlq^<8->g*8 zQjkFNzOdA8jG5f7K}dXVKXYUkY2CFEA)|mjOs0HOviRAGsx(ys;_H>6J&y*)k^$C7 zIY5tcF@X*H`T_hzyu*x`rpfbsFtSgqxv}!YX`_#+j3&d(wXA&Eb(6dBR7C?` zlgN@Efr?sw3_BDzG(EuSTNIQa5?eFRV!(=&ijOOgMp=0|-!@*U2^^i=ChG}MWNS#d zGr?q@)gVP@N+jxVB2Hb_waluu9JFc%dESWBW(Z?OmxIjl&o&Z(%0m2UCiI;Kcg~6l z*!C&#cByoSW64rv2q;cjLz}?BRj@3$mVy;9lY-(jjmX>06BzjYFQy3PIlH~E=82~X z@mBJ0&D24u=j6;}Un*F4?cBfj5S*?|BRrr>A4L74K?d}1R-5|oqI$e8n!oM;sh<@N z5tYh^|Und`XtIM?;&hAFKGBF4j`;Gw&`DOwP z9}n;&ensNb^{vWDKPQI2DX}5)f}WTMgT*-+b+=v8Z~MDpzruG|T7>PR>|m#W(85fB zX$GOW|L*S1J87|t-S)=wQ~3Jz0^c8!{_f7mdO942zn<%hG6*ESMPMS$%MAnNri%0`O&)P3Mh7 z3dY3UoT1bC4k{y9crRYR@V7Nz9ySJ{CVy%#`qR0;bE~FTLZNs47HBisHFaiLbvP?x z<#|?qx!|8Ia?@k{pvrcHq#UMdo8G2r`urq^wf;_kowK`<>Um*>waK^NfBQ1|do;f! zuP;Xf8*)@3=En~5E*`!5m4_Bi`~Ag&wd|9BGaSrZHyk#XlPY4lH+8#u3FgJjfNvT8 zjB$|PmYr#56oCk z<%>}^p70k8R>}W@y;EFX=7k}2(%vr*#T$Q3eo$5ww+c^><559T=E829S){qHFLE^n zmS6pAutGOC*u9?_G*DImQ_QJrMFX{6$K2m*Fp&6QDc2GL=mMP3oMN;9Dg*RLM{Rl` zJV5;Nh#2}w_d{b+N^DMs^!0OlXI(FkO1r& zbf`pB)Ldhb442HAveRQ;O{%V=xO&OoOVras^qCV*J1067VN12D=uX$lum)3!!1u6s zd6CwfMP71{9bP46&YWjvOk`a&-1~UvJXCAbm>xN<-f9`I1veSEX(Mxnc?&^EIqN8zba)l?P0xy5P09#=ta9$35JPUE@-UjexnhoDYXgPS5}G0cGDq?SFh>8S4J z{6@6_kd-_g_{?^^?jnWl6_}vLHppV=xP;zn1#tq(V5;U5Rj8ymjlzUjz12QIJ88vy}Y%&J_Y@DjSyv&l>$2?F*oKHy4#+JK>+O&o~18-*`=LSRW zOyodf20~M)JRJndHCbeWOpr^tN zA`X~QW)jq~uBB(6D$~5}QpR2`^}(X+z?Bohmn&#U4nbzsR)WYh8=a32UIP#sNEPpXcsX7u3Fvd; z6bOiBvb5y-y|`lvfBD+&xLo;0GZbg`2>udAF@ zctf?8971c{jk2UXtyKjYMLI83TiuL{x5s(AfD%Nain3tE%0)65?3gY0VjxCx<2eJ~ zRs;ii(SlTovqcA|CDgPPd(9xd@o?uHTS^{jL~zd`cNq~b_8liOKO};pDdc<_H9}S= zqJG>tfS?=_8LJns&RDil1|=eLdG zq(el~5+7^=C3EpqSZbmuJS(n~C_*f5Eia-L)>fSDX|QIx4`pEV|R%OLGFtg6cMoWqulR|AKzP~A}1 z$U?vbZ8mOQMYTi6#pR-CcOFzMC~ss(j+}8$Tu}@_1;Z^Z`=gue(5Yod!=x_M0MJId z#Qw6|xrv-Xi=Op!TuuQZB{f>Lt{-x`xC>M!kR^ZYX@B&xNYHR4&XT2=Wvw9d$fA`i zK4dxM$}D9j${iFa)Ix4IODR#Zh>9X<*4E@~uLA_f83byWNWX)p6f(j!5vpxH-Niy) zUqm-3U=#vhBJ--Mew8^i&FlLKs(Srs|kP4V4YAZgO@T>aR4VdfOPnk*3bF}Mcc zxa|4K{*M~GwLxEBP0egTMV(zwT?#Og9NZjiTu$DiK_?Q08?qrC7I>Aop>nKI-42X~ zZg)hjtmHiNW}(YLKNEUFt{;0m+^J}%L~dDLGaGpTBbO*KbMO>edB$Mq0R5n-ZPk6b1woi(nyw&M*NrLxijtHC2S~8s(Z@mFXn*<~Y{g@3ENyt8>2 zlK=DvxvZaf$(x~n0M(9t5Qlv+GD7D>JuR6apwlsLdKeDi)OuQ^Xs&+f@7!Ic<}1|h zBj!%7Mqi(~56B(i3)R7kHt!UwJ`bKL$zYePXD{D+`?TE+iOrt7n=mPd+vI1U<3c8_ z-A9Pn=s6H#-hQ&$Q9w6X2oqSnun^ptam5by2W9S(8~xdAo`!YIsrIa3RqjXhlg-oR zLC&+Y-qi2IZR5|!;c%Ng0v@!cC^NStH?dV-F9ZIXc1HgQtgR>AL*z~A?YuS;UV@1wH@_<~i)}TJhjH+i#oem@@;>;23V>X}YnM%<# z+#tK%DJ>A(SFWNgt}>`lTJc`##Pwj0HY+*fp%+ksjm}^lSaCfl6oP)SZ9uNQio+0loX0D(PIMy6O~#|I}LA}Qhe*0;%8qoXbQA{ zMC1VMDWo=#bYUu$ldGbdesFTV!whOI+c~q=+e6wJNKitb;pXPaJ_m$1xWyNTnGR>- z-Sb=QET42;%Xt#vM|LiTHNqVhIeEZZQK-wRsdzGO#`jLnrEc4j!bo3r&+Cja9;?7v zVzQ1qao|?iVTr0KrWmw3RRM5E(F6`~JB(^TMgT>x*0lcN+x=#p#5!&XHQA z3~7)pQLmqsz{$n`-mz7rUDiRdXG$p{uL9kOFBVs1e5|(A9NCi}t#N z(}K>Crp#8?Zr-$Q%K$XldpaGj+LToc5l!0h8f+tkhBjxYqY`N4_@MkJ0!lk>h#Di< zw4;^@WmU=fL$Caa;20J%6FyEJvzd(@DB_7eI0HEu8Y<4Lda6r^-H`_|4<}zLkj(<5rt~X1T7W!+ zQ?|ZXQNV|wGSpg3K+dq7mEMBdPYTr&!o;7Bz0d9!0J4y;1NuxgF#n;V*($Hup*X}s zW7GaPoW}(?2U7T&>#{fp)S#(A=)(tI|6!V4(Ni3gI?Wi^jmJ;}*~7T4R30hSaoLC) z4k{PbinJ6DCX3*ZSN-V#eq+%6QrjfbsVTD=&7h+e9=<>0;f>B{8ow}TF zS9h!d8fsFZ=@tbY){`1_&WMG0R{RlczDi9hQ|0HF*y$8z5;lvEUgv*vJMI54=^+5}M6lvtQe*V$s1%rFHXOn$`@UE}(yExvG z@4x@!x0-AHemf+8oVM?Wr>orkd3qSkxBfVuA5CWW0o?E=;eL3sKNKW3{VV?V`&Tv# z`l>&@AI=GcY`qY?|F^D7KAPTH+}@IXRRVF_Oo^^!m;3$KUSE)H-I(84Y9eodeBPNx z=iw|nUm$uE-uh*?DhJu_UkT;9O9dToC%5`S4}l$_?~<;touFMla=Se*(} zVVY$Fm6bg{Jbvu8zZc>IXZJ_R<7H!K&8Po$H$1-^o@*)V?X&?{#KLW)F0ky!}$-dzg;fvME}f-QJIABYQFqo zC(mcE`sd?$+~LU%@Mh%vfrUbG^w%cY6J$}ERhZpoO}#vme$2l~;vTw3kxZ zd3AF$+hi{Y7bC3LLEmW(;lM~|oU(=&OXb#XD2f3~f6iv4bI6f;SH@d=dR%$cb z2i&YL#X26;d1vg0Is=2+PoCM^+|lA^bAYm{hNQm?QO@0a{}pF>BF ztAxK1V*az%bW)JbQDoC^wHQ)JU_)W1=ASVv?>>@#KfTx1R=~Tc^RKY!iLPxkro+Fv z@eQMf?w5ui6>-y5O$58OQZemH!{$bLVR!UeqoR=*_Ol&Z;y*_+@ERe5ub)Yu_rVFS zW>v>YJ7cEfr9>B83sS0a2&*w`k{seX3{E8)UWzbHgzpL)*yzTanu5m8{V`fAh^Sq2 zsuI`N5@1lD`chnx*qd87R}C^?y5mbI)CQnk3B6;zDg zv}~#H;M3aBq@c!YU%+ZW7C>jqI6f-JdkaDENK2$F5OBFCd0MSl)cP5TsOQ$huR-p^7skdISxGGXcI zMxnJ~OzSFFfKHE&S?j1`sX({W=6KY7P&WT!;RYt$2BvSg&Jogo zwxo?_$TZ_liw7H^7>i2cjFxluvdTQCd*qD#IPN#Q!6nKnWFP1|&rZv-stpC%^-OiL z?^V?{TqIppHo$CNb1l~Ut!gAY>6Z((Rh7yMqe9nJ;<756jzz9l1@sEa-EE}leAXW- zQ*b$dy6Z*+`XP-?PLnEuX$`b~?l$3GHv=7Ha`EaModV+1nCmVD97%6To63hXCWvoycmL10%p4 zWnQ(m%;kuwxl<*`&z7K5%$a@A#$0NWMI5D@n*~cC7UGPOqobMV5{WQg#=%KZz#(B( zZJBSbb*{*3rmzWOJvGbGfqlIn=x$ZCpabAW0bj;MVXcf5O5HMrd~ZYh`QcH^QReE` zzwJvopq-J2B-WOtDp!KPk2q7DbI%{nItaWo^o(yRh{H3_2ydhU-!h1i7gKxOOR;4U#%rEj zUy^*1ibARYx&;+4+W1%P7;WA(+%4!+L~4{8cs)5D559>?e4l~|IMHyTUl~nXW(IV66h78cxLUKeNCwZ_Au6OtbG z7Mw-zWTx3D()MIWQ*d_2c>SHQR+grCPCgv=o};*fbQ^VU2D%qAT%xO(vK{mEVRT)! zpbu^+-4m<=wZfXF$GN%jKUFjxm(P%AD_cJWKj2Fh$;*%ZEaM+Xfak}9X`^re z{=&4DU={qg|MI`t!b?@WDE^(f?|D2f=z3dI*&79GTZOqziuB*%|6G?DZj&r~k((jK z{Mx!`X)b(7ps4_I;P)tTffCiw#~RgY>;3Da58%0(<;wC0plFpfWk_xp^X)uJFk5@=r$G8I=~fgb@Qr#m{MzxH7>ziE7bafdBpS|KaXUmLx~AEwMen z0us)Wh-h(-#ynK7A|fM)Vh+)nHFOKB7jSd9J2K1+mNtZ^C#6LmxqIu!1(&sBC5IK? zlB)`?`4ROcyyxmw1z-n}uUH}^<*`Y5mElkW6bc>gaPB!;D}kCFaeDlh9E*aarBBHi z*Xz?mbm)JaWP_0DI2^4o=-FrJ_{}N$TCDcV$o)%(;OU#gas20(kD%3c|9biogv|SO zfBeb*^wqFE?jS%d28sR2RRMhoa^>#1iHC3c?XcH*^UUAZQmpvb3Y?Xzps$Pq5_^xs zuQh#jb=Y-sRo_sf8PT)fe)juIO_?3E6iA73HKq3E$K77Gd4^ssK*7Uh6`t-+j?mmESCzTIWxkXs8r3v@-ng7w=q0v~yFHTUN7U!6 zkfQZjOwarKAP2wf2+CRu;NE(YX>|LEEp9o><@s1e`(E0%*FYu|FVYMO*=uxghxxR< zLuUO(UdQFOjn_FT!+MOyFhoATAQ}?e9C^A8`v@53~bioVv%0n>v5EpYZn0K>|Kt zQOMFq8Qu;BzkYeUNha^z{+*RFMU$~ip^u0sFA#QNt(wWn;96x>88Zc{Z{;id6CjUM zjzIe0KiF6+aaJip`8ZgLEfe$OmqetK3=Cuxo^+wVMD;)zTnT9P@uw%I8VnAmbK9o3 z6i$M=PzL|6gNzU=X2c3x(41WL#s(XMV~q^sY3vYf#Hgek$yktrOjS{E#fuLcU7txz|5H7>(XQqI=(VA@vWv=^JM6k*8bRi#bC!4pWfy8)Xp4in$d)8nh zV6pEO)`4rF+IowDuwbZI!)!sk_2R;a51^DCbG^be5Yxm3=VkOsdRd$u5N0vkNx$~` zyGdS9Za|F*cfG8tjCB=9rx5CO<$}0v5a?~vioxL_Ztud)$9$vSn>TU6SlQr#8uvo0=9LC~hEkz?om!Ce%` zrlP69C7u)KXY3SlE%GK|x=EO}?s67` zKy|^fday7xBwINOZjs$8l>y-LCb41cGMG_`r!1t9s|4v${*LeLBLWo+K`wfI3**bX zr>w~?y$4LsT2zoYrfB7TA7+X)Nf%Uo!*{YqWu5Uh?DJGI9Vt53o;>+!T{jthVl$6@ z8(GejNA^)dxeDb``iR4i{w)!wvUrYtO#-V>3^p0=cQ`+esXm=R8)&uDNdT#q@9H?n;pqn0no7dwy9 zH4(chp=(ug60QflyI{9n&(S%7WQ8ytm`pBtTV@W|R;97AS?L9)fSic1>vz%cjSB`{~$P z1LYWtVrb4cj z5>vMSSQGGoE@Zw^yIhD7&3ut?ZCwMf$8hQTB$J}Vrd&e^lgzHi@d6Wzcx;bB*|=!r zv?j#KEMHncBAqp@13$d%vdG~r$&mUxELe8aQqR0dvW)31Lq!&{;uQl5e0;&$WY|;* zwy++qeUYGS_@Jp|H_C<+Y{E2w5&Q99MCJt-1O$dqbf9D%O^tZD25D0ltPR<1-}n1r zi_DboX_||PLPNIk2o2C8rvv~b5Em-#8m2Ixq+?XJdf^b9=BeW<1AEr_O-7Pe8B<2J zO>~_busD!A8DeDCsFdkWk=lKy)~7Gg6swAy6$^4&ivldn3qHVmi>*tGIOme4a4NY* z03ne979!j5XKF4U4>_z1rK&k6IQ_+v!L`^o_CtH)OG!ayM?)S^ToAXRWvzV2cYE-m zx<+|c%3LEG6TaO@nguKb*UMo`_Jf{7IE7wz685xav9g<=ciB^~f-``>okfa#6It(c z0*$GD=vXi*+9=17Df!pJ@$!?o4jF3Z3gV{-HB(5DFVGp^?ECj4iEWvbG0(wJbeSdM zf-(kXB4RF&;EFd3Fl!5J5(#VRItcAZr;3Vb!l^pp4{IPsK36#H$8>*;G&L*X81Y+-tzzws2lhf3Ru>DpXq z)>_2k%F{M-)55p%$QwE8@|rPebKpTyf*ywF>#cpDB3;n9-UsgAys6@XNU=vCpiID0 z-uQf;cJhgajaN#S9VjNslFiPp!kOk%dF0|%UH9|*2bOd>NizoY9tVHa0&k1bPBx4; zp+$ubkh^Q15Hv){SHt)ngTA_~XUdFQ?^F4TCF30?yxAb*Oe#W5TwJ(%Nb$14CwYGN z2al9Y6PP7rg^v5Twn`c?HyVm1J^=J)4a%5xir0g$+AgGr3q--lSbNIZra%EX8SYlZ zTbE*dk+(PlA3^a+Z1eN678B6d8;g~Z=M+zvp|_Bq~S}TN(t-Op8VCK;b7&w23*>(u_NZJ^`V4^LQC)3mw)Fm z!$+>y!z%1}@(OZl(}8-^cL!g;UBohRxcqz5hWc=s8En{ZTQO5R4+)Z4fq)>rhxa=Z zG2CVSamCUE?3>6j5VKvkY27Z8h;y8=>3`P^cl&l{E^?ApOAMHr1+xYS6)@x>ke!0N zH(@;y2P!2!l=G3U!9``XI!NqWv=On*bK*EKhl%MAP=iK zMX~FKU63|MmsqHAgulgNrz~+Y^p+%u!O)sA&xV!W#0A9O=)C%A@HDUxp#cJl7Ed6QL}&QLd~AIw?L+k1T02-^=M{M&v%BB0}}5?_hqgMoBFsr(5BCD!=9YdtM+ zmQ^w&6hb^4WvjCmpu%F#vwq}CNE?|a8QD39`_rgOpB6FnLAdhKyLGW_)-j7@c;C;p ziN&)k*dxCdlj|y6K|NT2R_YFWt4v=o_$0 z*uS(>ue$d1zU>cx{fB@0t$g4QKl`g51JmgEA3A%ye+mF}!~Vu$l#n*V1WJ;0VVZbQ z48{wb4S-iciCA^n-wBm#F1`7(i(bihz3Yx+??;&#Ao@@Mulm03H+>uK;#A_- zvR)joao{mn8SzJu+1?Hh4hW(ex(E5=h;SF)2A4&_Rb(v`%8?8ak;z*Q|)R#(Z=yA8@GYw znOlo>GOFap~;p9t!Y#yo_xDTR^)Wu2#T%S!etj{t70GhbsPEP$h_5k=1zY z4boDA#FfmCX>orEkOY(1sb7?!EHGxFYWrT=B5&e^RhI2bx)yT(W&xq#%F|J;!Kmel zS&~5atX`tUau>mXzmQ#$)%wIqZI_&-n1G`Y!_JnJ9SnV-YTjx6OSm7%al=C;do&=p zQ1rBHjToj|2qBx`lI7%0W*yE+d zsbQvo5b_r%EU74BGmHP$rA#sktNLx;nj|qx!-gURnIz3ZEyX%ymuleK$gAA=z*f~X z1&{EUyTtXfq2$y#o~Wr(=7+n}lg%{%-y}s!reVBFeDW$~I@$mptIBdg#cBfUDN$v3 z-qCl)sw|(nrtH~TUg79LL)Ay-lDmu@jRjZ?U4T3`=QV>vUMg7xD$+eJEu3-Gs=uFCp1AW98H%pflb+dr)Ve-!cnM~S4 zJY9lZtNB%C83X`yijtDrfeRIsC^Ui?Wef&$JIGFV*Lu!Zh5-x6iIufIQJj>yj%p~vT0cioDrc6#=LvBnERKL)UH-_!`RpRZr!hj-O*b`nFb;$f;&Ym zDoTN4ujg2mv~I-fA^hSbOIyp_U{JG#sRA!}Rp&_NE-a{!E(U~%to9j@4E)UrZ^*YE z9R!WYjER7qA8jqf0i?WwglDd|*M6Bp~Bt3N$JEn{#1J_>=4t`m8JxES$HU zus(<{(!>p+dqRr3iqnN-7O`A+9jhs(U16rl3+BEBvJqqueG4(!5-yFJWC%e{M>BK` zx4K3u{Z8rQw%Z3mR5_m(*u8H&hj}i}X-ecWNs{`@3l_odUhG_VUF)-`mtxM_SY=@P zq4Q*(?w3uJ+`1VFgKVP}=Wb?VxZ)5&3v*!)wd*O=aG46)XsRHSvj7kB$c-eG{k33y zj46mW0rsL`HBV>yxF(6RIxQq){M=P4AXYe@B##pDuX5%^6I=70AI}85M-4~DCW#?1 zp*IY0J3MZ#2QP(M0f#}ZTEAcI+gmTG3TAL-q4VW4;HCf{32UQx+(4WaIV<4GfLXq+ zsl+m3?HA507gbWT5_-Q0qM%J4vqN}>l#^pNUXXhT^)8@VsF}hD-L-sMnTmtMY3Ui2 z1~^MOGk@&Xo$e3TNQ_R|9VjYuA98I)32H;y1rP5msV&Q{Di=7wJjNdRT|h1_By?`1%h*@R)tF7j?oQiurCQ&#&^AJVci;bkji$z3_S+B;(f zNyYQXxeRmUm~cYuEZ?^E3mB-PG-tk!S*1{%aV5@(D3hH;LthNm6N4 zP8$8+=XtOcf)-?t%AUp-;4%hUukGRzV=mCB9S*0x+E&xd*s#AA|7aZ{t&8_|;)`!T z{JigX_xQg*oI>j7c37{62iJ#ox7!atDn2FJwJzh3G{^fZulRTi_GGUGfczq!JS7}| zS(js1EtArVsl4}To-qNaucqbp+m3!SXxY8}CP+j=kaQc#_;uR@vcK?~Lq78%$ukDaINKP?&rNq7{2V?VG#zJ`#Fc{+Y3DD89togt@B6L(&xe1; z|9;h-!0n*){eTGHAoH`0u0iU)T*?ncVxDAZUB`8q|@73YH+(+pKe@|?KOU5 zbP-=c+<1FA``ahowx4`qcS#8{SLd%v=BD!{vB=v z%JD#Gg5|4eMfsfi$*eZYvhvDrf9%|3w0=kJ9X#7()NNPt@YTIjE7q6xHJdM@*LEYP z;Hn*UMD0Tav@Zr&HDhmNc6mniZM5q4AP+#+`pVIgNdJ!W)#=9ka4&z68D;r=WU1xl ze*fX;b$|WTbzX57M_CO~8mmZvQNRSc7Y87^KI!~FM#*K3Mh=B?4n}xmMali~@$8>i za-QL8ls)!b9W8JhvL$K){!U)~GP-r2b8}iBdpRVd!yq#kWA=CDN0I(ppna5Kc%qUb z@IG13e#{S6Fs&98km~Y@(@Yh!nlpFzMota@Ba>TA$=?-+yTBgr$kj8L4Ua?S6k*iu(GP!b-+Ddtim}yMM44&+DvYQe2 z$_$c@TNJcX9=uuV+GHACTR2hdUE-=jE#Udf`1Eas{eTs>0xyWtuFwiF(?(mAO`b4w z$rK(cQwAk%i*=?UnO0V^5Xgm|6xx?%*08>Uvs!(Fn7P@tfxC_^gB4Rp+rYQVnN@Hk z&?xgfsVQDM^2C&hEV7gW4i^``mit{J$OnEK}8qRj0&WE}~BdW}@LYk(7iAusY#55DMC?CXpf zNJ1LCT+(NmL1r>>auOzX8|z*&Et=z#E||8^aB!?@Xa0wV*}fr&roQNSw^ycI~B{8*tHcS-p$+x+b-l zolLAGqhTp@nyDP;ObD`8D&9OMB!y`eHzhd?q2^6| z`Vwl}AF;BnO<6Vz%3r=nT_Fg3yJc5akSxFu4Y1LcCUQC-;eTy`cxCkExG z6uKxp4zsFGDDy$ndBITe$^HU+G5O|pG1dqSoo9Q`bBz`Y*vQV;qSx61ALbmdkd$}^ zwFwx_G=YLVuQRnSyqW@t6j=f|D8jN;EBy&`l;5Z0gIAB0PsKZ#ta8;Hn znRET!R*yyhS#mk|V&a}~{*!M3DAGq!Ga%k#*#*3ZjOsq1O)9V*o=@c;ck3Y(#*0sy z&^!)lJ8M)IDH9uoJdHIfmp0N(chh&CKSj*`riK%|7)tO9P-QG@E+fZIW8VNUu$pc2 z5il-?IUE^|$GR9}n3|BsNSHco$|i}57Dmd2lt-9a;J?e3RK&JHsUC_+wBTnLH(3S% zsT0C+LM{gm<%)^)T=K(Ofio?Jj2 zH8H!%B*($jo__RgGWnQXX>wZmL8&Tv5(S`cdwC&W<_v5`&A}ln!vHoz&S|>GgZ~h zX)y)Nq!k<-UXZ8v{(6Oy5iZy|h_Qg^r00c-VdF99qxg2mqleA`c_Rni0-3e1&4|Tb ztruu;T&BrCDMPfMHh^>dFZ8WYOm_Y>`9ua8Lv_`H#ih`RFzaPm6tW+akv)Z(#voQK zSRHJvhlr)WO~YKMRbrPjF+@^^hflcIq>cEkY{jB5#nY3>C(qNSh^tAHza1&`|42UD z)wHLV9$ZqyD%moLk6URv67Yt~ITM}N;%kK#tOjcfsEm8@XeZMU=zwn&Yhi!1$9u?z z;zU&CLv8-(Syue&`DLV3RA|$On?7rP75()e{>gy{W{-2v@wNFjIAfu^bp5_-k5LQ# zne|%kEMI1r)T=S`^TGods_D^Fd*O$he14a-zPbTf8}LSw%y&jzw0571J%Q= z0(>5Nxu%U@L@%QsPOb#oBY&LUhUY`k&H%#T=qFT`Lx-?+FM|F$FwHloZhIxh#g)Nv z#Gg=InpQs5zb)dcP5+}}WCoB2NCzIoW|i+RrrL1^Y0`^_cH29fb;8=PRZS74PcuRR z`QO$U0FEQdNIT#-68aB+e!q916m^kaM4zJbiFo&yi1&G{cerwT z4bY2m4>Y0zttPn=Z*>Br(gP(tRTiBM^80=*bL>t!DtCT!y8H0wEfj0ks+f8i=_V9? zZ84tci!Z*oRIJ#ceK4h$(GJ1GTChz3f+hdG?H;05wz(U*Y??{ey55en7a0O7ZoT%4 zD$GK8qXPuap!(CHHfiWX8NR*?`I zCWnuFfQOO$0CWsNCL|!rF2mc&#on!M@robtzvv@at^khQv`2jSXK%ks;^&uv1wx!h zFi8XqpquSk{g*%epa1*azujbiuOJM_^v+}9UgxDO;+T%g34{oooC!sT4Q~Jj1VqC~ zBYT!5O;ie$LOpW$yWwz6%AB32xe!ZQX5uH40XI#+NuvbC_6xG)fp=5eBqagcFD~vn zyAJD~Lx~%Iae=8joYo0q3stcc*z&L`r5hlBYTRP2QJxm$^UIiR4V@Lkpd#qa+*(ou zQIoOT>^?>=&4!aZd< zXtQFh1(?In$I1ysh{0Mc5XSn`#oCDFzaZJ(ji4G7N>hkimWXo>YH_r34p|L8=};$m z!9@YOJG$#}%u%4TSY-v&)&dvmknvpt_$@sS|CC3bjW(DQA#^vaFCG zA^$mmSwQgi3py~eWi?sCNyNebJ`=x~x5RL`Tca$E8S;1>Zbm)?hMtXZE^#5#no&-k z;zZp4G+W5lx-@11ToTvpIqTlFfP5B>QE*aR8vvv@_J&j4T~}g_1-zrX3a)j+>)xI^ z&o6*{9+xQX_x;^Oyd$qr`A(y7#*qoPl#9|~(%R*$su%oDkBKPK0_w?J6zG`-#A@KO zRCp@xvP_1CHc}HK)64lnLb$h!rHleWJ|29`yoe!aK!KdGVyHk(Hm$jd#za#tX!(pT zpEZ)Do6uxcqrUS9m!`=IX0&jQlr@1sH62{%5SK)@T>7A5e)}a3j7!YHh$(hSj_C#{ z!t=e%*^JgC*Oc7yM&X)(Gj(x8be9pFPbskM*NC;r>xDAyi3~uFBH;yATH^_kPSzYs zKTFi+C<)`T(Fwp0MW;3I2cy4Xtz;?DJocWRYZA-W$FZ>XD2rtmig97(W{jpD$-|+4 zoG*wD><2lTgDQngC>9f&@xx|du4Z1>Sw14GWNb=`4EyXZG$~jI35$2rul-Zvys0^M z92BjfXej?b*N2Wc9H9T1McJDfnW{0-4l|AEX3Ee;cxgXa&~c%;3U}ULj@i(sgg^$S zRmLhQ6M$=wH)61HcH<#j7hCBUQ%KSxPMK?bqT`Uu@^8qPZ}+SF{^-eRvRsQP$kS@& zkoI^Z)Xp!EZ0~)wY?dZN8UkSTeE^4Ako> zn37BS^^DIvRaF#1_|8e0(+WUgS_resad5ngTW~%L^whToNC&Z?5CT5^PZMAW#Eo)R zgBe*R#OMX1%=y?)d5Tlj$03Z0N4AOi0+16esndS9_A(4|fY(fWq84ZG-x=4p?$BY!^4qH)#}eIr|j0lc$vH0zjbrF}%u4jeZ zgc-TrRgg6>hYuo~c0F54B}_|QK(knL2Hu?@V^UZ4?JB2PX1IhLVdYolHw+za*-;5o3v=pb`fGZ4{nN*xc` z^oK(j5?OSd`n%hEntPm749=Y*h{Z}s(G^-#TaP4&(**)@7t`}edD!DzO*Jw|B;W=h z!(L7-u{i@|A%?&R)ko-lWew_rt>c?+4L2&>q=3Q!*Y4;R>=$L+fMSdGnvsJ^8p!Cg zKhv%d>bEAcXQt$a?E0V#09&07HH!w!K$&ojvYdTP*^jQhO%ZF=#{vpq7!!L{F1s|z zD7QXspM#U0<1_osm=A9f6s2)Bf~EXqj~DBd-&sq5ePUh?$*Q23*y_I9gct~TCbF7~ zz2;Rt(n|JMGM8_~328mHLS~hm$>$TVaJ#D({00 zKGQ4GqkZQS*NHd>Aw@N|eA}*N{W)5J?gpcF{TRL6-t0g89DUhEUw`;#=~8qp$2<7+ zLD{2iRRt|D9DI5DvD;jOoIO0*KWTEwJqj7zetpyiU&;G{KHzEel{g}+Ui(=h8}CRL zG~^bka^Jxy8Mg91E~EDwd6Kl3k#Q;G?hyFQRqGbD`|VcQ4BS9XChv4y%CxkZ@XEhs zU1aVp>ngsC-fVW=ewe3C!>e0f+da|Mn+*UqGM~RQafT`+ zs*jJly8KJa#L{%H-CSq+R=(f0rQq$oqID}HfatpHEzc6MAfMYf*pz*Z&EdoUyv4;P z^%pUc-G23`f32R6BlUVM)9mr;l^ox9;s^IIf+#}OkXz@X>~a-CNJ~+O8=o9SqJRj> z#yBMMwBp24Fw^F`YmsLO^IK1niU~yC#G4|HQFnA%3mHku0O>+-*O8|sZ)%c?rzPL-d-dS1&Gnt^;JG zY#AT)z_kWNU2`?!S$w$$VP8cPE~peBZ`+g%EFIGx@Sd1A)1r1Y{99%}oJF2%m=Hv0 zKA~Z(D%Bb#2hA>m{+piQt3IH_PBZy^(88@y#m;74$4A4(Ci>*P?8yqF5;8^n+ZMDAS}I>Lg7J?tTXPPRXH5_+AwqAg~tg zdY*7C;M5x}5oM#DQw}f}$}4kdOw;kDR-rYBWvppW*m<4FG8Olc?hh}?T~1=CA8>v0 zF);GS2!PX6Z&$XIOCXS`{*RU03)y}11@;~95Mw%W@mdSJ*RJQ_A*9XQncc2Q)nqIX zhhhlJ#a3m`rCN9sPuQPAA(AbpJB!E^Pz|MZci#ox;i44Rhi2u)g?~U<$c(04W13!( z-B}j@Lc%ccPoS)Z^}==IvIbzuprt*_(t@2oBm3_p4ugR~3K^)132(-*dx2yG$C#au z;99`Kq*6oUN&?RjLkp~67}}F5?LK3gBZm$+SCUV|89CTq8B()R2hz2u%Vr_cE^p|D zF^AI6l8wAoC88jUWIe zb%NF7)68{TW$8jq;Yj_G$sQ(q5Q7q?VbTl~OI+J}Czj$?;~>^b+AuBaB#B>%%}y`< zAXDnzr-LgoI4d&4Zr5YN7OD`sUl>h9kL zm{3-T)kuB$X)s}cA}JCEM6TNXwNIbJDyoSez~|<&EHA)fOJ<9`N7{hxogAD=mN5g% zAEyQ<@mw%Jq&15RpRWBJN}vct`;r{^{<`$zUt)gFP@#@4OHK z<{%g0PNY%9YQVZVzeWF()si>GVqUlRdSheMK32FXp(LukSYj@dr$ z8v;eflt|8tbxk1I!^eu4Yd(vg7Y(avpTpTw*=5U=Y}-xyW54mV2v}FNc4q}!sVwh` zhwyg7;g>=th>f?TE{!=szA~#_bRglGXf1#;U;&~Ac3c}du_j`2T-W{ezTF3^3CQg* z@7&twSW`Keaz@(D@*%Q#Q?RAcxb2P6Erl#HTaiASB`;fvZ?ag}Yz;76H0MkeN&E$< z5Hk&@l?J9V)f(oD)2f=#U;!FF!=t1rbRPLAnP`FlS<#RwJr&Rq!v@O5gbG{_*B;>k zse+sp+rwf=0GXn_?!If+mM}$&lBsljblbIw6$!BM;cHP<|ItrZJyzB!@LtQAYj@F1>1tS4}jgYbj zDF+kibhE5~g^9V4vypc~WeUIdi%^@y9Q$w=#p_O) zF~+3=cKZ6mhw5hMB(!U`ft_ ztl`JUyUt@%fR|^0nk@j&Rsm{!#-RM=e{+{J@$h+>9d;f@l>j!F&=wuY+7gI7*2ERK z9ei~(3gQdSGvBG_;PEXWy<+;u_Bx=BWao+tmYW;`RZx;9I0HC!@4@6rg2*C8>?UMH ziJZM4Y$e>ir}~pwlP?c`+i?wH zx2P7-`2n^P1Kh;%t!pv#jvi(p^D{0f5j1C-sA|^Zn#o7TN*}^1cf(@ws4Pl@qH~-L|K#@me%s?VY`^XfM0Z2s98$cTDx(o?t;m zEq3vJ7dRlG%}S4&;W=Xvn8j>r-Jk3-su(I-c{`uhk_JV-uyIZ7eUdrQrflUM7B0rbwe4s+!HYM?VJeo6J_Dxb9Z+mTcM3PBIwjHoH;7%)fH zH|-9N>ur0}Ieb~*PAxLZN54kKKv{lc>2x$&^})3aNe#5$0cE@d9Sb1X>o+uAo$lOCq-uA z9^Vcd$mi@^!;3wQ0Bw&}uKlB;h&)XBRo2xpevm%tNYV_b+2p`?alXtLycXZfPIZz zb@XAXj3Q6#6`+GH=I|o=1G?bRIMV*%cz@}@Jnl6hj}y>(dz*EJXtHEj7#24NJ|Cpj z7u)TC5Q~i5>u}YPBJ?V8avYRZ04$x?xMlV78A`v>|lTjQap+JGq4n_J=>q-@j~SMf8|z zz^Z`{aBu%DZ?T2ixdRfel|jddqhVJ-SVI%IXc%F>i(X+~$AgDv%Fo1&^HKN8rk|mjFe{F95+H9m+B25zHrZrR3P!&I7=}Um%t>-~ZP0 zYQRI_;Mx?R1*YO=sgH7yxCRg*pujOcmAMI^#LHP*7M7n~2Vwm}Nxb*>K|hia>T=~T z#&t~&y4VlyY6jvQs^bedx(pJoTt6~_iquFWF%mJLR}}}>EYN0ee8y7iVA!t=AiJur z(}E&Y6Rb&v|Ho-6oceYhLhekse%b~NkqW9`3z&pJmsx?~fEkARKwezs;@h(1vYzN3 zaFkDPS?fyOoF*(2}|K zhMa4pZZ9jSPjDb55JCh45&OXI0Q3yaN}Lden?sI@7L}g;z5#-$v@)H+Z7ot+Hx!4R zAnOunEmNpr5}OCsgD5q&23nJGs;QXC935+rE~puek}(xv59c3iitCdRSkSii%c>}g zG^UWe4pB&B6eaMj3kvkkjCO6(GGTm%IvsXhrW8eyhn??+oP-P}nQ^H@$pxXK^IJ?b zJ0OTqa4)X?Rk71bdX1GisjiKdjgh%{CguY5LyjJsd>^iZDTB_{37|8FqA^$VngbWm zSTMesVrT+wgY7DY%r9WA5>zbE5}1k-fsz$gE=8zDAWx1mkCcm(bxR2{IC4DkDS7BU ziY<{bqeF6QuAgfn$7oKg9bQ{NXo3?vWm8)j8$5hTjvD4xo} zWHvBI49*OXN)p?Jt0$StqJ#uJ4~^f_p^{x>3(7i{jL&5K${D-BZEWljur5lnVAf++ zER&_NAoelS-a(L;4)M(p5J6=eGZO|xwuY?-d_k#u;fS0dSWv@Kv&wqdYh+n0xMMSM zLNsB4xV4Ly<0W!(N$$PyMN?%pLx^XT2PtZWs6J%E#Gwi?>@nSRUN$YQWTZ0+$B%qj zsIw_))!q*WWmvJUmJmoJr_W*-)&!NV3ndU{*QSNmE0NtSZi8JHqm4x)W@(TVtCnff zEVkmBBq zKtN2UGAR>21wGhKyiD1-XdL$)7=edC03xmuYi}Kg&9z<6#UvtB=uB=0;#UciZOC+Y z*FmYk71ZZE+X-N)Dfy=THk^|v16u&FhkQ>`Bbz`r=7_cgTUo=b;NBLOn?r_|xktx7 zirLbzkqLVsFiccMvryRQqV(1R#BpT3VaP|rBdHX~M(cS(Wg%u_#c_%i5(N|X4p@Th z%+8@43TxqG(t_WvQJ%>bL@R0?w8rJ$aScwv{tAgyVG z+l@bmYyjC=I;z${HkKtxu;YDq<6WUdthtH_AF$@HZ4Ra2prBO+gz?cQ;GnZyi($bE zfF4qk$_x8G*#E zt0%#+mI+)ATTqj8^8VstFwZTCWeZ@MHw!EICNbA*DUfoCw2&FV3{5xP!G-P=(IHj~ zdy(p#`4*_Vphr#k{^AB}nCzC3R}u;(3&^IOOAJ_(RL&pT+?B!`ezVsgk@07WpIKpo z3`-{eAKSu6*^lHNH9{E-@(z^;F_f=M)@IpsVk-skgq(Oy#hGIjigPV04NQ}rI>OOo zn4cvrRQb}wG^1WY^RuB-9QjRa5X%g&$mD{@-gaQ^8sEJX!$o$VkIQ{P6p?4S>ND;~ z3?YD7 zW!-T3%x2i0ko5Am67K{8ujf{L$cDORnanbIe{#wCjhG7oPf-=|0u#-Q+9iXk2E`2s zP$8Co%td!IwY6AowBX@Qg-gH#Q?fG2yv#IKz*9`CEaV-n6#yD6nmvQY$U)h_dR-g< z&-4O9HRj8518(j_4Eaepys1q2?s4ZyWn`plE|v3N*EK>qpiJml@*bv@0}w{lP^1RB z^1eZmLF$NCe770L1|HS|)8{6r)Wkp{i3W5>eX;9=icKl-1q4h?~(HNpp1{)?p1rY?M^?wh(2|6HM?$r^oonp2Qrqk%(E$? zMBf}%?LPX0*aI@+pYA{WeDmSwy<#VZ9bV`+{YE~^UT%OS_lJMHQK&=T-fFMD12`r6 z42lm*SYes^H#eu%x%!7=jFG!*C>CpAL~eNIpq*^iP+xWFbNp4lVW&6iZtT5GN(KTT z#-ozQkrhJ))K_E?f5hYo8!oBQY z(RT3nalby^_xqbGoT0ddYbnO$^cG=A!UqYT^-Si*sV7F=Mjr4etwtSf8H_A*{6%go zE~BJ!|As8gJ!D-lo3*ZF?;apO7SH-HKu<$2sHV&8p6QDwva+M-0HhkQAks@4-o`y} zFU>I-4;}2j*3U~fi1v~64y*fNSi6VIv;g)^3xsTHU=99mtGw*MJNOR&Ah$d!0O@)^ zyp#K$c21WBU-AfSRC^L%ZqW5~^4(Pk$*tT<;WkdQ#En-3^9bQL(_{hPQB&40Wa&)R zWHvj;u%tK`d@6g{F9BVM{=F&rIG~bOPdSZ)H33X1bCWFc;po1!{6H2mO3ndOIa7o2 z6V2_N(=XvBqRqH_>FdEJ?9)S760N}|t8tQe+>5M27a(vHWT$JuwkB`v0-C3N_xB)Gk{2LZ$m!U-xH$F>DNv;;gagjc z&i;wRnTVgT9MLrfpjf31=_VDCeU}FN4AUaF z;dJnIYp@u#g(*N}2ZU=9H)TU!M}VC`jU^_lY=m#5aU#AjW#;@yp=64EInOBtd10o0 zu~e?|2jwiCy|YgZ=~puJLZ(@-C#*qkkO+WYin!rC`V^C-7akFXK3FK*TEYmPIeI2;au)BHqc6bm zLu<_)M%ExNvIQn*XQH?!C5ls+$~WZNu{$!M|1?uwr*&PiGV26jQKMXR0VQLj1*BfV zNT3mj0;Pm|8gd$6Dla z)==~6$nAXOFB!(+cnD?V9Gj@KxspeV~(6Qv@V!FHX9A)tBOY~S|K|fC#gqA z$29`8{zvrR6q5t7++U2uLT7^~To@47CRgAM?IXs0E*o}LP)|o1!U8ln40&@7xNDKc z3*70`M4z$UZ*{rxvKBVvXksOj5yA zF@bXCh^ZWqFcu#fV;Zhi4Td2)y~i|bjwCDDD2G-KP2ZqaZ28i*V;VUqFX!V3p0{1k z#4909I9(1wD+%m!F@ZAXT1kZ%W6c5&|Hjgas1p^d_xxoh_%6`!QM%V`4A(-&L$)ZQ z8W~1TC;2k8iJ#@Za-9bY^huVy&S8bnnHhX~Ym+b>>LDl&B~@HzWZcelqohe5lgk-c z38B9LaK@qm9n_DIL!eOd>3|yEt%p<VQQUT)usv$C9rW!t=%&M6H!C)qh?8D^T2c&0;QVX6kJ5<$q97~M^ z4Hq3u(M`2{3qLAgI*P2wEi5H;Kg-<9m$wC;9TMxYONvCM`%TumtK9`J1Bf zhtb$Gep_i1Cp@{~s@r%};v6wJcaq&tiw39~uAv&krM_L9@vv#=#x($Pw4`{3jhkx% z!Cm%%ocCdmfgDaocfo1t$3@AFDMkb-n(%p2F@#uC@>)jM!3DvWQ!g+(?ah^g3{kPb z64wV3np(`^C3!usKLm^dJ5duvDa5R*jKT{ea8*v>1=RAC7fAN7bV1k6d9D5~a`vTM z7(ZjlK;Z$GiqGKe;CT^`jdq0z^jMmtC94ujCk=H|*76hzh;*kL-&HZ*n}!U!efNG? z`(B8FT+#AInVGv@Af7-OO7Y6HSO!N`Loc?7B5PFf_ zxCOiD^{~J9ijhfy+8RPqy9h|&Qpp>2b{EX)5jv9t$3utIcGz#k0<>wF1BXay=^5%0 zX9G9@d1=^X)blT(I()V{$p9v!6Jo@MCYkI2Tn;e7bj#u`E3O7~IXG*P!5b%3gXfHZ zo>9T^?tN=wb0}7lbDs=4)J6Nng|LBa0z{R&?gNcZEC$N?0{yWar;5012^0JK!UwsW zfP?dVDCa0LXza@W`5Uk+fVd!J$v7Chj1z}e#{|_T`#U1mv_0>KfTC8|RK*OD*2Y7T zfa5HQ0QDSa)PSFtrqZP1DX!s~QO(!Gz7I?;H5io^ysw!$lS&UVQ?UA1uIfSd`?}@{ zGY3)vKwB_d?=cXeqgoS`!DMWvu?ZR>K7o=Q_7?3rUWWJLo4pTu$>V{&A{)t+?L55; z3X?f@KEr3BprK^21~VzblFur3#D~ppIQZ`%PE^j~!;>7GHIwn7Gr+AndsxwkugoV+ z_hY~Il2PJEab5&&(qq8;6uEenIn^ah(f(MxsZ0$%KHm@Pu>>O4wA~W5u+R5p1^j{a=cgYcp2?3Obp*egLpyFV6jPWhF!A~BbC8tTnk(0zz z;lQ@x1I~JwRpN0qWX?5@LcY7zs<{>Vj1pr}zM`d^b9O(5FCFDln!#|>X(burD<$u3B?EqO5AHBPb z-hTJx6kY_{A|N6At>YNlpTKGKyY0*q$xwfkYCD5L2|n2ml90922d zy+hpuiQ?$l+hP5v|CTTIg?akj1OVk8B=7s-=gBMB-><%FSL^n1b|Cv4jPz(z zH8$UoXWzO1I7DxU=gLde-Z=o?v(MkW_K$z_-FNbjSEmC?x)0HJ8jHoBU%%zQe)08p zZ(e@#e5PF{-ffQ~px>?gt(?O1fqCX$FLb3lMqhSET^y6{+u(n$9Jwo*%HzKx{8I;2 z`Yv;5yLwc7m%eGYf8VbHoN6{_91u+wwV$`RO0K#Ox;b?4crG#9aRe#N(YSy8i=U!MqibxW_ih7p$`M=J#zM#CIjau*NaIKqFX{kD zY`5E!sz4qQ#g5+h2k;}gX)<9^n{c{K?`;DjpYhnOuRf7YVg@u6ll{`~LbCxQjHWes z5M%b{0eiKqUzzuVlIQDiVGM`UoddM4PWwH&>!Ss}Bc{!)Oj~&INy#q$CU!{mcXBd` z#mpG4i;EQm7@&41{{JsX$RWSMrNSYP1uPra!2A3wbnKALr5yUvL4I;^FA8!6@5Omt zKXG(QWx~;zlU=M6)|PJFTV;6z!Lwfott3z!sF{`m5!BNYCr?+z3uS4B7DKR)ydFz@ zE3j@sxM2!;L!NCZ``>!7T2`^3?-{U8CPl)qv3oZJAntA`Z5%K(4mL*7kf0Ix3{k=a>^f$eOHLAho)7-Gr;fKpdT`){(Tyc}j$R(Ap#mM4?#R?y|T7F*zgN0|Wh% z{G1{l8dNmqpa#-nAB~2(Hc)Y9weBhK31~Ae$DQB-6$n<8Tu(L?^Am?YrTTrd-mC@C z9}9KVKKIk4S(&iM;^Y1@H<*8STC$@NJw{;Y;cN}knoCP%j< zYXab?bCjbq^wE_xB=ui`Jcmg;IK-tAespNak~V!T*%?#bna$2{El{VmK$kWF2viw- zdoC!QAb8{!dCaKN8Hg$=>jfOPAKjn)=z>$XWc2XpELgL&&NvHo?^CX*wdD-EPX?Dt zG{SlDomZR7%8JY2rb(kxCL1RoC%P2i{bjxS(S5N-Rm}>DGXPBq8V6PrI<)={c>{%- z^CdHk-2c)fVTp~43qxm#qlt21-Z>uaI5m$feKA~5`=0>ETsd!U}?DTkwX=et-f&^^^hRo ztV{YT(~Pw=iQ|mU`DyMx2YV4im_~?G4dfA-0dBDyCQMA_ia{o4$lx4Q`W#4~M_^pb zQbFh}X=Yc;yoeXLPi^!H=!eo5o@Jl%94c`OEU)Tl+PGvW4`L1P^MrDqfmnx=ya87Q zu_{XD%h;B;NeTHAF9NW^SQcySpVg=M~=N@-U! z(Dbn>Wr(Z>)$0%SpCqR|;JEn>*Z6aru$*>8Lk08ePY@qzHH5TAqmU$pp>><&a<^5SAEFhNlCW8XYvYV0??5v2^2}mu2ut7U6{xqOA-< zXN+l)S?#Wae1#U1k5;)R5Y$a6;1NW#vbaFKJ39o9^)ziz#P`oQFMmkaqC42&P^N@OJ^i%F^R%uUr= zBy;qPG!U)CLpfhg>No@pSq0b)@7ZC`myMy|f*?&YyX&Boj_H`X2H0&K^wQ)YV+VZ< zdRr^ZFrEM-@rL;5Y0N^zWK;o9k}VYfbSQJ5+)gG<%~IK&2bC27cz;x6P<4R0#A1fH zNC1yq8s3}UtDA{`ST3;M^zZ#nlt2`n^*v`YDNs0)8A1FoLy)WkVyxmSx-$kDv5)|h zqmQY^oV5Te44u9cAUXhFGOQnm9N{|i`c2_N1z4$QCftVvglh#uV4mZK#yVzBkj@8Qy+)iAlk$%2|nsx#=x!dDLx0A$Bh`1EDJE=NSYiAkU<2cL78M1 ziF4WqbeHumbG$_Di}C7%qF1WS=Ik|x78Se?3hK@BRMxOU^&2A=rht>GN(u%R-n9MN zH;7T8&JeGVJ9W*Z4KNlcZA@H1;+~I} zKn?{XzF`xmUG<~Jn>0#mMmo>Yzi|P1P}cftH`n4(dD}8BlB!I&AUUvH$}K8j2hVRa z^azV24(J#IX3KrQJB&nP_>?UowHvQlb}P$ZdxK_r_tkXpCwKj>>-0O~zrO}@ z9{=%eciZhXGOSW|{c)Uu-wmg|{r%I^-n!Ic?@@}gA9eWGk(iP1w9(%M+o;7fGeM#t~hgzyxxz=29KAi&le$C>9fhK?L7> z2BIcy7%CbJ0stjSP^h!t;&@-uEY1pMLGBI0P{UJP&QS+)odhJJ=jZDPfJrdO%nCf_ z{gWIxaniHs?dRg#E#xkInjG9aMZrXu7Z>~9RH?w8n~=ja-4k51tccl|yLKnd69Xuu zF82{@6EU-(!qMN}Q zhy~A!a>RSrxmLK7VeU*W*95SHdU@XZ40jC?II@FvX6beL*dQ|axVKj|P()+l9&;cn z(DW?KTz`9TpnWV?4NoCsgtzASgLaXu?en{Dr!avMfqD+G?>sXy*k+o9*8g>HS7Qt_ zr$^V*XE3?Qj6+7=fiW%^V!Ird8FL0tbEj3(q&cnbDSr>RauVYQ77O=U= z%kZn7{N)9zrIk7k+MuW#ae#k;wx_6b>_EIQZ+(JUw(){^eWv)C0tK#wg}ZYeiM7ab z4r)&5V1Vr8(8B7SU9P|d!!fh1hc{Fi7YJQk+)Mxk3s!?&OB1f>0U*#V$|BPW>xI&J z51YtKS}>_aiDhOOK)!1|UjitJ`|Om2I6&U z1PI{z3;DL+2Wk}Af+!~!VqC6GB~94*SqFlI z6r7_4v1%H9O9|(QNHx6k?%Id}#|{!g9CkTx7Gx8ri6v=*TD*@Iz-sj{r(ot0Z8_Q- zGLxng&RWT8Ps5QXdJ zbI=8f$QLxo-Q}WK;NQ7E?fr3^6baOEm(57mLRQ=YhvigA2+FYyD-VXyKj?6B=5JKS zZfgMllz>cUj17rQ!-9cKGX*HX8Caf2qqSa62xc=oiKY^iL<_KzT>TLYit# z`An^t_Iv;y2RWM#&SFjEs889bo+g&#R8|6)3fYMGHe$q3y0+h-Xs!)8;kc14i&cVx zNUM0w^6^;44m#0d!+3>`2CHW2Ogi5GaBAQA1Tr);+1MX}PfAw6lCsy``mQ&79tA8Z zFTNept$}z0hwQ?%cUP+tVHdW9P9swWpUaNxx&a}?d4pk8x+$A@A%YMR0=`vNvEyqr<@lC4=#rDT{%9W6{_8*e z7ye(GS$%RnC{*o^hRa zj-tQ*)Bo_?=V4c^O8`329+xa#Ssk|5K=}joUfOMha+r;ibP4cqv`a5#bH6N$I11D{ zNYMSJ-JVcSp30O~T@px>bgkADhnitApZ@ZvcbhbNmM4Gt)8U7|{AtraKdassi&pmS zt6BG~WIUw5ez)~GJN!9=evoFur8fuUudK)5c9{HJ4NLq(`k20mTrNyW5vr+-$(X^q zhQ`PwzpOWSdWbgd_rrcl8*$rjF?g3=a0b~uEMd?y%YQlR?J~M;WhX;@HhO;7%1(md z!{dwSRkvzShaiozb{VW_4+aJ6JjzYl*~I3rS_)X9m7ZJemu%hpkP}b(sjpZ z-QV52gnfUEoDAQnfHW<~{}K&Hwuo_HCW0jM(tSU-bU6G_Ez0Fwf=0vr=v({4gUvRz z>Ux=)ADecv5kR-JYI5n^eR~Zo1@+eHK%RAl~p!Ur?@+<-JPZj#=iTQWw+_SZ~OH_D~~h5a3etI_gCW1Y!Awg zaDTd1t^wn4co?EtFdVJX?(l0xEr8AB20~RQajK)2?%Gtl30O|_)_yj{dEnXKU>(@c zUPRw0F{TswiQ~&K1W%P@LON`O3+{@0y{|s}ENk-8_4yDL;-X@Ua^D&9B;EE7iGRh# zP01Xb-csnG0)B|l!8DN2{;T1z8Xlsr(C&r2u`HaM6Ub6-{2kL=K99ejjdgT526d`& zL-X69Z|o^+e$(z+zrxZoU3?LpDoOETvgr+H=%RznYw?y8VHCcssYvMy+D;UHIc(SN zTdxA!%Gg<@-7qoc0ynFyCN1RDnsb>?=ZChAp*!?U8x)(-ZOE#7wZP9OBH>vp;>)$LFlnd zTaV^zNH?3WPKOUagWBes{Q>L6{{2#aDxhr{r(LF^uSy#z-)@e*iwz?1HbvLYE7NzX5o} z4L6jeFlksv5 z;wt)9zu4r;WO;cGvrGBEb*_(-o0ie<-pEbE91?3{v-7Al$B?Evlj8?|BkTMz`ps!= zGxWDISmXB&RCCL zy@tgl@7JHCmu+`w_ZYq}Pe&A@ZcKo4*h?3lJ;h&-ed~{XcddW=+#l>t-xVSUIz8k* z_VZwg6r-OIy>vU;HndY4EG@;#_{2?f{Nh_EF@rvts_d-_}*3vDWwj)PtiFa2I@#Rr)JPoGcRBP_bnHpsty6xT>Uw(1?pI_`=S}$9ZS;0O$VasIkX?Ke&0>4#-M@J!Cg@dvxRTey8`*IQ ze5`m@n)SK*kMI({Yyn0#Roa75Fv5_QU(Ubbb_;{a7KU5kozo5X zZ~TpHUSI6-Em!ho0*jDk8=288?#ydZHWAM)l@yo_dccomuGvYb1!Tb5c_pI0SL;ow~&HG1u|K_taP`+@8CJKLT7+7XCaI7dEy$gk+V)uK=aCg~o?^WG-14R#AhwV|#bv?yoiEeiHz5UgO@0H`dl&#Bm=?^%8r6caG z--wapU4$?K;plYTKj8_cU91aUEslplHfuZAwzH!OKsuwW$#ssS$fQ+1y^q#xU%HUGwXxogPEreA=pZve*!6wceg=TLnPy1p7>! zMwy@(;Op*oFd=3)D8p)XLQ-Cy2=`mvL?E;sx2y}si7n|C(Rn2FD>!K4avUGkGIGQA z$0wg_(bKJ6IsEa7Y`kvp{`f?Ga)kctzQaV(eZpx&>(kfo;kW@{N|gRVGN>Q z>i6g{jM5K3A+Sl=Ft;+K!D?)G*j>W}aXXRQ@IzR-sVjFs8~@KyTByIgJnx&9Y@ObP z?Z>|SGuffkH`5-V%h(jHYb^o+8Y`N1(nreaNX2b^B1+c{0;1>pe82T61#gMm-D~SF zVgG&Qw#b)vDk&dn&0h2$zWDv}{tnGj&P3h#Jz94OPbhO4dGPAxxCglP?cq;h!BN^{ z-KuLcu_`n`05t`l*Yls(2QlPhX+YDCph>UK7l@ji35t}Xo6+M_(3YkvbPRs~#TS>d zq004@{`fn0GGEWnhRaCtZO37F=eClw(oy{F>j445Pw>HBdx;qI);U|&+aIOPW-OF@ z8A9>u#;u+0aeP2TQ9G+9ku$^sH#MXy#Z9^CkSN-3p5lYzg4@rXcdj;vOo?-YF^TER z%9b#GC>PZ<^JkEm_P+DSg~s8`7b;Afoi+pd)4xc%cRr@RyBi||ytGe0tgGX_U z4$sGtw>XMy2djQ3AK(rshwYqO-Yx$lElY)UMK4cxr^E4daLLvidr3<(NC#lu>6+Ia z`DWrL-N6Yz+I#og_w7Lg!@^@QqwA}o*%;=!zr#ncd)tF#A;WmgNso%hfIeQ z|J+AGrYUkPi?AC9IA4wqRHh8Tw(~LjeeMiMJ zq_N|5KT91aAky+1jf+{WQ9kPSW!gM1LoU@F1V}4~V_~_)(xFl+m$XvKFW7InF zfcz$mjz2yT!{+$MC-#XZVZqO3zr^qVBbMva=nw6Fn#MNUUT=>8gS^$1D=d~Fdv)61 z$*=K-@(ds(bWS9Y1@&&ib*F}azm2%wd)-{kc=SOFbv-`sw&KYAUO%jRRs@L`@50(U z`ALGWo2T)=Q*nu1??fY7{H=+)TWkj!kR2535WU>5)Ya*)k#-rExr>DA^meg2cl?Ep zY0uq~5nBjF+<8_mZCK^;{>uAe)>9cv-FClqw!(Hguvne@VS9HEr4Jt)X%X}hYxT>n zy*t^LwEft2@HDqwFiMIljlMg|EC%XwSj!TAkS*bp7RQs53c|>EYg1N@DxHiO%ePJ^ zO;@H)rFVLixaSVDQNfIzWDv6&1U&AtaS4({af{i z4gQI(7@t*sVRx|)dO7>p-q3@NA{ws5esX&?C{@m2CB503>i_t)A}0SHyal>h%^&5B zqkqdDfW-JNcI`l|J( zmBEFOAsd5{W4!gZ(i#cqgBp%2gIky+>%=4UB0VlR2sFWtnhxK>#FW2?#~rK- zF$RajogFb@@%Zm3Z|N40j+>36<%0YD%8)&m7Fyopd*I-m43gj7+)PIrdf$sC2Ted{w7@E)%GyNMI8>nyzDQ@lY7rxt^hry;!~ zE4|%>2%z3Xr$ZRUZWe#WPJY_qKj2YMJc&<1M2ji#UUt9-91+2-Qp5Si1siaLs*|qo zy|bzN`=G~m={i`a8s2w?<5o6h6o~FWW&tn`ZwgMhT`QFS-iKG+2yZ_8EC=XSNIBbr zc-4Wd3VXoQAN9qpkt)*+k8gEw?w$V)qX2YZ{ zOv$k}%ZNYBTX1h6OULgeo@46p5qs@3Ctvj{wDY7s3d3>wTDHZxJ>E~i4}ZRM`@p|H zA^dq|xQpq&@q5|oKm7A%wwLJ6A;-edhd?Tp>lb2CuOXUa^D3#2_m-FrYt#5pyx%YL zLq6jKC+G%GfrNRm9Wk3@%&4*$|5Vkn%pbSX{EJ89Jf{rt%v*3@0pSI?>xX}oH^JU0 zr{vlV4)~wtt6eF`WyA`A4=jW|+7teCXHmYFVr);c2j2hnmVbDty)!m(tS`jNP0UL_ zJ7oDH4X$mK{s~DsHz*F^Vw{JQgKvtc36R5p2}Ne=$K853*!hLhnBg$U|E$KN1TI=FLvl_L`dwlbt5sbaoA9#<0qVU zC-*^aF-dMu`)#j(QAGIZN0^IuAe%!DD@xPo8(CfM9hz!pFnX?~gPqpn2#Mb#4jyFF zzCUetcC;clr`gWW+Wl5+iS68IJvcxKaK2UD_yWsZ(xb~|RDYwes>k(v*t)N9_B|@TF)7?g*{85G^!Hzk&?)Wyt+^d7C_eG)c~p~)WCl7Y%Vak~{&M9>)ZXcg{^5jx zhs$3)B6O`i^2r^UTX*u%^>CaZ2EX+&$KQ5h8183Swn`2XhRY*#^`!2<4x4(alqcOCEtXPHOp_;_1ub`P{N?U$?tq-N}<) z$^F5N-1+6R)6l29*QYvrUk^wDPqMsoi2WG2zL8Pexv!JhIXW@q#--~rCC*{f?qa3M zspik4q;x;VF&pYm5bkn9cu{HB@thif=I|=B;HO_IW3+FAu5$U>9V&TN$H+kG44MQ3NY_823ADsX_P&0>1pZjBD`={GXbevtiC$p*ia!Gif2y%s<8 z2msvKy!7BQ96;nb!rFSd54wI+EZ11K@7nvwqn_;Q0#1}jO|aA!y)*db^ptVG8s~!x zD!IHU_Nd8J(c`<$Vogq|a<^_g7Q)fSVpE2TN)rkq~l&g5;7jGEz5myqx`<;J>pTb_4EGVw;Jq|Vfm<2 zs=H=Tzm*q2!O?CvIP8aCF*Y?dX>u7jGvA!K_rw4cgX1NB>kB=^6+?MWW~cfJSt1Klt9Bsn$xX< zq_NCM_wcxApw%PQL<6-38WuFFxZS!EgoZ|UR6f^7)R$fn30iHxIiH*oHUvp zJCFTbdu{*lC#-~+eijEm0|YglBw%ujPj-4maxwFxF{gV&K6$`|pluyotEGwzNNnU> ze`XH9CYYR8bzUyYT?deQs_I{X4+UQHZ)C~$24|;-fBC0>L1)7K)N~t*_$+4e#Drg6 z)Tl*!A^uo(&!Dh{uFz2WmV0G9uO=UJM0IaBaFT;!10Z}cmZhOc?cBJ@5|NcR8lC0I zZ}?tQW}0%%v`j4p1?zno54o@G zWjg`mTjmHK*!2}zoj7?OUYDs$72D6>JQY`9`q2ALQGERQ8@D(uXDVizzTu9*?fH%b zu%WUsI>N}>)djL=tR%%MT?S`o=bk%(ClS7P=-7?y!%ZDD&3*5Ad2=IHd)xq2!)&F}AnIstfm=Lj^Z6@yt+EaJ6&$`STgVG|+=79%nhRET8u#I-*s z4LfS>7Ul54w*N2}U!F~QX}`VAnRP`Bc<=EK3wYbVtsK#IP4k`Kc~#m|Fn-NpDQQ*| zu#}&QdXARW`+5lEu4&N?>wi!Uw09}a`54mZO}^y+Rs6sA#j+}EM=uT!EkrSlAZ`m$ zJbl1)S{>on%&ZwjK4M)ASq!t&!7=P1IYYYhq5^UQ!uOaAvTQ;EEhenJTGl1xRW7IB zD}V4B({hG%QmOBcXl*I)xP~;-(05qovSxt|;d1T(k4sTh8cRhCA>Z7|q-DL!%@~u% z0OT0)tqIGQEF?)sDpsA~?8~;j0v2Pfb_IhBJcg97a$KvNmNv63Yd9&JfNn}9D0h<9 z93)96fFai<(px0jfiN!SH*Q$Slu;-zi|>0Uhl~udX2jn-|0MsFcYL6#W73^wE-EIG zn+^~z^%x1Mwu)abgrjZCd%`;?>7P`chr4vqcX*luH5q_eaJMaq;_{ciJLI*BvELSA zJzWiLt(fvN*CzBinFPrVHqG zn$T`kyY^B%6aak1L~L6|twrI&M3_yqRs%X zny8ou3{}R0IYfO9bnsx9ot??Ji~E-Kc$5tW$4KnVMT=Qs3KmyWIJd>ZZ38A{rujqT zXZ9xqbDsQA-u3o=di*l4rXXO_Pl|d!;7JXmXHl3S9n21?3D!`tfbeo1FqCK-i$`5- zn#rxWi=V|YP6~W|#67O~IG=pfxZiO|>w?y=y*E8)-8F%R*jM$UqG9n2J2H(v9*KT? zMPhq7PvKh|hE-c$uk$s1OqiiB!6ypd{;k6Bd2}4KyM1Mb5Chn~DK%k;y!T?{T`X}X zFA}_1&{7&znnD^uo6!7S6$OkXcgH%RxAp-aoF5-HiT!hO*mRryuhY(RteiE&^~6A z5vd?P#tmGmC5n8-(csCqG4}REJY>3O@t+cT7;Z(EHgcoS#j@GgqjwtP#vD5mc}oj} z(s&jnND+t(=1MdFNlMGcfc(eV*gt?55K zT7Z=yQ_?#+w9=ey41#RkvOW2&)&e3ZP#N76llt*bV#x2kySUzGwT=NzU;$`E6poZ= zU&jf%nG~Xqm@~^)oebkPqt}4$(LQ4e+%ZiMFfDY!Et9LhuH167lG9y7=uczDQqk>y zuZTRnWD@xf3E1&`Q}lr|l1=J)ASDpBK*GjCaUsuOC~B_y z1lmHHap;3lD^jmbxIGAPetN9uGStssz?DbI+2j@mlh08Om|(au4`5C?po%#;M5?eE zo);-h=eEb*e`2>xe!pSKX(?85>f3~hF~$ouHXa^kHSeeABb_4_y1s$q(nrc`CH5ic zAgqXlK!>z{OOo4`5g(s2{ZTwL=Byn4`H$izot&Nj`Hwg2eHylz)hhsl5I~COC+D`K zPe;KGFf*8V4024Bi1#bnHk2yj@hl158dHmJMj?2>Q;vpVjCg66pB6p@^ihVbMK6|;WpV_ornm!>t=bl}(eS;t6Apnw@&0g*bi&_1f8X1C z|M;arLBF`kgpA7H+QWE1olBsO2no`HZ;T5@WoNU#&l#NURQ$a_+avK}J$9Ebt$a`fsp3|x9xR5% zUDkas9-H0HqlA%}LJmvqs3N66*uG{DCqgFYCo#0Yh&}PLY$nLMPoF#|m&h!Zk#$Hn z@YA!C{d5G1%0QIwO$q2SBM**}c(;-V#Hg_?1#W^W3i-eD{VYUL(p2cvQl>>t_v4?i z93HDlCgA91)s#uQ}#c>h^Uu*J5z8M~fsB|V5WwaSkK&$waw;7;2Q@yUW%g;Pt zR*(O>EkT>gdNivHWsXN-OZ_4wW8zKE_zaQ{8{wJZJXxrQIytC+Sm!{f6E&Sw(9!r8 zi@zK{VP&dv1*thw^SBrX0g%C4W7qV1g29Gy(JG2CR;GkXB@|A!0`;b$mj)dMvu3fR zA#7oC3%`ZWVL-@$W`);}SiIkyGu$v2yESBksk~)LMse5ZoW&#fwiE*+2Z2ZZvHJYb zhFy3lkk+i`1~T12>ta@i5^LQzOr_2fnk$XIm(G7NQ&Y`iIUZqVIozU}hRUxd9%|I@ zW(l6>jrVf1^5`(QEF=(H`~KwM)c2-|UCRO>0iSS}4Nh!34b0ZR+=9JxNN#B)W)h=Z zo;Y01VK6(oFP%J7FPJ>&d5FoO^6`m07@m@gkFLOGT`2>b(pVwH)TsI)HZ;M@_t2)L zCL&Sn*8-bkn(b<*`r?~lbN1n1O{o@pdmmo@wD0Uz4IPaZ1EJHC{pFwkZ4|iJ zRI@wDR)cB-Id$_8SBm;=j0qkc#toQEyI61+^ZwuG#WeSAa#82wR;9DTeOUqpG@ZNy ztF@Dag6^yXba@Jr$fZ{ZWW*9%Ijy(VJtH}?lcsr{mW8YdRq!Y%;N)r}6SX6gJOv2r z#@thGnvm$XgWW)`A>@>Wo2`B9S$G+ll~>oVUT48)90lh`AN)JI@)0I6IzEiio8i#f zh+*rd9BLQd_Dhk7)RO z3ytY6O3SU>+_0JxI4b*r8sV}RM91`ofBxg2|L;qCftTL#Re>*h>RXIZUiF5u>57fT zmLb3BSWvDczwM8WUqTk!-!3U3V|c9rSX@Z}+1RjALOhn=*l}VFR?^%r9D8~@^;fw7 z$Y4QBS^G8O5GJ|Z4?Y?Y5s-So94jv?z|1a-ISQVUrjY$g%1}!tj7Wml+Z&xyTI@5# zf(&xLP~+Ji6Xvd2p@@cwTam@6%>^9?X`Ne>bI^Oo4C6eZ09DRk@;%(hzoK7>;yV`I zprl+b939j57y!it-$Moqy30abL$GM2ly;uT47AZK0dLyWj54r5853)|A&TT+@KHw) zlJY3$_W6`1k7cIj4@^@!*7a{oC&#GxohGNni$TtnY+l?^T!gbzBTgM11>#gqF+n|v zjZX}bhvY^k)`vsAuvRwFd}_7f0jJt_8*I*=$o`@cgj^%j+CiIzw}e~)W1bQGAs?=} zy;5t>j4%NkbGuq#cnC#3`FLM8MIQx2rj9!4@ZMf5&l_tc8UjqApqLMJlG)QUpP<8p zSvI(anmG?SqMQg^Sz;%P5K(e=0!Bc+$LQdsHO(5-8y>pjKRoxD{9D!-(_qK)qm3b; z1)@!x0XDOoInN+pEM)X{GUqz!PwXzMzmmrfv(e$9{o?FAPKo06F82c1YJis=Gu2zVb zabR*%)^zK^@RH<}5x1?g7Lg*axQ8%BiQVDh$cB-=fAMJ|mY^Kq(V-3C>_i8kdDNo# z=xXcEkI9S zS#DZSLKYTTeFH!R4p5J%n-2HY*IpWhCO@2OZo;C}h)ZylVRnBe2AMk&=4AUgsH-!w zSstB+y`WGp{#PVwV-NcM2B9KlyVH5;?lF;Wu=A&&|4{q&FaPuZ>YIEx^80sD3QbWo z%JtoAP+RF4nj z6qHLlZK7V0=OA!VhXNsV1Mq_qyvq9m1t%6h6mQIXc%nS9BonURCG(nVIYclD$Mt5l zMocBYd!RE#(e)v*=RoLIv(6@&R=6Zb!-JD>9L^4cq|lH%)5YMaG&%#dI`KPMfZ+%x z0)Hxh4F(5?$&Uwzy=TS55Z7m?=QgLtH!;-lxK}#dt5zEXQS`y^!RP$=e zP<`vY24arn$dEDyx6~h(1hN#pAH-3g&niaiJW`l-k}pWwx>szZsqN^YMZ<>fNz;I9 zzjWCow@cSfzrL&1VXy1I@ly!;G#4ThFg)BcsCi#87!^ojwkNRsVkgC1I3Q~%JPAYE zcR#aG!lbni+N-nm@=|jObVN|eb29QrnVxMAM_aZBb{WIBP(6#hhXw6fx6KNHU2<=) zLb;$oewUyT&9kqV0=}TvJP=?IT3;N0^M8Lc*#*9h-Sld=l zkC_IgH3zDCLSt4@=F&fbZn$@l_!8(hz$#;k)7=n*o%2TM@eV=Y>FZoL2rL_3G= zA=sQDz|l%}IfHv79m^9d06;LuL0RU2wy*00287zAuuxaKwaOz}m^(`%Oz@#(k9*`7F@(bIcbqfoB^%t}=nsTv{ebD-tc{%2# ziccU(P}@Tl#8#KnTxG?{(pAa;1Xg}q1Tto1#J+z^43Xe7E4({0G?h%oAQO^L_ z-;`vg`H8I^-l6o%ra6D;|5P$M(!WA^0VyP!M(H0+M$;djPBltpvtqiRkwMSsv;)na zAS*nic{Vumu-f1#iCt;7>zM2i+Ipv(IXgeXTYi5d5>_L#V-b82WHG>^GDM8*=M zW*E(ej6%LEPSq!yAZ8zkB~3(yh1DB)DT(qo@xYOKD{l}15+MnWnu)q!^Ldy>OSaB) zDy)npOzsgA-POwF7R$R*POHkotuuC&qiM#+-n%)8pHoK0C^}l^{JI-8w0Bbs*zC5I z=+ssl-NjjI)?asJC*!4{WWt5jt`fIug;rSc>DF8+BvRJ)5Qdu^C_kMQ>+iI8!b#i+ z2#7VA5o~#RsFqZX(@P@5NqU42oWOe4O``_BVoegu4?Nov(><{kA{8QOwO7)(j~yU_ z6K1<8JowU!yHia%JUkDIGMJDa5T2q)r&h3|?>X5F!ROS&l&dw0s2eKDri9eSd|TX9 zWEEu9TriUGBj~Ks=q|~P)xUR>itl8#0Sdh-2|SvxBq}rV9T=5Wns0B}J>bG)Kiuwv zuid56c`blnpW#oHD#_QZckyXW()w6$?nm&ZDdOJERvU&}8Q5!s;UL=n9t7VtK_@3) z`JQ+R1|Ns<3JZqn)J4}9j5Qubd%K;&XM~{LqAxie!7X%CHGcV|*@XxokF5=-m~5aX z4c~{ZOr#&uF1mH2wo^yphh}Vn2{eH32|Q8zE%(|Uy_Ze)Rael2GBHDn>GuHu@9^xv z6QDOK7obj6aSjG2ymc9VWKe8W8_kw5ZC*w+gc%2=1*DTaE=)qu^{w){&0L=;0%U1n z;H}#m8#e-ttTk;q?{)>weeJZ8KGIIXdsrzi^UdVe-fpv6fn-msR@VgL$RZ+k%9>nL z?atA<>Al{3l8p+E7fbjsoE(aWl$A8O-B!S5_nn`jFwO14t!48uMISOWi?f9cyClGD zvuJ;A-7l=aR#ft~P%h1ep{qIz8^cG*g1`9gnU_!@wyw-nnna77Q9C9~mo6NZ*zayr zueUZxV#QpNJc=2qrb#N>TQ~5b{!X+4XM*)4SwL~!mBl>(Kc#jQWI&vpRcOV~WF446 z`VPQ6H*3JwzrxPyqjYEQ@egYx%k{}D9LH47iy59fy~CYCfryC&FCJ<_-eK3fb*ycB zLwRFH@FBlv27ovCiZ3%8M1B1omH8{=R@tR&+v&S(SL6q!NjE^8C)9JIovUuV&0GOP zk#QyZm+5`XAeiU(s%#Q0)wB?8hHPGpu6A4p+?p{>}i=3gk3G1vMsG%eNFqZ>Qlj5}?- zwmW?N)qZ2G;kI7a_q7(WuWnJc)_I=l-B-4T>gidZu!r9DdZSxydmR52ozgztmSZ(a z^v4l^@;7%(^PQk|C#-gNxG$y`-!5Va0&>r?21Z-B~VpQ{prtm3Iuea60EOCK@&kma* zyxClrPh}K-A;-PuMxk9%PKqsJ2A!!FD8GIxhE+agIG|~tCZz;Wt`7xIy?*?%`h`X+ z(U-Km@6jkb0}YUyMoPVkpRU3JgoN@O%(qqA+*r$W?+OvY^Y%Mf1=1vEWoxyT9r2*o z+>|Wm+Yp#Hu5!mm7V?8?6$AZY+u5yP!Fp`4Srt@1SVc~)Ec4o`Dop9HFfRQh-G?r) zt-gZs7Xkt`m?v)5jD@7{Wjc9jPCzIq+r{?BZ63e$_AcLFi%F-#9q_AUltJ_`7)n*a7GD-eTLQ7Ww-2H3BlSWeP%utjl5UBw&B^Geh3>wjB)o~=^Mfa|JO{%pOqH&16)Ql?U zR&wS^1?&692ECTc#q%Mv$6E5wNCIhj7Wv9BbIt>SF@oX?5$LIl#j>VQg=Y<8uGRb2 z2sf84my&Oi6vDxXatuI0JkMN_Q#QbU4P!Fa?_fwlN*JP*M6?eiIcP#v9^t3*fSGoa zj&vV2!sccI=*+TR0a+?2&e*-F&C3}3XSgREs+T-$609i2(yD}$|0v>=$+vORu7WYw zgWfy*Rncj>ubgh1u8bEi0(Nr+}#5>=j$~?nqLKTIPYl*cbg-E${d9H-@8G%OZk4-~-+|zO%-= z6}@RtABTb`E+gEh7Ws8MV%nKJ(mH#COfGkf?=hN0b)Yg=pYMo3Qx1u{A&%|F#T}g* z#&gJ#pf-%PRdx$N@eXtb8O3dLRx{KGT5Wu#3#M|Zub5ErpJq$-B|=T+tp;FI2@x~G zJnqaeO&GM(=?cZ8K+XZct(B=sO{x|@BI@FAiTej%eydp3+dH3*K+l1Hdo>L;C*VM_ zZ8QmvOnWDVbv?5ubu`MTvUcJrrSiF}%FyaZ!}g)937rPGVX$_snZ#i@pe z;dk9~<)cupEHf}TOYd*X7WE)mNR>}t=_~o85BF7{ilG&0lx;hZH}YIYvJ|u{;mKAy z3j~XR?Iv-j%ocMn#DELHl*@{g%%N>l0MkSZhBHc_z_L-!p&$eX=Vyl@z>9bd`_{mv zfx1E>GMu^L!HDb8RLj2nUq3w4M+N=|5gVgXI%Sa>H(u`q1BP!(ZTd2VD>Se2BNMvwrb<<@sRenfLGsAQd`le2X=j3=E&f{kHTgSyyZ?q-v z41QoGFbT-fe0D85KRD?o#{8jnGSwz~=s`fY<70|nN{1E76t=F7g>Ns4aqX$YbLFq#C%Hi29a9LVEF;b^sfo6XUe_#_9-p0s320m&odpy;bM~6~!3>8K z61d<(?-rQ*sRhw3IBA!m#EKJ_q=x7%<|RDi*#4>$*0u5&+6_Tepa@r4EF3?1LaO^2 zt-iW@4=pE-@t?SG8eiR&IOQDJ?MZa$AR1D%gK)vY(Z*jI=oFHJjpvZNr(DyBg7LCe z^1_m|xIN~UH%p~P5Is!7kP4HlJ-TkSwpcu|h1IU-lomgK^Q<)6Np1dGsTtVy8hwjl zF821GO#P}57mh`tPxh^d+*{{ZDZ4u@z%it_5mQ(vXWzB?gYtB0)n8;s4qxfKg)nyK z!KavQMM`7^+I{Xkg4Z5Hn$a#T6#q-_*y`@_^EYkLEHk786Q{nKEG--(KSJ-p&2PPz zT<(pjNxBf5!dn+uanQ7@S~FoQXLrlu6GLbCT;%XCj;737Mf7&I$tnMzX_xO5%6IMj?iJG z+dGLQY+sIMP3VW9=2e!uid9IAtJgA^c*kEelZ|tb{J^%OjJl zIwMlQB=)}_qo%hv({?FvI~HR$YD{d&^pKKH%ni~*g)Uo;4>6bOzRv>r#tE{uIH1l~ zCHkU8PT>^!Xpy=w8nwzRi&relPb_y=^pKw0o0NJTi9-$Qq1}?V*?@*NZ`fH2wMXeg zXa)=jniAS0?q$c(`be`JjvXmc{Ts?W@y;fGRO9> zeO~v5$-hrz?Ohj5f1dY$tmb9W+xri91sO4{+Nqha;VDiF9c6PP+>Vb2%wuJ;FQVjx zKM1xr6GA}-i3kdPeq}iF587A>{BgH{){TuZ06xVKGogrz!#C0~cnHD3Bcm=cXDP{V{{ipJPKOlBiHRwv*kRvI^;>`a1fzLSbt8PY0g?8+e@mJvQCXu;AXX$ zEYdCW`9?dI>Gt{gn<;FzWh{V}UEe}wsW-sZ8Wc5nfo7E(nIm06rZ(l0136UMIgoK) zF|@K^NJJtzS`K)mY>T8csNF-)qeI8TMBNdwB~}Kz87`S$-6%AZE~Pnvc&>Va>sSmU zQp<{945~%y>s}q;RVvvmyWe7$OVq8Mxm9JFb6#nW!z5r}3o%s?e{(C_wvVDoMm~_w zqJhK)b&J%%^t6Rl!+FE=P_~j(6SWZ;-eL1;QO}XKd6Q4;Qfs*1lkb~AT(p}Z(YHst z_fEE0a`w^3r)Z1ko5D9-ODNWP|hJz*`B$co^%J zt?V3RIXnysM4DS8BSQPwnYAfD2L!^j&RNElbPkEZ-PmI^x+j5(apqe`(K7g zaJ-ix)xntx)>E8Z0&vZruw;h=ie`ij`#I)5h8?WM4{op+2Gv>1!lf3i(UVO>%@C#f zsv+EPb2Hy)tdW}5m5x&|9)#dVvTi&k%JW7Z(h$A5&0#lug&aR6v)dw&^bk*AjvNe+ zv*FQ+kwl&=bL8;oFgqCzjE`;@a+ z58LjUM*qz_098P$zYQ8sLW6#Iuq=()sij8|E|@-nT-TmkNqO7P-`s4L6Gth?y?*{C z>-=2n7OiN(9f^FKx+UZMx z%Y@llcW%q%a3N|4ykrrcQG4*DQVN>p6=j#{qu* z%5h&>zXkO&W)C2)5V>}It`~mT)F0uUo)5C&`Kf+}Rt+fbV`jiF7@f)8&*Eb*fM=WJ zPFy5oJdR&plTzs_pFk_e6CH)GM#*F(=ZD$x;ZZU}zc?8u!_(~W^z3h#j3b%)(dhhd znT?Zkx$>+%9V2A7M~A;K9|IAHqm$9sCPZ|wc-DVyLR7@9qhOIqmVN)Umd)41J*ZPG zr)Y5AX;%=gXeXW8`)g0Tb-K>5*un5HJ32Y}%Ck~Z!PA_v&?Z_9Wii<$PgCNK#jDSZFv%O-dS;07J-bG_rSEJV9ZO}00Moe3a*uH{I z0ze1SGj~PcS8-UpY<24|E{00ADv6nAE*ojDy;m>ZU18saX|yUHyAb$GGODD-2XZn8 zX@+`BivHr3>uJ#Y#<~)qgFGl_auiGeE$=heR+aj4B1?Ba_^bD{t`#D8!VhzR-`FZ4 zXQliw{pQEw9*0iZ!_qIDAIX-%%fBwZ5_oaP>WH_Bj;&2|N33GkW;>6B9R$6A_UEbw z*DD%D!#~jXGi}+0X<4fbwB@QR87Rl{L2Po7n+Ls%We=2u zQ~c66Sa*?3-#sxFmD_i}XYOsLgFFz45-wcPjBY{&r%tts#nDk47BH4?kt{U|v1vKw z0n%F-h9xYP0Xl{zU`0j1VPH{opc8hSpo@ItTspP}wch_^mn6?x6u+G-cS2G)5VjN8 z`MXy4bRD~O+_4S%%d)roW@J7dp0R5|){;|%-z}5$Fq#;(zKdBq$yNfI%sIZi6(v-hqQ~>lZgghzR)8(Wroz$dJwshTrPaFE?l{z6o@Xn>7m3;G^xG z*mJfo8rYkMi}x|ldduxob!gw@vzke~*p~4_cS}N79?MpD@hN*RXIrIWv6@#8g^C8* z=Z6trHar-H>$S^zkbiot#4}kv{ySTXJ>h*yBE-hnKgcYE%$*Sdniy5_)+qZoqU*GU zBDyvVDU1AtWv~X!cKrLx3mO5Q>kV*n0^M)GR%+}H`ch@=n~8M~Z8KeCZBK|>_M^it z!f^NGMPSyT^9!V~cj~wHya2ynX;EtktEFXL;Bf9vtuD;Y6DlX5g^g?tcAdbH_6>Ay zw@yw_km~y#08=3N@p_dl)tDfy*h>g44{xj0Dx@sI*s?C>jY&W;FgLZ@n4vFAyT|-Z z&~%ZXAldF#VegZW4FBbgG&nlS(7`m>KYox#QTeg| z*Yp>pYO-c>QCyUcL^U*zxr@$Y5Dr|{dkf;C`9oiBtYI)d=yicaeubXrh*z%_{lGAH zQzmT%9HHHv!(2;%k_m%pTp5-?FG|Zc9b?&lDuY37@&*XXS55Q49Pu0ZTAqMv9ygPe zE4p3=0)9r(j(31}aoCdj$8cl<4Ph}YbDQ3%52-hq*>Ab?2Jv54tJwW>vC3Js3C@&= zhj<}#jbQA{c{P?3De{9Du!R4kT~oVl9~|_q%TH;7|LdoFmnE1viJA3kNeR&Xj_n9& z?^yyJ(fFx7JuTbd+G+)se6wriWZ>XC4l|R<)O=K8na3|kUA%n!vP8yX&VVY}m-W<^ z%cKfLcVfDQw$xSG#3B`Mx|nd(6kezAiyk`17%S3W{`ucdPePj$JBf%q%NN7rQJ^`( zNe-erIq29Ag!#zr5Y$DciiIpOvfIi5=-85Qt~X(0+PkmigS^|-)m4xmXJA87#)yO7 zZ*I{;JMI;OzP3_2MhS({RA9(GXq8p8Mr00@D)sYh*Nrip_8mvVzj8nQ)z^#+_g`+6 z)OM=qw&u3HS$nF_?Q3%6p@ogeqfOCQjx-Z)ei3aY6O$q|K19>c3iIUWIL;kAYL8S& zA}h)XWbkUu|IdrMVtO>|73;~te!A(xjL?U5mhr7k*0UOf)|u^`v_$QxA#C`?^q_4Y z^E9_F!w63{>1GGKtCTFpfvhq~4^zCObzOUz*K$znZ0n$}6&c17bgkv=?mu5l#KIW5 zFyvL+R8JLY*PvBb^TG)wpcnM$tL6x;{<<*j{hz&8CvaD!}s%s2h z)=JK{kM8Ghvb-ii?#H;7z;%5)JQGbYmE)+gwR+<&Ec z)}_quY8}YMP6DDvWlHXZR4B&3?bf_)MMGmE(kN*=KS=O5iYoc-<9|T`vRf}_-2m)g zd!S0yYnhCERn@w4et-b{RRz4zwRw&&R2PSRE&6{mMuS)-6hk}0%}RL!<4f}a5btjN z>SZWAC3A;|Na6-lRW916Xf;9OizJk?eu{Qj={MS@X>_VKzYn-bj9xhP*_NlXb_`qJ zOGrg_!&ozqaz&}mzW%hvrc(^5WlLEo`0z%~@;m_iwb`vH=Ye@z?*92x)943i*Tm2$+V_ry zZqa<+?K~tb3t>gcyvf21(MK)--pa!7d`z-i7$0zJl5snF`sr1+NTdGpRPBJEy(gAfreaDTdkYA5x#jHv6A9M*Ccd3lOg~!i<**jADDEz!D z!IP0{9+45|35Wr=2>!X&gNr;^SjlL&8oaH4tYm^pXR^@(q_l)2ZIxvx@zZ1O_V#_k ziC9!=zpTU;Sc@}O!hROp9@vW+yN+|q&8o`l^~IGg^u#m?3+xUX4{SWa^hv~y?eeLg zN>=(8HiCK+rhS%JLqWvhaYoFIO^}PR~-P}#?o)(Ztr8!*lU{GaBhDwQ>RBJ&3hp09ICyd;wGEGbh$9q2Pmm zWeH8c;$kyj=Hgb%QS7s3FpMqt#?>5T8kumH*5L6iLG0&6og|(M&%j1cwHIy^4nP~d z5=xq4VEO%KIV&3tuWD>G>pqR&t?G)kB&I;dh1EvyHjrDpIRGphv}2ikM303+T{T*? zv=_xv=_mG=<)?iMS!b^H267t>s$MlDy`+#B_>IXX4-q6Kk;|6w3I$$@F-`P%x6osi z5?V$)oSVh*bhG16i!ZnIK)GlRH?iVu#MYNNZs($5K=u= zSpnCYxUoe}TGv>DEqQepMupp+2gO)NVDo; zYQ1*!Ez>){T6)>H)*+NzQ>O#lRj3^kJ-_@6Y}w5+aY|~_+LlApu^-`hgAWR`O_;Gb z{MKvplRv4;k9^ur6mgqTh*Yt;ODJPz)=>N2_z1vckkIOycu**L^3T8`ZoI^6E!yx~Aa`a3dk4oOSD}&yWB7$FRD0bWRZRKyFwoMhByKK?>8>aDt8g zrNN5LY`}b3jd8n3Swl`~MQj4!18eD|;Hu_MF)}*aeuo&!Sa|2X@q?Oy*e>NAj60Jq z0eL+b9t`8HEVAc!42e>Q(Z%V)rwW?J9@WeQA*~aKSFnOuTrf0}( zEBBq8z-jTR>jPTc9KH|>mCAaqhF<(#+Xq!+{``RCGg}+YiV&XW%h|{3Bl>3^oW|AM z*A8Y55cN_8x!d7+yvW6!xXYt+=<>5*Bg;@ayAFY{;lWX|pACJ47R~$}=_4Is6_dhN z#?dDs9HUQ~e(^qX>S2IQZB2A(k_@-vnX-u)Hxq3LAZscPaM%EKZv3>yl3Hsw|3(D( zMs~>Q!RhvN8r1taXpQE*4?Kr^#|Jx3N2k~PlXrEKKcL2Axa+ndj-dTDdRKCC%W1Wd zAMCigwfW@ywHAPx#bI`Md=e%shX2oxSoPG0Zy&z4NUg395+}eRAD#5T)e@}Pa}91e zb%RB~{1)3II8h6N)w2|k*2I7hyD01Na(vL+#?#gW4K&eMRUXg9hOI^)5}|9mrMJ#M zd4m_2!(#2BO-P8pCmE~LgJd$5U`R#SiGvG}$l5frT_9KTSX|u^M`orbje>j4gV^cv zuCX}2tXo~<4EF~LUVc6?UJiTorDltD5rGD9FzVdItfFAhsved)TeN7PanuPKk zT4KN(c(29cm;8QWY-4#!XJ_*0=WjgQaZ#_qB=$=7NlAN5pQU&s!XvaC*UJ*zike2Q zURv(;ddmW62X2y=Yy}u;klTbBPYl4>#A^Z_$0EG`zU^jK$mC75m-J#iy-53mY0FY6yuu)!#hKp8ZYht@am`dx@BX%`?R~>$&pcrT}<-n_Z=>lkJ0zrN`3;Wr3e9|P+Z3GYtwSyixd}1 zDQQL{`vEXE8p+wJ6|Zk5x9H%BTFE&WT}G`kCd*)HyJIcHz}{OFEkiwU;n3x&*`G1R zk_o9OX*P4T3z%(CQmH$v ze<6ROOV9gaS(SAR6o%LhDKRapWBR5eNr<^!>lxCwHf$kp_Lv}sA$xKGW>y?% z`CSI7)_aQ@br54;KP<-8oS(nQH+THUe6Fa}XE&gE#%?^=BOmd_xs zYr5dpmZ5Vl9li|yi=~4qVjAQIjm33iyY3SmLQ-#&IIyb1-N0>H1E$p=3S+%TKCP;z zjHJV0Y6PVkO=vwJhC4pMOLA~NcUQ90&B@ce$=$?W2*!v|yue zPUmZR>*~JwD7S6GZ7*-@(n$E<%PfI(90Pq{R_wMQC$p@pecQN@L^z?jJg}Op$@EeN zDstvx76ihz%C;DMUv9B0-B+HAif*$S&4Dz)OzGf94zp1E+;Jb_922g3nc$(f(7h z+>m?hO*vNvwT&SRZ6SNF4FW-PRJ(X-NKkz@?zQ${T2eZ^y1H%&`a1e;?hcwq@Df~G zNJ*P#kZBxC!FY1%Q#m`|qf7~_rnK%BG4W!`ebGV3#s6>Mu`^CT0QoGOcTlh#Wjz`gx$W>m8+ zcEZFRlByO@wjG*gh}ViMdi+Afq+QsUT{M3ZLbe&A6h}(T-#$x+`)NbO)`>`0vx&>; zzV8bVm-1+-Irr11T(1!bI5{8SNgZ!?)BF2PUiO}fWh9mhIu$a1#(T0VwYofW-}?-Jke8}G4Cez5eHUaI-&7AG)*t_QrsZ*$ z^Ud1P8?}>v~ zSLLVFz5qyqEDUiAI_9hKxm1ZXEgK`3U$>4GcG5+?z`y02*D`~zmNM<7Cl$NVmoP4y zg{)U^mb=*brG!tEZb$Zj074N zTVkLs4qqU?k$cxaP@7oi2Pd!RihmSkeG5}x=60!G!8>^ygB%^XCD5fkje99y*P**k z!02P0l2&aN(662DIPv7$P_y1%LgS*p(hJluG>g`L^E79+q4?gz@GvQra${ZH!{1X| zt2Sag^jLu&cAy6anQ&gu$s-nfcjZRHClE#%Zg1oy-3^+tJ`v}9T3XT+#^RQr3sfjVr zueFhOJcx1YCDfhA5gRVMoErkXV@O6j`${^bhF!E)P2c+&()@9g!Jc(^=8^h&`PTM(rI>_(Z7^ZLXMAbZl9%$+UOj@Slee+{8s& zbPJUwHfeg#O#w4WiH?F`Koq=DIZaz{U4SyM35X$hM@FQ^yP3smKDdqAChii{GNL9~ zNWzNcg1B({Pk7&xv%>>m09-!5OiL-!{IpGJ0Ju2(T~A3+8rPUz@%1JkLM}EjBN;o< zIjNwH+eX~>gWie%S}ptSt@tFM7BNjq`+{i?{pS3HvrTWZjlS<}qBqqHfzj`SMCfk$ zm-*v=7L&K1_DD#TDz<5C5uTJYu!bUf0!~fQRX*cMm(On z9b*e&i50q8*O3gCeG@dcdhWdtbhwrWge#HP7RMWQ>aFDRMzO%sL-26hBqhc zl(}NfZFO(yV-=qP&B7FHkGWl==c{ODJ}qjojB4v)rspbzJBNp1%e4&prOlHT@?re7 zIO0vQx@81J=CnTOy@oZLb>1mjQ_o-?iLdZHOe_OP^8!{{SzvqKX7J;YdCl)28h_kO zAD|quXTvJIgb{Q@2H<>}Y`{yT@v9(&lP6Qw*qS>@p)PxBKP5W=1KY42#^o)n>C)c( z10L`m9C)4c8hkc80`cI!mj9Jgv5d?R+m_SGZa+UdiRbUK&ObkhF@ zP5x`Kr|R4@Umt?P8B>eY<rYfi*;a=9b>T@ku9YKb+A`uf_r=Z zu`DkvpZat8`7_A8AT3yN>e2NO0vsHj9=iOdZ`O*#%NIXZqVku$ABu;ioTvk9*&>%O z({$U|V`;HiDvbi`qZO~I5IO79YJvMSfbXE~u?(cI!s@;*DJ`&atEyNxDN234_Jhv- z>!z2+kE14>huQathBFk1%&bgi<`X<{UQA)*w0b;u?qR7CN`qEg9?d+nv1tr9i(Nct zAo#Kp@1bN2^mUF91-L~D@IlN=2%D_;KA-Rt+5a_8IBMu}<~E&)eu7GgSwxA15PMS9 z7?>hG24Zs?(^Y|mBvN1j#@n(*?S0XJuDnU8!D>$qB-z+Wegn2v%3{XozxOJ}R%ORe z)f|%AV(5L3vBXfhIwu*%xs%2F( z%4WXgLF#SFdqLRzfJPBxW1xRq!3B7xl=jWa`f?$@y{zuqtqJsi86R8xnd^^efVX|z zCki%~w}96#N(0Wxm}N8G6Tqu62@zD!!j8CeD$7bPN+ zfTn#fJGUl>wq>y{^TkF5te5!whPjd7$dtZ%MeTE4E-H_!k7|*~bjaV2 z|C6i{daJ_@z*&;R^}7mzpA{LSYJI^AF#1-#UpLq=SG2Cw*uGa%RNGe!a=fj^Jh|m( zt!oX+0I~*V*+=H|@@3`4S{~HAO_!1-7y(AIDkrIcW-i502kEqjt;i+=fBC0>QM&yO z*=Vl6mwUmcuvU=x7pvo=px5>UXDtc=JtloIA!i%5Ig2{mLNrPPV0f626^^}w1CqHlkR{jc^!xCQrscR*_4oddeK ztb=6+o_WRMvd3IZM3$38R%il#m-B0L`uNO{--k>u+{ekUE{dLKj?-NF1& zk_^YL2WC=jACOi__3-~94r{bRHlI^XHFN0L1EN?P9HfyaVH3!0#^OA0YHzKj@_(be z?fyI?dy*YxlKh~TD0#)L+2tzHUS+<4P}EUEFN%pOr_HPw^dh%Gx^8Hr6s+?6!|iUk zlm^~5*6-oxZxHNuO0K|d#?sVAlL8h%M6h9A%q&nz=R={$th&l#e9=RD#q@!swD96u zp>QRvt_>{2^VnV9aJmhU)#v+bIiz)AtrMq}vifPhLN-AZ-e!)1Sry=-*2>jNCQ~NN zJFCaZzCGhjuzMn2Wfq3R6orRFYU(CGC1SoFdJ`<6xzV(M0St3cPB37Tpe z9(?f0jgv=mZA_8V1{#6}6Og?5p3AKL;B)?!cB&=Rog{PADW zbnNxxmkQ|;F6}5qk)+HKTgLz3*7~!KM+Jv{55SbbK zn#COSv}|6guNXO`Hwt~b`xa>{Ja8Ry^y(GC&JroQOizC+CznhZVm00kW2s%2KWe~c zr8=&vS(<`v#)oCF;Rs__&pFmf`VM9gFw>^77+FL@2DsI;Vl0-jEc5fa{3tUoXO1`W zK*M{nhvy>U%^5NrYo9ceVeDBjh7vf-VwUX%m$Ljn6sSmHzP8s-C;N%i`#fOIki zU)BlOG^knK*<#!Gk;_3>7;5`w{outR3jPf9u8|Hh=vk0qOa8UNs&X2waPjQyP@Ffk zmymxY2b9hd^mdHP5gY&-ghoG969t%!8aXoyfe^WfvObRttG_s|`67Gh`V448(;v^- zlQ^HP%9ZwrVN+&pbDxXgwU_r!Pu0!ZY|Fb6YP$FMM-kjeUfgc#=|g{p|L#XHWbjbO zc?1S(?Nd~2SZYH9b2_yqSk40SoA!oAwT+Gc|LnsH4pl%&WTOa6TxM{yy0R9G0d958zZJuw+V{MqZFg#jWY(DkHRjn3dl85+M zldKdX2fYvQ7p!|}iN4s=&BVb!-1bgmO3@bqB(FCNvOfi0SDuzH0fe0{utKEFeTCpM{71&YH%GNb^(vM2;RGREFq&8<;l*#uG=OEcGq9uVPQ z&F8kf*W18lS%FHFSCO3T7F|XjoG`jneoPVxm%5T7XqM<)VYEuYf~bptA>cYQ|I$vQ zy%nARreZ@-uEh0b$q3f#oT^mQsaG4v_CBpPqDj}-_({uxvDhlXidOmj3 zp4}a$GU-Q*?1lX<>=4eQ-%a)uVlqqj{9u#5GrOc?gm&=dvTUblM>Gk{DaRo+F*qcJ z!E#9!(!35LPRZ?o&kMnxX6=J(5nc7PgRl=)JfcSHVuLh95~kH!W;5=#!;s9Q zg{)N7>^3owmoQ8aeA>i-{i0lg#3B;sT*^u=@(6MkloS_yD@y;(4%?u{o+Nvrx3m=5o$Q&@A^3E#Ad-A9#0qO)R=^gLA+k#i+r&UiQ0 zxW)cQ_7i=H_r<4>Jrv8S_Ste^5$VghtNOEwN1-MfC~sQUjHjVyTIxD6{#K5{=LSVE z@+3L?t%&RoBH$nHt7_Vx70W^yR7eBDNFb4Y8gVmru)#i5UvdU`PZ9?c(hXyO$PXuzn>PMiAc3B+KSHe6>57v{+;hU z8NTlnf4D#6x8=0#X*2EOQ@$WvJoJGfJfEf@Ub~6#6qA(5#_LsLE+HO*DRTF{!?ZKm zpxm1&(~x?!7}(i%x+ml!J5cN{f6r}>ppe}=GX{%-rmg%81RHgsh4~z1y}m(&B<9MI zDxPmX!D5ZKrf&R@-w`5sv7F5nGLs~kPtQ*LM2gQ3YL#d_N01BlP^g+ffE|!*QAA9s zML(gEx#SDpY$(M%ITzm4+c^3M(BdixetP^ug2x>Q8}vUYl13psE?)b_q;;CgPv>lOCI*bP|3h76ZUV-<%t zhRe@c53seJ)qg>a{szTA;lSq_R>F$T_|3YCgN`ew>T7Tg-u)bQxmV&CNv3}26sEMe zGWRjYbAcbZiLXNl0baIRNgK4h%|(=|W@D^?QfDi-Nu}`{W&#^a30adCIq3+x6c%lS z>9`c>Uy=?)&JV^YOa8uqDZycSuV*LmfwyXeaHff^9j$TefTc9TCDXPXBr`*rUSIV# zXT49F8l0lSO{b{FTGk?L#K99Eh8u!?cYAAENr{^NwANmWygz3}u_y_v3FuDf| z72a2isKKKml*!!eDCbp`qW3!Jy{+!!ggNcCp0Lbhu)aK&L9C3A&)iHZ0 z>JhWL72U|qzT*fy44L8q-M#e@85*HY7!*~+v(iq^B_gu(R=X>I9I3Hz z#vf-zF=d%V$roh7^?&T$*dW>iO2L$|8BE6Pf#i+F8-GQ!IrJc-p5G3nAJAW#%Ki zRU%f5X2a#VfdKlN|9K6eFf;V0eq?f@|FmY@?1070MXiU(U($$>4=$$Fe1&WwCYOj%M^RVHC4C{srohH7 z2xR2r`Y-db2+w)%)tFRhrrPdMeQ7Qgus99SK9k$racYQNN4LpI4J@2#{q0G#YDzwk zx-5(PW|gm$v^byTpXoQ!`em1_-0AInYrm$Poib|Ddr{8vy2=2o^-k#goCG-Y-n+2^ zl;r*R@nudDC@A9m@jovC*hWSR3Yg#Pi59U9z>sD#-)#yoD#H1A{KG_CW7l?X^0n-s zy{$eta`AgnGHg2Y-}OE`ei3g}wl$7cv7txdj{p40;~##%0o-A4S2q;JIf!N?k!qne z=+~rLo6O;6Mkc5GYMT!xomihU_r84EJS?t0<*FZ`#;?EKb864QuS5 zl0_Ab26Z}Za=dP*Q6j1k4mukh`&{;hN9NRp(n%c)5_n=z_SBk+@Uxf?wn#_h=d%ti z-c>P1bsRDuU-BW&oS3kyc?DuG1dqS^w6q%jh0*!px)6oGV283MfERRH8qr*;52yZ^ ziH!pf5k*(mr0M7o4fXHJ;-0SBnzAX0UVV5m$Sl`nP=no_eSCs5|JNwvah(76;ge&L z=qI~R)I|*u_UtU{{YK1_w;$x0Z{Ppe17R^!(m2iR&pIcbL*qks+oP=a^u?Rrho5vZ zUTz4{sJ+0tajLl_{-$yNq`t+d)ZXXQvI?&r_PX`+tNHD?uJY-GM9Z;Bk9QiheWVt6 zvO$v#BKK4Fz!pNK{cN@f38p!#h4p1@B*_?pRUWosMA4YACV@0Lo@u$K$Hv1uxoW*^ zM{XO{oP-wEHOy(tk6|(<|W7Cp+u@wnk76o36>sFx>5ZjBS&vW>RW>Ku! z(72qbYv^czuRNFA%MRv^X>=JkXsSr&A$qGuA~ zsEBi2R!iW+3iWDZ8@rqN!-{=`@658+NA#jsFm^i7wd_1+mxwrQdj}Y<{#r^VO3)ef zNR&Fcb8?XGQw~FmPz_#QsOq8KXjH3M#@VwN5ZjgqPZ?rc@D=X5EtE&@8L7TZjRuzO zTUZ0=l?p}ybvI)I{5?%Jh9jN6t1$X^Q4^ry_S`uHwEODgLd@jvdp{!8!Fd!{m?asD zit+CkkN;`yi(>#~ky_-w#~9fX^{se_y{8aX$R#oTyvTd`24VG54b~0rb;UwWEE^WJ zCf++>RHjeZ)nIAG`~F;D^^j?Bf(~OJNfKd+dbAS#IFm_`juBecaG5{zM%g4TWpTD(j}$u8`V(g%>EIjNX-a0AyW>tpTw#({kdimeuc*6KTz3>vi+J9plym zotuG2Re3_fF$4Bt$^)a>^G=b%`KR7!) zOox$c5SgVo1_s@rvTubhk>f z+L1I?V&RGHbV*zfWg?J;zOvM@+4GF{@B(DRbypxTizwd)iyv? zkJ%B#U&?0QBXJuAxiL5xr$2Ijb{U$&{XCTDv^CrcHRUJA@J7}MMNMN!$jdp%chwN= zQ|*w0G3q(p0lRZCU`sCMuzcx zP1XAfo*@aloR46~9D0tGc2zPfSKzG?<-XC}T9jo4sFtTJwFhHp2}XYZQMgKvY2=`M zL*KR~_o(*@z@45I%>-6gv*T8-auHr=T3v+FauB7CH>9_DcAj_8&X(001vI?G2baQi zdoK2KQ+0z`K(!=_0EUh%0BHo6ucicz4hMpA?Lq>vr_9; zJtM(-A8<5#PGbG2G;+QWIbmnL7q4DiSe4}EW<|m820S0eO14H8xXOYZ5pyb#6%Bn-al!kxX;OvXwuDJc+IyCrKVC%>BB^7y1lAp2!5Vo6 z3p`{FqVQsb&&!*#P%qAW0eUr`>h)>-T7+>39SsV0`C!M*?iAfsMt8`m1gwx)@&3B$MxkPA%tun;pdVl#M z=?>Bz?v4z9)p+z;fgywQ;Sqm3Jd}T2(^@sYmWk)~iLRzoRZi1wDTiQqczBlC?*~Kw zVee%QfFGJu%qx9x!W{ z+ZiBbTuvS)VtUjn@!B~^MPeoX+Wa2w?+xNKw=KPVZ6KsmYp7P|6ODe`wbPCd`rQ0O z8R&;%xwJf+`}-}?AA-%gWFbMUgZe%d8}C-&aAAJ2?Fe&7NT8VaNCj1qT<9f25=~?? zu;E-<@6>iqktU4-4zK4T@;tSegaO+JeXVy}GqL}H7@4RM6a{=CW6yhg-dSq?&{O;D z0wq~LXv0RmM&SZ56P&`zFRD{khckDcb^a0cN7loxgQK^9^`6{O%)I%ScVlFGPqo5|)1FP$5)t@8R*WFghjvlnmn8LMgvW8LNr%5@q8R#`C%1cT@rL)TmEEYwIY(~v?KfL{9^hbM#`Mn|WB zX|cs{{dx?Qt{an`5zert)ESdlL z`T;Jt{`KSn!J1l5>}UHZ1yR~ZYJI;(;HB8N&@gMRxNr9SzNY!CSM<^+Mt+-%9bk{P|_{Iu{!VRon#Ub z1KIPR5uua(fdMWz=z{(hqTqK39DI2GJ|RGFA8{EEMHS#>#vmU*aP3Y<5PB_LDUF_m z_JS(fVru@}rO)Vm7*~!}yV>yk=+A%rxajSj9UdJF(3fkUxIzrbz4w5|lT)~vZDP^` zI!`e?a|6w*E8_b`MPV2v#K)|LO|!020y~2Ob7xzz4Lw>G(Gl3 zb9^uyg=PRNl3S1fbSkg&kP`X~PQsj^jw|fuqr)SlaaPs}Zg_Nz{)L#I!LhY6Oy);q z6Y$0)a4_@Nb=uG0gsOu=HlitTt9K?b!^S!~~ z_~2aU40yN93;WK(a@+pCDC?aDNgGDEWsAgP3;#;`4jyv>%D`sV1H>@GwQFz+!girp+r@If17j>T(d?)AMa1GHlr%r z;H(gYJQ(iSCVrSa4-2uZg@ZEaI}6<9K0>~@jOMj=)Dg!Zjy`4s<_~FjJ+fOQZ$@B_ z#vy`PIx#;kgzm&`QUOBaB8BccqP};-E!hMslT^D0zck4nr1Or%)97pm@+k)lZSjAj zY&I^jPcG%97jidDKgv zV7l^|XUa3oNl110R_nJnTAZ(wa}{7mBRz}l_$F$&(lzv8rVMs+YlINmcvaVGUfuU6 zL}Th;LgS-K_K=%6EpiZQl^7N@qbI4{(H};^WcI0dQjkmAaJLA{Z^}h8(8Ky0W@Y1j>MV0I6H|$Egj|zvcJ&E+ocg- zjWz=s<8*#g(RQ%kg!qz|#I6gmHtqmj%p+#zJ}8VKJX+I++0Yl1RDpEbEFkPlVN1h4 zD84kiG(}h?YZ-9Q#wkfpStK%Dot%*4OHef)P4-Wn4ELPN;^OxIo-SRIAUOW!CzP!Tz|y`js}k+0>KYI|ac5y3Wtq`M0AZTe0EkVOTGX2Vy5SM((j$ zcXMz_0pE&rYxohIuuS});N&zYIM!NmI3E_rFu?*soZAPj?i<^AwW2X*B;y_P5TwLy z<<%4uOXEQ+o2GVfyn$vu-8XuWp$VA5S$Mx#$wgf6RNdL&mErhO#(|LzONNQlu2Ndi zWPFR*j5h8-yjUzJ^6<&6raJi2cZDn{fsL1TqX4^?{lp*uYLuolnHX@^x^(r{#dB1Y z8&qgteJcQ)+ox0K9DD%e2Up|5NTz;z{4y`cPps-bl%fu`^RFYT+-FtaP{HQnY7eqA zTqM(^#N#c?=(+i-BB48fYA+qO6G{6an!}N^3=ddvS^1f~I>^nroKDpM0>ye#gto@R z%n#%7A^r3sc~&GOFe%mv0L2F6lgh4TYhn4{NU{HJliQF<8HOU57tjL4PRT1M$^}@ z3W5asK=1DalcitesoF7Mr9Jid)RTjE;^T;gECZE0JG%X8F=xkzthZsO?YFr+sX*g- za1%3HnctgxztoVadD!R*!9+hePK*0qE*8dgGg(`pY|QhvZoNnqtcW(P8h0Ve)GSpB z=uHh8WM8Ehh7^P(($~Qg!(}dY%u;pp4FsQ)a)qb}&lE<=Qa`FbT^nlFV!?1p1dQ1Y zGZ9gXl_TWl-0O^jhil2^5Qpml>SVYv1I{;{qjR0&FhdB-g{pQHHt(!)dOCCY;5Dbe z;`n9ruU7M5A<8YtpAe;UB6G8&XdxY)$9M|YeWr`i6bVx1f~2!^?VFfnFd|caRqE*m zl$oJhuW4K}Q;vn-yLvM~V?8(FjTIo+OuE}gq9Fgj|1$f##mM&U&fClRtRdD!Qi#dQ zt|vCSwEi-Q0{hc!%}hi;|8HzRFs z4-jZ=9)e+$+Sd(_RT*LvU=Q3^=_-GvzmT(8I3(r6rkv|xe93RJz%&ZS?D*s;Y`}Qs z6Ml zolaD;dDD408G5SPdDTT+j`S_bru$qDe_geOVt94aTKdF5=cA{fyFXab>K?q;5uxgb zlGn$CyLMZ?W#JR;qRc?t75X0|G!40CxoLieDS5h)gV*3`n?(O9+R@e_i6s^Pzv~fp~s;x*uoiQcRiKhyGJhNV8={cC9_2G4@;i z4`g84{G4OsaB9?~D@a0{8fi>MhNm`w++{G@_r1c>}wYsOnHPhq|VX%n)Ja|uFcv4Z5o%*-Y|mD*8t4zrnZK<`P37= zW3(Zd_f&5+smrw1YEHb=EhOr1LaXm58slHzW+^in{N`3MesZ`&d9_W8H=2~IUM-gu zVk9zim*HF0{v~}Ml)1{Q$ijdcNXBtXK@?s~-@qA2P)fE~y0Z#mtIVygwWB-&3I7v& zJs($Dv>5S_b)PC;*7}bWA+_5!JhVZv=7%Y9wT?radb!EEuT)atCiSA=qwOjhGZt9h zj3R>NT5;h#;Pyl}>|BzB>{5IIKVHQRG6!$vbc%z45-u4-weDJ{Ppf5)MJuB5oLiOw z^dO~{MO1DbI8-1Rl7`I|p!m5tq)hM5MJ0a-D|*GKTJ$Dcdnokum7Yt($Vnd`DdwnuFS24<^GU z+VdKtqnBt-i^Q?Ye(AC+)DJJa9maq0;{MZWEVB0D7w-Gy)m;kW{mSj0=AsujpTtsB zW5{Zyah?*SO=mGZVDt`%V3hD`6PGPE^B>CvNEqspe16?-eAkfoV(EytK+DF15#slB zJx8@ixL`OSl#+x0_pBQLU#!%A4?R`wWnaTo0jVCNK(S#gJny7ZAhAKb zJCrdEuzX`e53`tVMH1Q=hIA!b@qLA7hjfb{3DV6j*X{aA#z;`F{zax%{vUC+<71EJ zmT>VC5;5H*{kTb7l`wT1l#0balQ#~Sw8$dtwddW7ON$t98bz@?JBN&fgT+{tpjq{F z(_WnJ3sI-kg`0d9Ug~BI!)^sfw>nDN`-r8=_i@P=8+P`oXROT3mtlFs6%d9&Em!wA z3az%|hNt@jh6e=C0wqe<8`Qa#%B^s9{W?8bu!1#`qb6$&VD^(=&qJnryBksqns}6O z?odP;dNN&v*6f!!{6=kz9mo1daoWV$xDpGIf6Iz_0Zow{etnR#@y1uM&->Rok;3U0 z;qb=~#m9&Ybv}^2YyF3~_tk!8$T=#Xy&s-m`s1HpU&|j)J>#s^H2C=PYWveyZ?B(U zJlk)1g$R|SCzB`z5jJOxijnhuz@Nh%&JQ2mn8h^AC^zj>zfeQ=~cHySVEA zaJ5g&svbX)8QR3s!gLIEy6Q@}HrrY&GHtBTYJ%k)#(=eA9ubrhSl+pTOQ#DZBOv~f zr|ak_JUi6ZtP}gLJFo--^J*R2tKY`?jruVQwT>BfJ5c4yR2t;hUtNmhVXRd{IW>bi zP9Pr!?&!&6rj8HJf-*Kh_zFx7vSZo$)ung0DWS9N$9vm^{6G?%{xcC0EhT$$Y;BM& zq!uP@%RaFS2u%Ry%L1{1N&{A}II+$wKhtCIRr>CCzNnXOt z)GOH|k6%O*%vX>9X@fKYM zYZ=({J^vyDc*#&tgvLZzmiGF zm!Jmsug#)^mh{rEVC-VR6nlhBC?}fphQ8yCK3lHt*KbAIzT2nPLS*Bs>?(#ldt2QZ zNB;ToSpwJE$qx4}88bs8&nGbM;O12Q09E7go+Sicd+(lKKHE=Ad%jLz5kU6WD?|*) zU1P3Yd;XqlR?{ztzboY6m>EO*F0foRy$iPbO|7`_3$PgAe1%Kgu)BvY5EoR$)HTI;Ca7i0#V=wFwXn$#E43^5a9o7#qyPYq!4 z{1d02NaByY=$q`n>Gi%DSJQ`nas179z5ghG$_M4ro)A7tKm3PsDxZIIKwS^;hKOeQ zd7mMieu1LYa-v#;_vf$4q5con;NJ z>&MxS%Vim=arNnMx^|i0?YcA>)8O#%+kf=l;bH)9mR7&~U9b0-fBD}-eU^9V)<~3h z!B7Ls9eajT5jvLjUuc{nkelPWVKz7%W+%s_8ZU!$?R1*}2Pz$qf~QCt=*Ga{@E94& zVQ=qf@aI1^zh{oJW0n(qi@vJK@I0~SlH%8~wpczM9TI#(8sH@#&ZGAke&7ODLJB;zFC zG{N7W>tPS1iFh_K)r2v))$O*hMh5d~zFLPxig-GsDkSEVwoRn;ZPoa>o&B1qq2LC_ ziAlWdwoUGPtJTPUXAB4}5ba0r?Sp&UZC;=F)onKzX2a7#c6xr)L;o^>A$RONKEXQX zo`36uY2PVj%18&J35ed3ok`}sm(Q+n^uw_cjR|602T zU2p0;<9Wdj?{;+OQ3xT*r5Ibc{TLI`JI()p_TFVjwj;|D`xQq{nAx~DaO`c?H6oc0 zvXHDyF|%3KKx*JO!Xv_)9`2F;;O*OhfYPugnv@0__N=Q0n$;IH^(FNsT5H>JkGO8@ zmNF9%ga$;^gLDr+53_Cix%S%qN*_VT`@X-@!_>rQ&Hk~PDm@b68nCs#qu= zN=T)uFH$I8KU9rveC^N>54)l^cJehlC$7{I-aU`+mnxRMj`zUU<>^|gc%>=U;ImVY zTN5so#2aEgLTD|It5rNiR6H-_Ftm__@TTgoAJ<;Jrks->r428g_?@^%P1y2rXHyOk z-mAa&H~8BUbN#yuJ3PnymZfv;?m*Y=jh)JKp(nxYFRbQhev5^m!^XMGrm4VA`2^$Y74bpD;s;oQXtym6(`CT zrAR5X1T7$zG6o1sQM#Xk=Y)Q04vJRYJ(U10Sk;w-VcD3%4*H3pa^^%E%C zsfpXXZ$y2|rP{ht-WSUH``HOm0|bWtw0Vqt8P2 zaJC@J;pXurPWGm$~3?%MXZ=wtqcXygxoot z5^S*ercD9*QS72?mBR(4ElWu`LOi{&Lt@{3ZT?zPZF57Du0nvXmz?lApwoPaUWhg< zoOk6~v~A&G`}4K0*JUf00>v_(dAJ?fc`x21r#y{v5|WE`5Ujo<5znQx8jdd@0yBB_DiqJw%C}DUU2@9;�A`c*hJGS4(m(BB6cOg3JKi-Yf_>*qOeUc z*}|!WuE!}==N(5n1*e1vS4xA4@2b-$DfBQKhz*q8LL|{*Qc{FvDLIT57{(#ti}B^L zBsip_q{36dV&OcBmrq!j>x12$6}sm_4?t3`pLX_X;lnA<4!(K0mS1zm3S?W!b!*%a zr899D;eRN%0Ada8EK~=KW+oMu(t?$$Kxn~|JWQq%Rd|WPUGJ|ci3XQ!8ZT9GN=h5v z-Z9P?)TZ4UK2#9i6)J3Ryydtc=u)A=MTT^B~H@FbWhvDuO6?{c1 zdi+M|>=v5RXt3Wws{bix*xo1%8O}Ps2*Mr8v9+f4!!&xLW=3;{^--=(VV)F+qsNHGzv1n1&di7TSEOLY={s)GDqN)hgcZ}s0hAT)TA70*s&~>DO`M*H?=2Fjda0>pFrl| zx{X3QRj7?}oocaL!>9BzFsV2P%3$Dp!p#<;A3_}Lq6#DNOr}o7GUDu{>n-d@6cV3B zst1a(_~BG@-4R~5>jX*HnowKu`X%{`Kz%wc3cT+$T`OjL^@a^eO3k_gRU@2fp64nO z7cdBLD2KdFf7;a435EQ?cN0De>|t`(mZwWe^SG4ozYz*Tx(4Y9!X)&hNrU{1+9V@d za6`OnhoBT#RvEqFEu4C&i1+hN*CcL!hF)cjZZdF+KLxhSg11mqk{L!6h&0q z?s?qRDT#EN!?ngfyR*ru+tAr#dQ)jy7@v6kHr)kJQ&Xxg!+-PFiEa}-oP{Ca=88q8 zFjc-TMW)S_;!4dqmqc0vK}oEW3d)!tfWIKZWJX5iTD-e@ELr(=P&gTGL>bSiqlb~A zk}?PB!}g2B70ikD+TouHL4rbqP>2oAIb@YMhinzZ(CQTV7V~vxj<-OV#=N$09V1k& z2viHx|1$7m)}G}O@g7*)Nt`a%pe81%=;0#fYfsQbyA|cPwm_Hi+FW%PxKR_0M;@vm zt1|+R8PV{y%ifoLV)j~y58@7=u6Nvi;-L(RN~F~a|4ikcpw^s{MJ9j3wIY1mPl8WF zWuiM4IHW^(XEMr?SGm@N_WIOOL*QkO6-|>$*1^JMc@hH5PZ%tXNjwvODnAd|ng$8B zvoqDEDqFDp2+*%($o3JND6^ZC`KDh!QrW8POo5ZCGTS%_2uGFk81awv#aI*?5&>_! zLM=XCr`XI>VYWr(T0-Ky$0CAZ2octJT*yfAjxte-vkiSG95rVrSS+MNL_B1NOza!% zvs8{ajc{aCUCKMSiP!$QZDbBUjnCyEt5{_r-7!!}i9}PZtq#GA2`b_uVzGtwYn}-m zp*dldld@*Md2E(&IN>~IUxkYt6QVMinZjX=$_aN3$3fY+03!=8To@m`c?l{YrZ#t8 zC|mdgA%yGfriGUQ5+q#@s5|(VN_l#H8X${q5a74{#3RTNzi0@(24&m*V2JjZwV}~) zK^8V);Hs+67G|GRbr@b9JY*F_!byQ^JBh;pA%R0AqI9d{`|I@6oF~QoM+qv3A!Xf| zd21XWxw|kN<=T+<1!l!l*z3v^fr@6JO?ec>dLfd#wxwMOA<_`w0@!xg7$I@YL%Zwb z>DzqcEvH>%_?(z5RH+u15vv8|cfeJsLG5p8)WE6)5Sfh>UN8a&fuGG{=lBI32!>cn2^Eoso{w~tMNpV^NSrV|nFIwYBx<(l zI`swfBhMRULFiw+MQqZrS7sNLp>)L5y!!ZmzWn?D^77aJCVcjaN8K3Iuc_4EenemT znuGNH?k}MoeQ8n^yp!iP3EulPKk0?*^+Fc>iV5IwdS?V^GcVAE?Bc?yZqIx$nqR_J zKgIL=(z(+cOW%G>UwhrX^J;APH@m~_?tLfvzgfKIwmx6c$K&JWdRKJVAA?-a(Ae`I zpZgR4dilIOqb}hh$ro97isRQDaDM^u`oB%J{?Wt!1-$DYQN!bZzQ4@+CL;FfbXc99 zUQPKTqn`C>a3%+N`DXe|j@Prc$oby2v-b8SvG%0hc|ib_k4FCGcxS;W=4P8GE6*#y5TxOEY_qmKL>Ep$|vB~c*Uw^y1n}@#6 zAN+QG%fVTH$^*?)cI??M{4+{~jC&l4_RO^2AV#}~}@Eyu!f4%FSk z7z}$e&O9~bE3BJ!v;Pj;^TPY={sN#YgABUD!lq=)>#K?iqAVAO+-T3v5g{g`;>5^7 z12y5iHWm0+AA%FnNv)nqErq&FB}og6z=SwEVQzA4Mdc785h21?Vnrhrl_n`QUgyT< z>p+TlEBL$Cv7k2OB6+WT&5u=RtaH(JBo;u z6Z12ZD0SqFB~*z8>^TuMDw-;|k4llS(BhoPd?vzxf(q^FYL5kV3r?J)KcS;MRu_c7 z9RplAwZoACg1{SL5m}TIVE`6?x-NQ}zZ)LVn- ziJgxKSE~?}giGN!;`G2FEus!wh?LASdyPn0qAm&k`b4;jV-<#?Rj)_7l!DY<$svxP zt}D*Hdg4>NPR#Nr)Ioz2YicN}zsD0HSQc?C(;YIHDXb~UBwumW$s=1MV&)3wFm1xyqi> z!7IwB6w+H%9gAh%Chh`@Z7HiG2jA?xx*RR=bV8@^ehpem#B%WGekXhFiUWbR!H`P4G62y2$&Fz5D zWr4(RJUm_q3Utk+Kzwk-rs1vaw560~c_6cZsZw@`yWof@2qD2#UUv^$b#!2j*#uRo@yuZ&Btlq7_jj%E~z}kQKy#B0d&yXq@NDv!I+zY_NEFn9k-^ zNhJA(a9$+Zc{HO7h;T-9TCX|Y>oork3L;ooM9#IX}S7euX z4#jH|6x))-mPK}L39|#&(+{|Uw_K1&4^zl>VYW}Smg5K_hESM&*P!8LnbX&-jx<7~ zNy1Ozio1~2H?V6h3WzHfwwB>Y*}WkYe68K!W1yQFoK(SprCpA$jT#SB`ywScYApV_24VI8|IPm%*yj)7bYDKnACu&S`JHu#6Cxeo7wR~a{#VN;gwHN0!Dp4jl# z3StVoUUV@4!XSMLL{kdg(^|nX6pKhyRZ~8ugY$l^1SSib93m98fq|XB3ctjx3j8A# z?PSD3%A+8>Qqo({xTvfk&L*5QVc1e8or1*nP)23L5}iEdYOw2flC41@(DxN7Z>+mi zOaZD-(it${p}N4?tT3#*86Dx{vRAQEO%g~R;emv~ao}Cxji;bats`N;Bn~GcVuXhh!r#mmD?%4I8-oe8FQS(< z6NdnIQhYAal!oM_@NC@^cs8~#)9@%sVQ8cBf_N1o6GdV$aO;K9XXmX-?!tmG=n`Fs z5$dFg3#oLd+$TtYv;N&Ue9TAp0l_Wvn#Qdh{0Allu3J-CUX-IGOY6XH$SZSA)#Hkv}63&TrR^6nTY! zrl*PET=p9lmchkFnt2^!ZHrQ{1k9-^)!CwSIbdJ$o|3P@iM}`c#%0S23sS0m#qH_Li$X;9P@SqI2zy*)@e5uJSssKOm5C7E;TCn`2~vtakR(LxFyCQxIZPFl z7*q#CHjJT)7VzRMiz)m;cC^{zx$xpuI=U>Wq=A!SvH~hm?#gPx^p>F(Y}*Tq5|l?7 ztU{wIq6|gl!Go!(W8&Ma4ODayZcjA(D^J&CF4rjaRCit_m{_P}2&8YC!q53O=NjtY~? zeUq1`NcJ!sx~hwhm8o}`QHIbd_pZRdkl>H05yztZzGSu9{Oa?+2;yq=%dbXQUNJ=P zUXec^Uh%|49|n;DyDsL~6y<-6S-k)E&L<=^0x z{xk{n$P@kl#4O!0g!jAM@f{~>VJqIvSk_K4{wt2#UBB?h?oSWzY{CHtBJ+=bcjWvx z@`V2Q-W#HZN@M`!_zXgmg6zAImXY2Og%s=~h7Gh^oZFTX>uYdXZ3_|?!^lVdL z&ht3P9(pyO{)}+katk?)h|5A&*;B{VzM9WHQhLu+`!n>@23q}UZxfC-%O?);+S>5l z6E@gn={($;l)0Quf$M*}BU+X~CPAlloHxOmC++6PuX^&t;01Bjm1cW<4Ix-&Z8#TQ=^e~oM_I9?n0%;L_6zlknp6uF&*^6I=b&%NS2d{o9tkEIRb_2!@k27#vR8o#Ze?%^%UPewQYkMOC<*;DV39cKxKQ4=mh4`K!E1Z)-OB9h;Cc&pHIA=0*2`4Wk9|SuHB2=Mxc))sop?sJm*rHQ~XSLUcyPo$3igc6{#v)jb$5UHYls8 z(XxZHMj_b|VMrR-X-$M!l2f5rX7~Nz6q;L>f+XdF{|@b~v(@m%1-1er&$^tXfpsJ{ z6_H5JaF#`l6!DxJ^6@?QGD-MyD2?%LJ$#kBw6H;elcN*Q=1e*F+dAQrav~NM-}Oo)Pj@j6>N7?xpV?Tyx5d9O%lQ8 z*#n@`u*~zyEHkuuwT|`iCoBpop>tL<6KiEn9DWF$Ec%C{$r7dKGGbsA=f)OGPBO}G zjCh1$Vtgu92eW~ad<%jyuoU&OTq8%GVrTWiNuAAn>NGU`_5=^Bf9kd165T-aF4f?&gkBYaD!Kp;E zlAKWS7P8(X4Q3>$ae(&CsaQnQGJWdUIiPGj;^Q{ZscmF4%7Mt~H9VM(5BNKhLZn0hKdIyq7zGWD-H%Ap2jrvuiP zeG3BgPQCy|It?922IA3}VNVo+G&yr?RQbNfRJvl6<*)$JH^y9>A_cR9-h@JF6)({g zDM}@&kn+G2VF&|Q*mj*L{NX52u1T~`7Ej?)^UYPp7%l|V)_8mfaH(||31GY4Si=!o zfK^)ohK62NkihJ791DZarC`)l;=PH>SJOxkMg-M#V(2r)d<#Q?11$+)1I?>w2=NWR zskBs(LKP<>LLN9AP5C(T4$D{t*Vpp{Acc8A9Gb)K<47%LuaPKms?#W!LXu`72{?R1 zr>Xsf;7JK|B|U1BE6-D)sEHtJsu_^&Qr9F9-PqTGJidGJi41&7IN5nCwV3hMNef8= z45g8b160IySXkUdx!2x#sgdB6Nh8cgz-izXG3!jczK}yWKVg_AF!dr9>!j)wFhQ?c zM@X@u`;v0dz_SKbQDyms=xj+?@x zup3qpRI>kseLm{_So`rRdxe`%13{$)38d^%>YJ#gNp+f&RaZz4^bW_7jHjb9m{!|l z)Fl%1fklrhMaBHkq((a!u*`|cDJ+@Na)^zB^Pb&s{JVNBDGYeOsdI~G$&L&Y3^!f0 zKzi~=DM&S2-KG7YyRF=)*qTz|(ZkYJg5#WgDw(x`3ab1fLc?baD?E`!T6>@(cx4BooHM{~M@igI zAYoHewH)l?wMCvYD>=!Yaz~J+Y~jenmIO(ZTGLKn`nS@ zIUWVNs||f=-8W=5N&+$~{SKA*NEV>UoONL5n4&S~vklCsw!w`&*b+O`YLiT|?2l*z z3Yhzn$YdegX<^RbstImkh*m8_4Cs;w| z4wd&>%Q$GkOfc!GRY#Y1b-;4l16L+rgr#llagkANYD(J_4gpg!1IP7fqD@1e3BNLt zms%`{F$Tz#Wi`0Jcs(|Gss2=iK20(}lU|jB8HCnkhp(ji=e{Nt4u(k0tlIo(ew4%O zNs%ivlaWn%Dy?J`jlvC2ll8637Aa=? zJ4u8^_IO#sMlgl9S3_uGnb`O!iE%Qi;#IMa{iZ{jxw}eM^bm6IIr30B1{~O-f?SWY zRaLLC%`cIrpQYEK$$WOyCPm%?Pt*@7$L?}AiT8|5M|kC^CH9xg#vHP>ip*C_r0x=i zdLrswIS|Dr^C)OLg4jHylkDU&R#yRqQPm2!vJ|$%3nsUkreP}TY+Q2~%KF_O$m0~^ z$%wur>kubtHsx`^aj8(5%GD)x)v$e6|G~!IDj&3xwt4(}PW~*fKK;|txBovH?&lr! z$zOcCBjEOVwO{Y<-YnKfdpV*(jas$ypUw|#kG4NQJFoWZ<#D~8$$0TMtJC+p}PNI1oo%+>bSFSmixowo5kV%^6B}rpUv<`?lB>#iwn>i`)FLO z_G6N(u9oERU2h+k@7BlV_IP;nde!k(i(f4_8$8KzzkauRdA_grSntmtRu^FE>m}@f z)Th;Q%XgYLk}SjP!{cIgeBAD~AD$Nb6@kKhrguc?-Yh=9h3pxy8*=w|%OBQQnV+vV zOS~JA*w~h*<%jtd^h!PBv28hSakD!-B9>zD`R1O;+pl(;yE8O=-hktIeXtXsncyzJ zU+oW{ae3{(DFS#we)fhUjQ{2>jF|R(IR19E!CT?QKiuEr&KtbKo0lJQv)=6>kk7lr zVf}8y=RZI=?=HK&T>N&w!zRw}alhMhX>NDh!*PGQo#o`;egA&Bf4e@s#q05nX7KnA zD*|YLi-p>+-u_22DF1r-^lr7sO6+&IBj0h?NrZW9&f9uM@_u%B#{T2kch8&E`3<*Q zlF7~oqaX0qW@R|$`Hg?Gxx@Z*FK@9kcxSr|-}EE#f;dpRxQqSq5C7Nydb9YN&~V&m zd02f!5pDYb3El2USG+OY_I$rCy$~ZiOpB)v$43M8&QkvEitNh}-Ls^bo+7NgA6(Kw zZy!wdF3JK6()G#qjN-n0wZd0FUHt1m{vSUxG45AdZVo%F(|_9S=T*lGY&LjA{^ntK zy!@m;(pN^YYpCPj;Co2a5KCgXpnmJP<$QmOm>&dM4%jc zSx1jy8F7>F4svla<-y`Y8XTkU*GD;UGg%xYs_%g=cajh%PBjt+II{g$ zwAMwZ#z>K4RL)h2$-4I>Oq%1sBq3ss0-njILd2FwBCEbuiDS29E{05rl0^aQDCvVC zSIW!qkc6P2(yj0}!Y8UxmU~k%oP>H0wsWeW3BFoU!8*Y|PQb(p!AcUK4wn<*>UCt2 zZ%QF~vFcu9`dUC58GKYZe1cGOuh8bCoERo;a%)LR8k%ge<;X3SXsRUQQNyKJ1oDde z#Z7vSku&NHaX4*W=@88Hh#!(LV5@p2f{dXm@N4-Kywo{pe$3=KzE}0f*O--DIkJBa znbpY{uCsz$N6x2|6ZtYp;D(>rQ$c(N%ClYr;ec_KgJI*RMA|{gsD5L_!ZMo@YgVWu zQf?=i$0JZF55!NrGwb4#P7)#bG~moFsBYwZ71EQ7O?(oXEKaPmsN`*plU#NK@?>FH zh_a62d2y119B;~5AgUNSMZ{@Av0i(k?-Y)}ksn<`E1=ziB%LryDmftvF<;JA;sOZ% zk>xnDLJj#_%Nf}Salse>i|RG44hiT~uQN_mlBhW-eL^*qVB2U&mFbTpW>(QS47XF$ zqYCqWin}p|Y1opUsq_twyP-&>8N+kOk2E6jt(G>|}@I=}sNr2A@=~3VU$?q43R9P?JH4lGB9xZlZ3-t>v z8>AZqB}C|s?3r>E##MQFF^(i8lVRlQBtFliVilK*^JXLN>oa(Ad3>1XmQ+3NP*lQr8oLJ~p zqvFnN7E1jD+Y}TZR`!9c;fm@8MsQHO{V?PF>4>ovjTubC+yHhgKK<~p@uGGOFq=-Yfx!;4kwhr0NNOH6Zy<4mvLr2nFXd#% zFIGglAm+3Pm$=drs;*NO!^_asH5qLr|52qIRorcp1|pjJHTPwZJ7R0ae7ltss4A*r)P zIxQI?w{l$AMJflMu<)t~;7cOUkDLa|HYt&GVN1-Upc5=VHd#UD@Y})5>)NCFDn?0> z{@u%YJFwD-UnHLoCm~3#y9v%fP-#g<$8xVYDp|#wbYPOKwJ3fo8?Cl!dYWqgR8 zdYD4GdUf8c?LlD&IJ+Sig>_(hEQ^N0Z1bu<{ zsUTy^{{%xBr@^SR(+UgED(ym+2sPm&7+^*OJqWobHR%nf4Pm~re#{?9eooj7qCbWA zOQP0Ea$ZJg0ZG|x_aSBi|2?2KcM5g?jZl4(2cM?au8ZDw5m9xnXHH^(Fj8#2jtMg`(Ze( z$tfocsQe1K1?n_J5|mTu(lDUl%@^b@ieLr4jTfVH97h<-A-gyvnsUb2$sR$l z#AK3KQ8i0s@~kT6yHFLx1Z!Bwxmu>^pnWB|Kq0-pk_FbVQq&cYOfFZ2o}zQEU^I>l z$WjB76h`H0n75o}w`13TqpNy}-ZHt1GFBH11MHUveIPlMA9Wk+A6VpO?Y&v72| zCj^FMIUE~u5{D6cvx$MtA4UzCva!}(H5eVrT(U5;QBL*S7PwRcZ^hJmEIAcQ>^p=g z4c(%qP>7jRi40<5^wdN6PO^c6yt@N)g}pgq@PQ;}>a{Zbqw1g@CQ2GX+jF5@9Xj`&CW_Y3%)0b=VwB9I%z$Ai?U&h{Wc>iXXVDDb+KFgUKwT zO99cV%7n%0<-13?R9b0fgFQ;NAH0Z!6IYIFQjR9kXVXv+?1zWbXrv=goaCe^a@c@1 zsE6+8a=5K3cub6Ul~oG#d6I}?5O%n68b$Z3DhIlW#IQB=C@3FIbyKAfkBD!WS<-c| zjcAe;1P6~#eFGI7bSE~Wi8_!0n~mj!QnmaX*xDp9dF7NpAr~Z7!m}x~1e>`O)Y^GA zvI&}CvU24Sm+y=^TD^Ngz}W4%*_<3X?9&CRR5^N(6#BY523wUy z<=*BPs(}pKK^rPZ-4(W!18c<_qL-eXZ+fX1pM7< zyW5gbc1CB;fW?l6TkcoO8D_WNon{Q?)B5gibGb~|-@QWM-YkBzy?n)Ae|A_rE&tPQ z--%KG_3!`jusaae)o?+%CgzI`zK%)$BN|F$tc*3EM9<@)&JKc5k< z5BT+Ix!HVJyj$HGvU0mva{@iFrFL!OSv79mjuJgX679Zso#yt52tgtV_lx81J@I&l zT}Nr&6FK~T`QeOU{QBK$zs5a&yV|a|htp=+VZu*DdD>&I_X}bGZ+H0m6)M)ys)YD% z|9D#MKU&!@mWQ>029qannU{OFdRT7fs~uLy;|zz~&19pW5%qh2G$n;Lx)A$`Ywqw<<;LUkB>zBuAZJZGj;LuXe7plX5KEAPtT9*`O|s% zzW89zvBDqBkMy@hV&g&jTmRzed3`2MzO29nLfs$##5;80bO(MK^8Eeow7Hw{-FSiB z3vl`Tk~fPlAMpW}V{0$7G(h!!cXzt|2z~yEQ}FrE32`-Q--S{8zIg?5{bGH1+&y3L z?f-tUd<{q5tRC>j6!X<~VO+lVrx{+0C&LN6T`eBh6i_bkyVX7S^#vEbyS+UtB!l=%5XJ+3#K`4G*UcUiMHHjfB4B)`~@Dje{=Z5Pk#1h zUc5j21V1@G;q(29f?|3CPi{QCxclaN+?Thg|FO=9b+qpV!Cbrms?(>YvAx z3(qS_^7%@3Jr%sMo;8p=bVt?-8zSs6qCRztqR8T7D+u=#i@awatg>??7my>>2yK&H zl1e=WKiTo7wSg)zduGR#!Rfc`lh-|<~?<{S|fJ5)0C=%KUl(}$fm zkzgO1a0baD7bWXz;9Lw<>0vTc4n7P#SA%13@jnNGiKI6NLQ_GAauTbdL(6BE?&6#% z0nrbhBxw_Z8Z!O~TuorFx~f? z)yWYp!|3-Of-r7U(!iEol_@4%%MEq6o>*s+N=+I@I6iX0x|@XO4av*#DHMkwR@an} z>eEk|Jb-F|O5lWbIeVH|_Mr*3rz0YvUka1rI2lQG+7jS0jL@EXl>k4JsKVM55g~q3 zIr3KYJ#q%n$n45*3~nf=L>gg+U@t;kh?i7BE~|yoYB+L|$h{}arRs0U)<>>7R0`f* z2{DPMvt}{DMrI+anFFqr9w^XtlPJ(&Yo|$;&3!E--%=W_a0a_V>JgYLt8y9QAlX%fE2yJaZk21c$(CiA{4HAFjyhQ+8}1 z6Yd2rRutBX7+UFGG{|UEtWX46HB}~cta-{wcP#e?J*yy+2X=ri^^%Sw=v(X}eKDC~ z4Fp{d@SQHĠ%gh-BON&Wl@ktl9zlbBxSaCpK%TGUT3ixQ*ZZ{E0Q#vgbb-T~=l z0RXmM0(RM`roh-d3qwbW`7bnmstV37C$Uw$ewB4-$VnKJ&OKk497&nPc`KaVAuI_= zeg;0+dV`h9QTBw9qc%y@R3O8$s%ki_t{dQQw!DdRyvI{Qn6ygO!c4be*?T}nuwYf8 z!BizskhDv(Isk_!Np2B15Jl0HKvk1+N|Y9Z@v{jj3CwpZuKorOP4!+Cm^=xACv2(6 z6^ZP-7%211$t9Pd&;X51NKsNz79uH>q2iQY^(LHbAkjWp+;QMS-@-y{zChRILVgw% zJGbhUCQs3Z9!S0`b_vVMlTV70!bZL*eO*{|8ff0^Y0023&R~wU4i2cDf=|WZg=99P z)v1jNKy_Kd1qh0~-{WY=?c)G;%~79ZM_(c~XXu_a5y7GLvSX4+*#S}50~n%JWv^JX zP6>r2LaZVx5^PzKb0MwaZWwGk8+Ay7axghGS#rCRLryLuymW}7cGtXD69PG3lys&- z5)00hD}tCD(Wz6NL5>p*g<#S1#7e3+$w3#h>Cq60BEBKJbWn{H&@q+*7#Ffel2EFY z5KS8%P4#l&N^t52UNI3mLL4U9O2!5#r&O;%2&L65v6InUK}s(>S?) zu4Wh5N|mf)OkX8QB4jlUXH z6-OP9d(}^agOe=;m^`8!^DUN(-8xd+!)5hC;Y}H8N=MGn1WgS{3Uo+R8yIJd5V<>^ zUib+k=WyRFV$KyLgUkfuyt^(&&_gQp;DB|w&8l^uFk@8@>A+5PwgREw!l=--$+~-! zZhICj-jii$RgX{Oz!u@hB={J9&<5rkv?QaCoN!K6$O)ezRLMGsqcu>fbw?Besj5nu z$O|Q0@Fbi{BYCnJh(t3dFK8?=XOYQaGzf^``KW{|fr`^G&bgmb&*m=GnM}86l6W5` zBP9toExp(h(nFR3hBR47orhbP!7NmfiUyhdu1erJ9!?IgvDB7KPgenHon39oV@^Jk zBFAv}SjS25?uPCBkQ9lwVfrDcCR_$(D*Zqxm1=?uB}|e--4{4&auTRVR2{McY&k1? zsIzWkYD8@^W;m9TOqv0ilZm}*c>u+w4AY|q5Jrwtl(W+im<&^Oy1=t3QlV)PF%3jE z_PFqb;U_6D5h4l+lVn3?@~RKZs3R3nui~F!;(519wGPcE{khonE$T6`lsK}2+D|f@ z*wq(}DJIR1P+66c2@0NF*u{~P(5pZJ2;;Y&}3yB4U zyTEQNuiA1F2bd@zx#yUAMZE6m2M0B7Av<*Ox`k4%~!%gTyvHBvOnEgUR6*?QTp@LS=yErvZ$Ftb1)G%E* z#X@QTMzvQqH&1598tjY!XiPNX!&spC1(TeokU5Q14X$GJr%0QN$9qKK>m!DSZ>9QV zt&2zvF$UV#Fn9w_=rPa&Dd@NtLX~kbYW$ z#SJMw3F0M)d#<9NBU>A#FqV!3Vw5|}VK(s-h%Z#0ogjR8^WoL!e>z_G|8=z4=dUhKlFGZ?)0*U-yRC6vE>D{d zKz-9eaE82Fo{_eT_0!Y(Vc-A!-G`Se8Ln@}3U6QGTlskW12X?E?$*oA>h^e8yyu1A z?e^Q7#V>dJj&G@z6RfY{q>v0H%okfYd`*7$AQ257>WD_8+L~?w%dK* zRmqte_|^IRo81uy_~+cPFPHe>j&F{q?G(@R)Bb1t%@ek3-lWg(i52d&$NR(MrX$^t z?^mmSY&rnH!~Tsc7uPy$c2E_I&+b;(k>&pBGj1rq=;-0j32q2?h{Sb!f}4h&g-UqY zDC-J1gY9>av>#SyaPqehs58L+&q3!RgsHbs zUax}BB0!eSMv1Jkvd53pv}E|goIHGc(bU2*>CzJ*TvIU`B(jdE#1AKsO3pCHioS~4 zC!+^cemZAs4r4F#4cr6#K!-J)@g>n|WiqFKwD+mifn~^9*ClQ)Z{jNgy(Hki#b-7r zjh-PLr-6&lypUoHzru;GoNqmVpeLKA!CI=UZA4v>M$nj;C=-5LjMd2=dXDuQ!*h$< z$$uoryuvs20!|l#jrSf;LnL#c@=S9W2PgN(^-f`_2QMSp;#^KjlgnJ^U*M<~=XO57W%$<&hLG)GASz_PE5=U9jcqQ2`LGtY7ga^s0sp_qEmHFXBE zA42t>nj-t7VjXg!PI3?^xluHviBmuoKxrChklNX=;}DvjV+u}XaLGp5RH7DvYLfl$ zJ{Kr`;TphY85ksaFrki$4!LvlkkN4kog;sp9z$*@UX2MsV6UuNCF7f7y@@QusWr5A z1#xr7iislogr)Pt@V;(QS@)9Q=@Q~FBCzW#=ZkXuvXKfam<;rS>d|Se{3sZ9rMM|i zf_ZD9a~&t~$z(F=DAnXKH`17xgX)81edIiN4Q7gs-mADZd5PpKQf8RtO5>&)KMe*(6QUrht)Geul6{!Jvy;L7nE>RB+ z^n?^E^+P6{(W~S_dcrfrFACm4_U1s1N^SyH4OJA!ZpNxOh^vjomBhYqLI+>owu60m zdW^^5K$(eZc^L9)P9NOnV&`DCauy1j|m-b^08==;= z#bKs&ccycdkck+Hrc_1+3oFeK`D`Xtn+bHm6>9P{s&LD(@g+$5V}u^^-Xy)#L4GCmn{r7a2SCeq4G3aV zelFB~&JwafCwCgkYqnUb^rclrqgZ$){YU7kRFyYxbt#z_ zXdG>%Qb-0PR4N?)_s|<*5|@Z?9G!Mfb{_`1Oh%(-6Bor(rDQ@OzJa@xhS9pyJlbR! zfn}0X320W7vV>3B|ume4FjesVms z^ElXI8#%ls4rRQaxNYDs9w-5#LD5+p! zd0@pfEFfmdS)Vz3RyZioo1+BTdsLD|3k|{4tdlK1i8v9pCiE(r8ii%bM$YBH&ByKy zE<3@*3Nnxo^UK(YM^_@9tk+edE3k#x1%)s7BsP>I(`^2=crr(1fNB#69QPFzu>$o! zI~COHFczZvRKFrTYPeH(-x~H_c;#$OPeUSzhrU`4dUQS-A?6}U6Mdb=$ueD#^5D0c zULnv~g_PWs0WaPSN+vW+0;fa^ne9FapcP^p5g&nO>ZFt;=Y+vWj9#)lXF_;nFlfCK zLfs*j*G;e06J(+~iUMh-1uo8!{$!Y7V|>ODI#THsV!uToO2}!DbM)|fUL{vtBMM29 zL8Ms3LowDIyBTM7gOe%;J$srgweZg7s;LtcIi^c^G1%K_sGN0+Rd^fG`;{Zv8AbJm zFEYe2vsW9)Ek0^Hrcs8L&7LJ@Uxj$y+T>JuB%TLX)M*j<-qqb4adn+hP@NNfPJ6ja1LhqeUQE& z(dA8WS+XbsU9UvmCiN&cobI;FCXnn%78*DS5`Kg_4y5TFXr^p~6PHGUZOI@ciFzoj za@-q1nnNnmlub7$N?rzbjBa73GO}gA?GX@*qvI_S8Fxi}hX-*IDQwQO)ZGacHc1aT zA*GN=<#IZQmt<^hCyG? z4&vnGYO#&ij~x4Et2hy0D1vlt8b+R}x3d+MgzMCel4+PMDn;_yGu@{KU1bPp5>jpB zq?FC~hydD9=9xOVKn!w%7%4S|iLCrdWyw}%#1Sk)CghA0KMgf^+Y}Bq6WeQi(X#mx z!BQJ2x`%`p!=t_#lAz?;-<92Qw5LK2s&v{xVh=_uU)QNe140t|(8r`y&0j+Vq5 ze7`z&V$Gi(cX!JV#vprGd_{8D!_OA`m3`SCXX}fCnu2eoG$S8P53icE+k62Mp=sdb56Be13X3 z9ge31fsm{1-Tacj-kp9R1@Dezshthg;KO$>uevcPBT!fYE2DVjX6d`K$-D$l!eguBTBdkceyI&B5%9ZE0CUFFO zA1M4CrtBk+DEGhqUzqq{sMh~}-yWNJA zx;yexA5V9<=iB3IdCP}+wbifrdSBsk|HGCmbl~fM1X1pPh+DirUvyr)WoNMc#c98L zK4XXfmN;lXO>f}%ovE*v`~_Z+CB-*qRIGuumzzDLw1bETq;CiBn$j@T^6vSxxnP!G z?z{Nn177awnU|#)|9ZE*+im;(@C*bW*Wq#OR|Z{P?jg7LyA#d{-t3j!9iSFYw~tgZ z_?VZu-EC%|I0TA&d)mJvWwJd-U(}fZm&^7wUWQia205gmshLYG zpo-ufVUe7e!h%g2s6*Thfx4yX7Fsr;*C^5u46K99!;|Vz@h`IZzluQ4aZ2bg-qJSK zM&K$85kx`HPPMd&*o{bzLq$G_pHxLOYc#KFpcT$*3_~upS5lbXCD2p#VX%AH4a$+~NSYGnMH#LoibFGu4^yawqCQ|{GNPfxFT2v2oOi1C?Rl5iE(C6OW_DagB<00U|f4K8cL|D+(vgg1K@_Vi*Uf^6!aSZ*trZ zS$8FyAh}#+^HDJl%P zjHM}hc~0PqGGFAH4gXRmGLkz;7`D)-ypNksD7ea(R5P}s?}|A$)zIvTcaWfY4woE> zo6WQIp{VjBHa(PqKihbQdaWHN%n*?6fyc-v!DoZ zU8y#4ZxpS_d0-0SSeu%{Oe()1_=L#at{M+2iP*$RE7U%+ctLC1ly-&n@^0Ypwn)R6 zRYd-mv)J7@iYMRL;asf;EU^Cp369CbDrUpO<|M!+cv?c}_wW#vb9)G@f*GRwDs#3JJ?GkqY3< z!3#k_16ev!)S-b_Qq`>N86T(=ekMD<%vBTop$wIWSYv@@StY_fP<%}sOg$nzdUy-5 zSA*%uOf-zYHe*a452Y{zRKO%rdx>s~dMs9{g*1A)0SeYtu#S_WT7$enJE#r>PRb2K z+%)xgaTsoUc1njfS#zNi=>=8Tlu6|?{(!tO3Xxd~Ihz%8d2(T(JnU`@i6C`j=|r+i z6;QEwFfgrk+s28H60un5Q;G8ijX99ClpA>bKKOCyZOUeRs*XBOCSkJ z!hDm2qKx`wdjvfAaYTu&F+3#6R>UzjqNdM@M;vk^ygbJfFSKvYSvLK|$vEI7c9^*W zl|o>*dMN@$1Mz}aqk6D$pE5YurXbjwrM8{Z-c?KmO)6##>V`D8%n^O&&WYEQL z^cHgJxkTz0Sf!-;t;iZ$*>-^uz(ES<_^8B=%5j zO0)iRm8ijy&qGGWB##LsiSr}zZyqos-JQ+8NER(v0QeDdQZZ@Jf*NX)y$S8;=;FDO z%2RWUgv2;U@0mci6)7$R%y;OBR@ixjg0bIdlhG8}CSR>-k;{pS7#>K_j! zA!meSiZ%+OL&DG?-*ZJ1vt-I&;7pUjqVPpn)K|YCTUHqOGtIxjbBPiNjm9vB#ZCgj z!wTU@3Pafx1$&~{Hi?^)MAI;rZIVtUTk4CPEmYVzq{hi~f?|>4B_V@sMq}T6lXEF1 zsv0L}>Et#VjULoCrcDGl9aP;w>D=G3(K1RU#Ks4aBP2y)8o1_c?MqXEZE$SuRfDEs zN`p#6l9hED=5_Zq34CU?jquY{Tof6Y40(?JhKLpwP|6_N1CXbCk9ksN4iQ;W!30?x zoP;q3N3m0ktt{^;x{bmq49fVx`uZYauRhDs6v;!63zQT|9Apn7yD!P{Cv*dxBnYX> za+FDuNJLKS8C!-d{-{(M!a2vy%YYCzbBMs&U2CRv<5<-znvKT=GN0>tbi_W2T=J)~ z5QH&?p?|-7yQKG4-Y(&EA&Z$OOuI!nD2Z4zowtG2cf)M39vcwurMwPK>T9xxB(gH5 zI-sI~;2|D`F>(Mp7JDoDPtO%*vd&%6_0o?c2qqx=fWo zQKX%a+x!)U6EY=!bvLtmkT zygF5ZnYpH?`oe6fdtj;3z>w49D97o+D$5T=FDqM~o|TH$U>clPtLQi=J1I7P&xww5 z+)ZG_GEgkoH=brwz~(2yAe3)Mz`@vE(hrHzONpvmNs}5$k|=blR>aApYS=+xT@7pb z>hnJxko$-2&tG^gzdCL2h$8!s1Z{q{_}k_F%-l%;-tPAH^t|pMvtMi<2w!}AzhAGm zcbgB3FHZXh{F*qo``!L_H6sHZ`34s6R``r4TjMT0UuJhY?hRr)^WE~AB+h+LK;T!a z?f%2!JA?Q2do5Pm<7z*{J!fRw87fGOU-Z*o@m~)3Xw?}~@mX7}?gM+WXEfw~Wk^h8?7TdfJ$VOCp0>yJW$G0jEt0#l3 zetfGluPc9OysrO#VekC(yjdO34{$}*`Z1XKw7n&Wd3E>3fWGgSoA2Ll)>|GFLjw1` z`|WOfT>Q=IbWbK_LWP%S%(CH_2}S08wj>P3<3JMU=lnhXdbKBVlw7w&!x}H{eziP5 zUVn@(Pc!4~aZQ{kQQzm=8-V;C?e6^rV}4kCw>vz{K;z5dIqxIhvL7H@w|O1= zj@+*Ix2MgF8RtE=%RTpl@atELbjG-LNyjVV)W18~OWrSUk2^xKZS^UsB!=!MYzrOq z3QhkJ+Mcoi^FJTGR|?FP*TCH`m-{Eb?5ykVu;rn-A zlf;+Lc!zD@KiU4VAz=Ca%ozOwKz_Gc;`zx?eU{8K0{N@eyt(<P(o~W-$1rao}J{k+y3%#{d8Du?&pm(!sEB+cl+~rbfYH>OI2W7 zV;{C8N3wa0vY|f0u8XW@f&FxnT#SG@RTd!;$Vj1SZWBWlHGRhggs`e`nJe?1A|FOR zieccZy+?^7l-0ZqKlU>AXvmL!(0xgWei1HaCPkYv_V9lOIAjl<*}|oKAY!W@Hr86={vdiZJV>C}5egRpEN#ttH(lr|innN$jq4DxF$L z0jKx?DjO43W}nkwRK^wp<^50r>{>h;GfXb3rz##K+rgBDOF z2?Fr+tfMqZA|#^rhtf^Ci4pN?jh}cdm*le4n4tvZ48oDWp~v_RYD2A76XUYpo&$D~ z%Vr~LQ@z6pr>e=UA=8Ws$y^Adk;9xaxjDqijhwof)MRu=jiLl7u*pHvVnKkLq-o=* zZRQnlVMGEakONfiy61$}fmzO!L?x$^GWp(7 z1B)ed4hm=BVd_r8lm_6_WQaZ%;$+W_pW09Zv=j;%eI@xofv-1A=`;0Fn8guvs?G-b zs)>IT)GS<37RYt?J(N^aI@LAU>Y)HS8KD)239Gx4*KQIe9L}F4>#t;C;v7bGP!kK9 zNm7;(Z>MlcMprHMEzkDQ|I$-iu>2At!fEzNa+l`H1|#KO*f}0m4ilYkF!S#3A8o%r3WmXlv*d z>0FIoLRgK|oN8Jz#vl0$oCJ=q6FEpGM-@3^BJYdB99Pn=iCC%Hn@eUPHg-kD!j714 zXr}8g$PBk|#)$8iQA{LWT@hiBMMhOjCw7g9sx!d^FL3Kr|6YR)9`9Vp#wqgwey{TL=`jQD1UGF4UbO^I~MsSMVlOEBkVNXVX}OCak* z1jQ=(3f_N$dxbuaazz6zfTD7QtR^|!fe@`A3)QY9B6j4|+>%Cy+WszNtyG+x3>bqN zuuxo9pf5+N?O}++oZ|9jNK8$Z+heU*PJ{^LI2S+Ru@l|KWab^B@t&=!1V~8Vm4t#4 zM#~vepCS|tYAn5L9C9RDugH@rvI_7|8_$klC%9hiuvMbVyA!UV3w8zT! zV&Q0WPPW9YlV4wj#LWE32(#K~Yo(yrxg;8CIia;luI5P4N)X6M(m|^^Z{8ZMlx}8= zjGPzYRcmO!IS%R2Ph~quG_KNtzJ(N%WH2GAdcMwdg&RXzOe5uX7so_)3Ke+A#`B=` z3TfdaogxVWRa^weGEW03^km7K!7$ryap}cbg#Jg|AgKc*-B;n2keg6?iDdp9<_paJ z6IkpgJC%* z3HC%1$5ODP2!LFsYLrQ{2K%RMGJB4PRsq{6I*&mPGGaAK#U>o=JqG+;0S9X(5Js+P zmCaEpPExIN0?yE~HYf84&eJ^~2xtTdFj!}~$oQAcwnte-V(Wn#`ViP^JG7{peyVol z#cv?9>I9fYJqsXj)KpW6o4>lA0;ewbkR)uvczi61A(b3L&gK{@yjk z07*?071=qgX(Q}RDK9tB9NVUpX*hItfExZ$YziE=Ie4#@+Ndqcffr;OX+i~)1)w#F z146K=h zxrp#cLG+Vlo`tL;I-)sM<_W%!$n#mxpsYD?90pSvN>)j|Byxj+Byn=`A54l@xgE|H z9*Xw75~mTX98RDYQ{;T1Om;EZ#zOWe-2o0~U-7MD)j3Q-ykVi&CG;@&L!pby;wlQu^s zSv}-LRY3wt$p&`5IbwqvR_XA{Nrt-rfuLMjaAz}#@|~d@Ip28Hk5HRL+ly^X$xFoo zora)vaE8+jS%?wm$+=yDg+V#`%ve_xrk#{`*}wYyPe=Ive}PHsG`|3e4Y!&{PjE{qhF@BKW(DZg9o4@=@*u_E?>)#DA$l(yU5Vzb*m%&f(Nn{-KR zB-iI~yIyUHd|q!~V5|2V+;n|-ByN_BsC|W>e*EXV_3AAz@#BBPThAcP&u^D^tEZWD zb-g7EB$n&ee`Q3cOo5kDLuV${+<8gWWeZQXpWl3oI z>TgMM%WG}=L-Xzz9^?K7`_<}i=|F;)ZYk#z3Km*I3r?R@=KD|K;f;e5;{Z*SE*rdT;WLhbM&e z>+KKA%^F|Ma^>o=Q?t5u!Hcd}2Y%I|o)3G-2e#3uW;-6WFCO0bxQG{4TT1gpiGta^ ze1w1q^3zY+`NbD6a{lGx^6B~T_Q!wNd^r60&#NmQeCzUdX|W%t_)C-LQ` z^E(q*%=5MRVCd?*^R_?laO$7d57^P8Jq1zdwkSK>(f#Vd*py$u*nO>EVD5IO&bLi; z{uVF(yxARQO!YHfad+6^|J}ZvrsMja|9N}F!6rWc`z`U_^R+(jO6nKe4Z8?dokV)^ zbG+AnceHE#{IkzD1jORhKcC*s5{;j(iRr)naJ$((+d=Hx2K#;PRN&>m*=>&xyUqO? zFZadXUh=TmEKfLuAK|W_mRp{MdwU9AX}Ngh{c)3bI88gg-ge>LfB&z4LkK#ah9VOF zJm4Jln8|3L32} z0!S&o&oHSA8 z=0K{dNP82<*{ge0!lhJ1P2zwwFixLy?NLtC$5;(sN5DJzV*0GV!u=omW@k!n3qMX&it8D1FrG(sx(fpfley7)yA^>)Ms!rb15uz zz`<1}gcaif4&Lw0s(?tJks)%Degg?sJt^HrfvBlqC6$#Xm`S0wk~M9}i3y2Q;nm?K z?O_OtgN%4iH!!GlHAuF8ITzksKxG+NO&9RVYAj%~nw-d-JT4`vfr8LaP%2K%%3w2B zQhX3_t$8I;h)gY}fl7!QPhVDuN-jLr+Uut2Q|3F?X`sZXJitx{Iiq=EGoq%#$bk?$ zRn+6zl?8^7h>5$tO}&95SxS+RxBy4f3(dvccO6*lrq1SGG+c{B3Ihn%Ja1=s!Qm$~ zyK&G^<`8hguEfC5@Jlu1Nu4aJ9;uPhOu=cLX596GHeA896Q@?F2(5U;Pl~9}V^2nE z!O5uJA&n}UWti&*-_-6B+0Z7d$y^s1`$q-y}>~ON?fZLgUZ4TOz;|$pe7DWNr z;#X-jJV|C{+ajTI>k)u~y3J%=B<|sUxG#WYr*Y zaEtBr9DF&n)%Cryaa;lu&=R?*6=;er6-x3sr=p4EArvmROR1tp#&8)m0$JrXj}_tZ)St zU$`Td*vl9eBA2BAVG-E_qMGnB;ZbsoJf{OFoFH)$Ee`DNwk2M78b;1jl922Yh#XZu z5h<)B45k|5Q{2RQi&ezNurNpfMIvQ)75NM6pYWd;c}k`~1*wH&+lCxs7fDwKEOobm z%wUQGTwu4x;LPq48C8jR8B)XsMWrDT(lj~IA(CcRxmY+3lGiiBzf@-^!JcF_%(%v^ z5&t1toGYgsbKsM-bfhG__9YtIKuD@4eO|8okg1rgkWL(o&~~*kK_y;2k$zS*PT-Wr z0g*6SzX2Os)FL*H;}-rtxhn)nW3uKWG1-VjeFFok#Vu1CI*j^DRD|$Km1D8iynsGdBDw-g5g>v*O zY^ZF~ioH!t=@N3g=vrcB7@#23%={(EUF>rl22}R20l|hBo(0iZIMM1r$EZfPFw=n; zPh~f2l3E6Sag*#dOg6~HVhkrO0$EqorSO>>9A?8UTJ}_>c?i>%%g!KDP(9MD!b%T` z=zLSi895Hou(N|mP7P!vX$H-XaCmT_6pYqqOV5a%W#S^q9G7GNjx2qUqjn$`wy&5k ztD@nG6$k>@RkkuSU#QAwW#T9e*;)m~EZrn{T;z7w+*f%ZvFRR6_5%r$-2{=8Do85zfJ5R3sLcbjl)nuB2y3 zGPMb-I!Ue#_G1q0o86%zNr9-Vape6Fk@R0s&8lr8Zb0xx5bkYxqwz2l6D=k4bxe<& z_EOL&q$OKnr-=DI3%vcG(mncy$2MtE!2$R(_ zhQX3sn_&J)Xj9f0>VUF0CC|*v_<18Ml~P!Vty!|=27WrxIODxJbV%3X2<1=|T9M!g zMG;nsB#}`PaQ;fPhjKyX}?|DC;Y3;=|7Wp_hGj)v?<=gFMe}--1T=r{Pmi6cXR0I;={~( z`m#dL%l-29@q~~6)9LAXrYJq`NGtm8!{S%V{dToqZ$BpgJ)EANI;SWWtDSH3Zgsny z;jr_Y9*)b0%k5@Z@3*TTcAFECruNV-{`dd-*Qd<~Vzi%j?-qyE`oGLOxZEFBBsHBO zz87pO{`B^br}d19e#PY3ox?U{e}%iYt{iHLFRW@m}rLHgF4H;bp$@r;cA z>c@XMt~QIi)#5we*!FF;S>2M*cDeXw`C+&3sNSU!Bk#DXe@#*O-G|K&e)zAvjrMZoiH(Bb`QgC*!&(h0WPM+&I5#ZMtl&x^aye*EVnDS=N-`E%v$P_HIv_2=*%i)t z^PF|1m?lWZ>r_c1*qlvK?s5zbqi5Kyd^-> z96;I6v~$zI`)h9!AoFU{S0fvmf+&ZUhvwUU77Arp$b5n@$ieImp;Q}2dhsp=;e;2Y zI>wwTS)hyMX4JfBtZv7& zq)4DzN2$;@iQ6R`TD-(Zvc9Wg(M&LrIz8FS9*SN(#Yy+C7s8~4ImD`mfgE!#H7RI# z;*4XJqpKWxqY_Uf3#wir;Zn^KKAZ}tqpOvzjUD>pEkYaYZGQ~R zBxPKxk)^ruGzZQy6t;zWOl7J{N|U0-tdNo_anpeNx!Lr=x!mWSL{Kj$?o%wAx+)aeeioMo#= zVwplF{3CQe=iaN+MhKjwKn`Vuo}6!=1+La=&`G!yg@ZMVu<4F-YzOLXvlC9ZX6C@x zM9*d1UiL{Sig*`ejfystlSInU&SLM8OOtyANofP^K(E1sPzUz(t~EIOR_R4g=1c|r z%y?wM^3Q3eaOIS0s?Az1#-Bv84yp04cr10OQtJ?~@du=$ll;<-$VXA5NCEDb=(6G9 zWUqL7*p-(Ds;OSR4Ni@bB`>3#$I@yYCdN%21{>6zTt|3a6B-#0JC%V&zZY%DxFg7e z90RiS0ID@FFpEy6szPUi&GgWkD1Hm|ZuU{(! zNsbjr{xNK?eUqqbl&V1>hVVtQM%o~C)#60Pk|98(u3>=A&IAAR3~7o33Klw;SX|0;2{i@qoy60z8{ZtcTftw3Kcy-?3L%jN*V$P0rjo; zyeZU)$AclRQO#%!6ofnMIP$`)>G9Pi@=1xi)AbdKzzQnL(PDiFn=ZNg$o;7xmt^9} zs)ov~n&d=jQe(N%V`U#)l)P(R4Zp6OWu$!Z;30{_*wOB#lWAAZ$jO|rEjqPe!8k`s zY16<)8p*ETQW!Y5XFZGPMUkL4BM?=BLdMl$7Vu=9aBXNG;Hv~8ic}AZCwZK#Bzl9e zNbnGjw5+;Do*$u{xmQ^vq6%$^^rNAFt9|K}n$pl-Na1uXhYzUKC z^2pqs@Mq&E$X*i;wj#R=Y}$76sR(9OaG1z}1Wl-};C&m^K))0d!oqLt)2G!J846p* zRz!*fRhZcw1n9-A+J1Z^NwWgh zm@hk^b-R?ttl-lK%Cp4uS0>r69SA-yCtMAQgvmGto9rw~>=~4eU<3w0Z;g-Z`q&p0 zArfjX=>V*kT+U&O%Jb*QVpamgmB{p}j9BIXMV+`}{#AWKpzhpun zcV`@01}B?3*1Tv>QK(MoPaNcR0O&N%TN1YxQiTnS`cmUAv3 zK2a)PnBqFv1U^?yHt$mG6`@XQn4V`$IZQt3ufpNlqPv9xEIBqBDNt1{hmC|mappj4 z6#2aT#t)sBQzuK;LqT+3g3{)wyHGy#KfxP1EK`72g7%K%cHIhUn zTd-}d$|_}}woyF*k*&jC+0aaC)U3K>IQ44Gy>zw04VE+eI5XkX03MHk+@o-d3lo26k3sk2V zNLc(5IWZ2z@qS=(pLM!(1Fc1Of!Pc!+#&KGsq-En*GZ+zEcYUkVx$$;XBPpE#IzRc zHxeKNPR7#%gd>sg$}k!VUojrzQ;3n~RjxsnuB-z3rKf&Q`VPGbg~x@**e%r>f)GENbcy@!o~ zU)S5)&FSv^-k zh(Ydb$1_Xp4huz+*w6RJ!-Zt{)ix3ryX^bO4`!z1PDuL4(-J%OYMlvt+^=TbXXhzi z@44=8U*y4vUk`s}S0haIm8ckk8}1T8jZaUjr(MUMV&^Y^{3ZVMX?vz_J#*YHHp>fW z_h+xstTVom5Nv#V>Urmmd~n_I*Ld?w(g2 z)XFC<#F{^>w_C2)%%Qx()*JKe)ADd2!L(7tzA)x?#P{xpwJ{^lWX`t+BPN!X(+>M% zy}djZfu|H}LRjB#xgN{;SneS>+rAtKZuR^_KT&WClq(4S>EI}A$BAp=rv2;rc;R1}{(PEmZ131_ z4)a?|V26 zwz{_E$~GD0rUE!?yk_qn@Q2%rfO~y7%=g8f&3}2A%FoI=vO!qeG+o>EWUDso`!#Jq zhbr%0e80CL7TNbd*_3}~;a~G_|M5SIY4-Ey=>01*pY!ee9rN_7m;3w2pGU9TdHn-r z$j!xRfKOF~et&*i=4LIcdwN&1>Ymr(Z>N*=a^OtB#@0(am>YL6BI zt)(I9#ORPwyvCz=f*1v1M<_kk*-LiFFW^KBSg^@bRDuO52Ai&$_)-E8Lm;g$^Mq4{ zr)i3;&4~3tQKY7xk_&?{=>#*f%`nvdt2?$96r&E_7)e&TANsZ$tdgP_k+M;;Y8Dd` z2Zj0O%rdFwVL+88< zKsc5efzE?g(iF3gDF&+lfD*)R$&8j3W+I&g0EHz=Pb=La%rh;%{6sb z@PW-OYDcTU1QZ! zQBu1Na<#OsLPpT5q|5~|#pFSwNN!DzB87pcic+wms;MBMRI-((S0aH!+l52Le^{~& z?TAY&2#V1Suz6K^PgT~_SC%?0NGM8~W596^W=<>3;pGPz0qI?kK6HI08Q%)3R1SPI zTG<6ZzdkmG0pEA;e?qo5r12WCGbDus?JQ9z#FR>xSSev5Ha zrzhZ}#Mc8G+`OxX!1h^C4diCg>DysbY97=%Nz9ja`bQue>3SEuuD6Sw03*>?TL>pn zQF3omVkwr+Y(hi<6_NB*RH%~qBj&&@yxMmtesqUP6eR6jDM zrnI#v;?F}QA(Kw09c#QRqk@r6l+ri#Ex2xH0EmP4Spngknl_=CU2wQ+#g)^mYbD}d zZ>iSKe{!fL=Tn(de;EHCM4M?JmI>4y@+)F&5(){(6(D_~9*b((SW`(3gOg(gJoUXyI z{^qs`I|5GHE>wC8k$TSFLYBGqh(49cPYP=l-SRIM!U(Wh6BvgyxbU zmG}xM)M#tqUvZLiQxWI+;hL+~7Pmjr^a%t5YUN_lX&kfY?m($dN@^G%2-{p~#pTP< zj_$FhkzGuGKU7xkMR9}%Vi2|y*Ww&n^Pav$pGr_4h<~zl!w8RTr=ieVs9PLbjDSRs z*Ak95i1;1Qtg=EQu`#6N0W3UjFVz_&_O<1cP}LwO8+=+op|iU)VV6abq=+Ya!&m@J zK(oJZk&a908)#i#90m z<`&hr8L6q&{DtXLvn8>srd^_q>VaaOwrK0xLI_AiD09IWk-(^OvD8?+O1z>&O_JX@ z8I2hS=ojckQHM-$>XU}JG&=t@C7-fTs}!(?HjFj3ylyT$C-W^2s&}DREVc<+@KSI= zNh*Q%dm={6ozE6Lu!K@Zz?KOv6|3XIH&q)Ed8Lx2h%e!RRSrRJ1yKXEP!~wz8luLb zFo-dPcAR+rTBOt%AyrVk(gsucO0laSq_$gfQz)q_{jQeii=~3olQ3p*Q3$BYMoEQI z2{5Z-iK{`q1Z1v*^kK*b#pK*NwINRy+4N8fN}b4MFic3&ho^uisIR&wbpF^m4j%?y z^w1QWNB06LHdO_k&MHMUmBsc4RtPat)mx_(xuy~_N)>~#zlajY6}v-mkW#Il)wt9f zcghLt4h`ZqQMH~-@$wu=3U^ks6lC`X0NMLH^y<%;GC@&{{sJ<$;_?{JD9vNS{F(l;q z5tRoOf^E;J3!Dy$C?PrE%n$CvVwbQW`Na32(i|#e2$w))I=#|@S^J)+9YPY{G(9wef<8H(ryIsZMZ-4#Wv^@!StB1Z^y>vXWSS#gvV3v=lQbgiqqd2^=66c6w5?QVJ5Cwk$lS+~BM_J_O8qdo7# z!+t&--VffwV>{W~=SgtACp3*MIznGi$BAtA_F0X?epBcyu?@abAV- zc;`UKlVHaS5PNqJ?$DoZ`qNHhVB>NoDHZ5&cUU`@=zr@MGFNR}{fF(^M&KYe;-}9q z7AD+R^CU3x8LIqt_0{^-t9i5fM~%dV&ykOefY8(a^%IHjyxpz~+c8r3{_+WrY%xWH zPq4b3-kc}l37PzIGylbYX`}6ERsH?zy)GsC+4Zt|zMqfl(y>q~Rr}r1X>;3ApcmL^ry_r}4yxUBl`-OjS3Sj^2OPl!3d{})k zoo?oh*j88NxlWtakMrLCvK9a8RHpT}_r5snHT7fBOYe_%jooQ~B{=K1)7JlM=l}JG zdAnQhKVOzWYy@7n%fw(WROWv@AN0`|j~dQUM&VO9{aeQj+kDw;o{p<{p;ibpQnrW6 z%6pGsJ|oc2Ys?;UeK;NpZh3OttAF9`Zoj#C2C@BqvfaP>etkV14m-WZ&H8p@yL(;U z|2Ok?zqWTg3Jz<_=H}TnulKfw@5Lj0d?L}lTJMi{4(-_F!`ZD{R%TejcZL))l1Fx@u8X6M_`0lKzbcayXA9Ux=-h#kz*1)`E{KS@y z_ykG$Orm6zOfio|{fm?^8YOj1!G^*wAU})Bq-UfY?nP@tp^o|*G3Pf(`xLXS$>M5C z{3oS52@;_OMM^0IEi6oBOBfp^AcP|dRl?JQVbN}o?wS~yA!G_dV?m8X@Rvk7LcP*E z&_pKk8w({Vtd7D>!`9IlHaO*~QImw5Eq4)o6XBh$>P#7kEZh~%hNc#Y@P@uc%3rB# zwpbwVgQ%V{v8Q~a#B=MBANzJB;&0bT&6GM#>d=R3PbWw-#Z6|)6n8_+MtsVxItq!r z!z@XrQnNC=Y9WT$ukXbnj+#8GL7^IWHxkS_J5dFMk*YBbmEK&*P#+DnKD>$=^DZ4<3@Z-{Zt;_whDC1&qek++oz?=kz@sh0%R>Qz<+$y5RTFc+Vt z>_cg9dkIN3QsF`=cN*G3I|nW$$WDd-D+wtVl){vzlGt@kSCyop%FC8ij=)k>Q_vzo zJ*i$qs>q?^U;@I^s1b?!23oTL4LH26Y;2E3b7(HLnSJIuMghV!2vURUl6HQzNWvkS zRa$X<*SaoI{R?|}+WoczZAAcBS_$K}V(s(QE0zX;UN*I|flrjU?BFtk?Mjk0i)scW z#zikPBjZh@?F0_5T&tGHJlm<(5fsgAv~7ND4G>O&SVQIZiI)$d%yxzmL6YPiDzI;P zvZN$FYz1-4Af1IdKSXCSbmAze&j?lBB4W5D>Q&G6=a`^L7&|%|zQ| zBNCjdJJjM~3^cM1PYfeFo&S`5Ze@XJfJbqm!ExkhMinaH(VE5~2{7>%A`DpL7if|g zI_(=6#rAy|03i&84}ObIhyu?70TnBYL$H)e2!zuI>9Npmx07c?b$BTc&~7k0r-mThK~B|JFW~4fT5#LUG0nAlW^sc= z(+)<+7b76DgbPBVaB<@hl1U{|L=tp0$a{(SyP9fvsRT=?01|nU$BR`*mmdorT44U7 z9Z6MP`@kZ3iY?4B97bAkL1geK*(|OBWCV8-0t|i?YvoF`*?*&c;wl1NVqAM0Qr%VX zOIkltw&)Q7I*NIl?5SXK10L5=G*%QCo9UuSQP`MJ7)2nb$mu4*_mGBO0}=CQ5F-qBRCSEpR!D$gBefDE z+nG7+T~%iAVQ-yF7Ji{Hjb>dmwrXagnUVq%CWPB$`yJa#8jlp=!8{v{j;rk+$nW-| z6hJ`+F~;_k(6b;$@GrnzD;R~wdNJuI%BpKA#;Ow6xUxFZRR;-iRB-F^;(4$IOoldD z*cG7ApiIU{t5li*%(0`egqnj}$4yZjJP4Lglm=01k91&lDzQN0S!>2`4Ac;Awlq}| zY<{ggilYQPv@CKMDqFN_4{C^bqZ>ZOT(2Bmg@!Kq0XY`{QIt*wz2j)7e$_qW=A1n!}#WD-QAU?QdgheaHQwj_a<$c|*vjI*1F#Sp3MZ9#INs`!yu zky<n_DOCS7cpD5*UJd-KY*xk_M1uDsZI- z0i3bZNjC@M(W_`g5CW+rVhgxzW?QMRmK@rs-UNaj^nzr<6iP8{B#MOO9`eT%q9)8J zF%as)RE8^ab~x)pvT7C`MDq2M%njm?g-TTAruHHf%utV=G$N@+l10fxTn}C@fRsiu zdrp>Plq-*ZO7H8pzJ(eHE1c-EcGZptxD^pfv`CVbg`V#S#o1frT(wfTH>4gu+_^S| zwT^7@PtCJHQ3yfwNxLLSsFIo%WCRAvyxI9KQ8XC(LG&AtxQOaGMEzAYH+PU6AD!(B z0Mm9}snwyr$KZ4p@U&Sy;+~!s} zE+i7ks;850qh27wes^-ADkhOKS7Q9f^Uc>AYm ze+FJ2cK7pfeLo+Zq4h>wj`QZ#nV0m{e&?Ky+xeHPzG;Lo3r=FY;NGUc*nd83rblPW zJls#4jkDz5t#5@TUEhnFcCt$x3gGa1wcl-Kr}*4Y+k-xJYFxjj^ySyCaKHQL=4Wha zd515i<8kvuGyCnd7aH;(pQCT5&0*(g-2!6$=lS7bw(BfxuXpoiQ$F~5-rrB#m8e5s z*z?@ZtJ~AI&}Zt>kMkrlPDkiETu{UI1hxG}WV74tyPxd81t^{m=P|vu=U#751^;_x zPrtgKudgS^-@e@op=V#cKONTBLJk+$-~Sy#@i`0VSN66x2EUo_?cd%`j~A=o-r8k< zuxanRkH_j^7VObR z;?*n+?felw+Gj|BNez*ZoepO_*^KQSg?%`loIkk?&h>Hq=Kb(2pxo=7&D(mR@ZH(f z_p6tV@z%e;e6?R++dp56c33b|;evOLKi;1)vzvLkS#DPVoX@^|hF|_>xBBvKZR4}O zyIB46-R`vdYP*@%evfb7{j__qHM{!Z-A^}@&B46Az1!`Varymr71HW=f?x~yw_twD zf^#VKy-?lGzscG<&!0gJ7arH~{|c>-W6vIE}Sd@PNaQ`rI_v5!57E`s#FJ<4=Lm#v0I1= z?Exy;_Jf+XV7Sz>g|bk#ipH)IZ7b2A$lx#|JG97grj+x5RBCqb*b}--o_nI!aB*mf zu?V3h%4}2PzbN6V6t-?J*)7aCB-Ky{ESYr;GA+m` ztQztzsDGh?4>?Vlm{BNCk_9Hi$W1k_N?}E}*H5T)tu*~u0U(Oa>07LP+Ng{pHK7EH z4o6(wX75Lm%@$&8WPUnXZUMyLH{xF>IF2wT$nQBi2|1uFnyflCEY{%SyHwN}#bH_p z<^pZyRE=<16l$&jBiUQ@TXkuiPr-_e+NZZq*oJH7F(#z`^v#GGs5Vy46$IO^i>{#q zxeqZkK`Km@QuL&RFRnawJO9YSNHNY4n+E@=Ev{YLlb#-@^&^xVqWNG%rLV~{W)kxLFkQ}reJ z&CQJ%PzF!pQ+=djuu5~8xFXb<(;~ve{IQlGt-(xSQ)SyqeN~NLSn!Uittc!?^}>8e z>Jw4P7nxzQI8eX{HiLQTZ3PT==TbtXM39%n^h<3xIfoJ4s~}Q;5=HE~PDP#iN>TkW z#7X0HzVJ-qq$gIG-eo5dQKd=}6&Ph25~W3hI)*9$qZ^G*y-b7+NJ76skEN~N0U_UJ zd_$_}{Cq76L;{2g4$LLYuDMAgN#)v9m;(hnT5xweUEvl^CrUm~PEL?r!zY79QynsT zN)~T5YM>?O>`+Lqq0@rA1ohewk(LOU zh6*5GVgqr~A_GJ#k`l~%i;fq`_06D+#LeIm@kH8cp}G*g;3KJ1twI9%?OkLL`$%G_ zI#TqC_DC&?o0thoq}MGNwYe|Y&I3T{-2i#)fuqgT8E1zYn608iyKd`N27N;!f_8Fev2&5H4gd?9oF2khw*yJbSqt`uxr z>U5KDS`oYz7s7mz4z*MV!NQ>oEpZ;^pKXK4$q1b*4ox^RbQ)uWcu-J@DQcaH0oIq@ z!-f};Esa_K7%Mz)@E_T_B!e!Ju1sfEFCPX$!-7&1q}I82>1=Sae>{}Ey&GyE>hf(+ zY)m*p0wzSqT*3)pPLi?%w1$M=N3L!4NnuW3T6Xjz@@zWXb@nC}j1nNlVvM08hHQ#3 z!+?iC&aOtxemli#W^9!qM{4=hKye}n%Y`Xlv*u>m?C5)+rVrTKhkA0|)8z(43pm5G@0imr{eI&e`k+ zrJ(zq1Bh?;m#vyFP?x}ojk}6?-6uqzCL*wuNFolME_saw#eT+&j#aA!M1^ZCCwcA~ zA~O)3H*{?@E@^}yeLwrj{@dD8dJV8nqw+fy*mSrY1`FZop{gXnEF+&a@{|csH!R9)b%MnqO^M)d8OvcwwFFA8DcwO-j5h8=u7oLkfGwB_NHt^Y zUYwKEWodDV*==a_R$`1Ocm=A(AiY%q5s`eY+Ul@SS2I^Gx#_@k$)nOObo1B{qo!JN zu$TOUk2&Hu*HDiG|$k>SI3jDJylrK!KPW+M5utH)t#FJzr)yAsh zX?b9IVaR{uoW=F@wly}=8)iP&^`F(^}YDpYf zdDTd~^X7CYH$nAa+XfBzhA`>S4vy-cTyc&;I=P zV|M>50jGUEZ=DzS%no`pufCp6n~hMDg8JT_ZVQBPzTYhzt=IeA+nd#mlXTiu?U(lt zht<3Pw0S%@M)tUSJMC`{tM7Kl{k-~>@OV!In!1wnX-+EF!M zkE5W3(z&g~` zFMP4~K-22_w0||f74G`m(_yo|cUbE3jH}Py-t8_5pi=*_zMn4&PVx+ewF~{ryghtg zaG`ni`|IP**2(hh_L8SA{s zg;<_*Q^3t{*1L`PhXn=I;QEg>=Et|fuNFGjC%EbRTkQ6ZE?sb+_D(iBc7xQ|bL}2% zmHFG1(Ja_*Te|BH2mkl`+iAN#l;@rH_qNDZhxvN{?q84d>a*X!{NeM}^|arw?McNK zJK2ZA_BzM&?*2+p%lAyn$2(hhw|DxY#>^kr{vp0!W=?3V2C?&q`64~Tly}$Hr-$`I zH~hnHf3@3g%S%7lB~K5lD?Q4!Epfl8zV(eg`F8d1Pt*Q5@AZAdz&W57Ehv8eqj^xT7X~Q);0<2 z*YoA-htq?jY@blj&q<}TAVV*bh^)k7ScEycWeW7YL%T%W5JVLAajgN`Fos)LY ztNna9ZH|t0F2uO|d3ycAM&-Q)5{CMsz13n$+0oO}09_sIg)U;p`JAAfnb z-cQ%hX_pH@vyaa#EB0u%o=L}t&a%h-=3c#;Q^ z$&-g9Vft86KYAQ$yCOqjJ*&qWGKVf{N-a=v7{odSNDkpkpoFYyw8a&a=1w|qQAlHG zE47uTkVSezyFTuFrR!PQ2-|=p#Em`Y(ncRD7q=7!$4a2YGiv5cP>-U#q83AJ6 zVA|r}0P0`Tb84&j!F`!F*1>m)oQ9##w5-IKIP}1vX%Vh+kT~UIC9LZ~N~`QX%&Y(a zr2o>29ywEXAu3bNFsrUs0eAm4NgbHtxlzJV6y96>P-%EH4lrwb$|c%flA;4PekE~z zLKv1pKUNo8cA!O|P*)!E&|sxWTAMiO7Aw#IIUsIP2~>*@DETPP zA)+Yir9DG@QoV&CB;hDRyGppIDU68r|1^6d1)zLN$`%wONMHTbD3a^yRzxT+IvaS4{FY|z50*ND_|2BgCg*GH9` z=`W>ek^*MmMiqMj3DP+S6x}_j(E%k@+NK>SjC7PXaRS8FxgF}2)vCoE2{JoL@6==1 zxgsJ;MB3uTwtwX=GNERaD66n@%;`C2SGaIaqDClICwE>`l-H z3BP`VlO>Z%uGptW%+9D@(NWL1C`E^a+RsWM67H%~X%?jxlGyIZTAR^AS$f;&4Jw(c zlE8xQ0enk;DzQ6K%Ci;QQALkbMm`1X50ezIomJ%LsE@xw1-K6>O({B})jQSG8WIN| zYXk>7Cr9LxRd$sUqJ@4032}2KDix*cM$~zhJ7tytxsz9z<3$+~QV0EMHiLb(Y9XOU(EbRF8+ zB5-kiYXw;q%yDs(yGnPp%!fHnRWC%TGY%bgCUv%gppVQ}iQFteB3qPdaTA@nq5!QB zbuSjQB9Y~?r*+BPyL=QQVD*vR4r@P@!~#M{E4NPKDj-C zpQw9-G<#wv{vl+{A4l+41^=j|Hp`ov;cLKq5TLN?Dl7q&+)T<(goKB**orK~rEP1g zgdpyMR3LcFELk9R8ub!qZW)#rR5f~*U_^%w>WFl<7l^)50;|evklccFK&cZ^3LJr` zU9y_d2#srQP=~N)4PsCIx5xQ&cC57-X<>uVi`W>P)k$9eqSpgg4Vq11 z$idW<*?7hB=xB-|nu5n@wmkO;s!Q^a$g#+NjzS@@7ndw{gqaIy#u@LT0~y6s)WAaC zp4uH*XDXD}#k4k{G>U^|+o>;tQrueVp9h~ye2fq(4EBO(gIcAdv5VHTJ6KK-5*a0Q39)F7x5adu1V%vK#FS>=9X4@FQ30*ptJZV4PRHi42zt8ZE8Zd}z3qbIv(!M?Rl$)QOk)K)J;Rx{@gb@bUT-Lv%Zktb~&w8n11qz$n z!FD@@FhNbPNIPMkm1nE#$fNC3m9{K z#fM_Lffe1T;{!P*+fOJS3VRcP?L-~m0{D}+?&N!LDcAczYckl{Li5+zxm%Ys+&cI) z;)ZTTHv&hs?U6(%S=%`+3s1DFXn+%H?zDs}$>SLMnh@7l>1anUy}Wo#m9K{rPP$k< zb$g3KB!lBNik^kex6U?h6;e0OBA~TGG6OCLgudYb@+9&<$BNY5=R;>$)rwJ07ai@e z)m8kpAcu?0W8_>^>%|_V0v_zqh`BN-R3m$>o6zh^hB9+kV^(0voTc4Rp;5~Lp4&Eb%sB20uiiJ`yKPrHVg@#03q&x&PoJrH(kz><3j}p2(gUBGk7^tI&h-L&)PD&it z`(accChP}uH=R@k?w%zv1bJq)v{O7hCTH$PKeUy;8|}GK#|i7p_GoZIR*Zr9ILSLv z-BStx)Swa+_+Y|m&;-%mbFffS{#zBRRU^{pqq+=_KgA$gmc|wgBDR;Gyk)Lwr zZuRiQDUEU->wOw@{^g3bpz*|5m*|72{1=j)s4uLA8nrpvW>T?;(o1Ov1^-6L;ZeR$ zRKXDbC`F1Ip~R)fH4au(KDrm{K=#PQhMXilPMN zS2r=1=_NakR1#){rSw)$Ru;cG7~x)d8R|uhs+bZ@{U8`Z&fT-W|NY?J|39MEgzWpr z-Dz*vU2o?X-|n`@hiSi^?4iqh94B$#9#>bV^~Mpd3&_tQ%l5=wwwKxM=f?ujwS)a{ zf4yIQ)(_*~{`%_pxpM_yIJQ|=I6i?#!`P1t4l~zkX+{Nj3JFTt< zI$W+^F8?@fSNmzbc{@F>wl*fa{p&K|fB5PTtIz)M?x({%omOA{7vlGR);rTf{^TJNtnXYObXRJm-q z+5+YDi75ctZ+C|$Wc7pn^V?~^($&@rJoeSX-~8%$xBA2UaJmu~^xJ8BEbqUWr#JJ$ z!Ycsz{cd}@KZ8hp z->#3xdB4{J6m745{rd;I!L<5fIvfRHUZz49=|tE&4&&!#BDeEf+iJ(B_4viwhQ)UG zyMNVmY#00BF!=>Y`^_GaGS782@f+xo{x4U{h3o-h5&k4?*G;uKt->^yv@ z)t^t(di8}p`{waO!2Ue#ZF|OGt44eBw0~S6_OR!cdD)4Mct20}mWR#mt^M3_>1BJ_ z-d#P~1K*wjx1st1ssL(WmxT z^Jd!a-u=(D1wHJp%>u3MpWKH2Je04UFx*cY+5#*6@RROR1xjvgbk)5GD3DG~zQ`%# zUDpV;8^qHGA&a`UsDw6%i=@I-6z9B`a7_*JZAQO6xXr0gL?#yH?IE5oj}0a~&41{7 zDh1jtB$7=c0F}amJ3`WO!N$eZsm&jvPKlc2h_dryd_~}@TBtjEk<<(#XNKn;om&>} z5d%JNmevlHjNa)yds}YHE7*6sWUXXe1)>6efFsA!+Rf ziI&!&GCHYoB#RKM2x^IL`b5~V;P?!rCJxG$(RLF^HdWcW7bPtgSv6Q-FhI|WI|K;# zqC|%fC)%inPn;KR3VlT-R-zm8K(XC>kXjDu`$SQpqUN5O2dhk$M|L*37>bJBZ6I+* zi&HQDpb&T|Yk@=^jk?_lY!cL-N#;=+Xv0W4ETgTX3Ld>A;>WSVLzJqQ5TKO#^X*7; z+}q2m6b$bo)ObYzT{4`iXHe^YCx9ES3`uei6>!M28lt=f{FQoj0ldf85{52brVPy@u^W~l~Z zCyrvf91)hQ$r`|drsQy1s*nUTxguMx2TX(#)lbsp&@0Iqzj{%>O3(rufHGUP_MX^2 zW!wA2TXex(Vl1(9LckuR8yebAv_RQnX_?H!e22WyrUkTmMR$bWa$%v5544_;TL8QtIDrGl^qmRn+px;D)(aR z9_r+)^i<6!gp<&gV~4F=X%urZ3GIo&AYc%tlVGe^n0!UpE~yZtrNnCP0X9d*zA7a8 ztY;Bj!#ODW!8Q{ST<3fQ5QOz+ZwXr|J}k=bCS|s%QO=yN+EXp{r%{3#fE6(p#7ff#C9Pd9ZMNLK6UF|99b2~_C2-Zj~&Kk5_xdxNcn z4gp=}Vts-a)_mB8356uP!MYoEB%{J^aK(rU%At2xM;Q#GHc3L7VtYx^i&1gL0!OsK zt=uvKkq5y#d#bMX65;``K#d02!{el9vBFKb7u$1_M>h_A1o7K0HbC;^Kj@qnw;n~? zx{Pn>%$OnANn)`>Lb~T9;u-p&RWB5Ujau{_`jDd_8^~ysTKVJPTX&r(ENQW-BHIAC zM_CNywDZsy#!KXFW5ttG#>*^lPyVa)dD_ZPFAkhAXyA}EzEmp^QF#}gGUUXV_gr18 zI0a%KY)C9w2N|kPtK};r8J!)ysimG}_38{2kyqmpC*lIlwGv_v)nSwPydj%lOF|sO z&^cluxC&}u!sPWrj3}PNo|?tY3vcY8#m}_5gmgqkJu$6tJ5$B>2)o;Z zaAdra%2{=`_@eBZO1R!_ew!mBy9HBdtGl>^peh4(7z?XQa6H?n=9__ct#Sd)yG+ZN zP?1~0I!(>dSSCxs6JpGyeL#%s5_qABY+JF00(qCKeq&3ZGOktscvBdVGoV0|Y7F@9 z^0H9SL*onR^u|InKq9Moq=M3v&|yHALHQW!?$vHbW$R82P?*EW$tkSS;ct7&-?fAC z8+2w2$_ZenoET3LPNKRJh1Y5vs*;3dg>=~Xpe}dpjIxc<_ROG?%y1Ipb?!q~k(ju5 zH3HwMdy2WOf)F|Vs#Uwhuq`)8w=Nm@@uZKJ zwgGHCQ>w{5-jHC^jH9~SXpstJIKt>gbzn9`ziVgPfVgJ$@gte8BCUw9J*Ai{HPB_! zk!ak)MUjAz&oQcUM~aL+2GO=})H#A_uF!YLTEEeb73=)&j{(Va%Pn4qMKo;DjCgBJ4t(b+6)7M+%_oqoERN z6sDHa-R%&RiXhk2>~2#jgD$RIBA01swC3P(6{TVyy`l?jhET8d5CMB9c9XhjRc5_y zDz1P3K%jIQcrlC+!;#uw5{k%rvXxqB_5Bu6!;Lzzz`g97Y~~WqkR-?@7N$BiB4qMF z?2oBZhA4lxZ3VSbJyZn|RVhKu{nVt;3UNsi1oqbAWWhF!F;z`W9wNj(1nk?6g6R{$ zTLib{GAS{l?tAbPJuQ!&s!H^brHKlYdg7;0A2yXxB^OkcKFXDpOrn9$Llx$irDY2U zn~LphX;GcAYhA$vse;X75-EvrAGxVQy2ZDxyyp_?bt3lv*b0r8T@t}=W+YN(u7ady z3(_=@))AcM~)Ro=! z)vNXO#Y$wf0@Lb~0$}{{-A|kK)h{l@nfiD*&i4yX>)sKWSL^G?>y1b_>t~-fhn=px zJ=h)ZohkTGctg(^N5N}_Fr7BL1zh^g{OY*+V%okwf8~dHy`2~8((S4MVITIp`+4gO z!Qxb%&RE#*pZ+dpRePY@LRP$)%4IH=NB5VUPPeDRoowUctftC)z6U0jF?)~l{BC`{ z+t@(=V7J@-%cEWNv~}>H?)5g?W6tvW^MCSn04RdMyLHeU97g#{qi{9&NhYO z+I6t)vxmDVH=RC$R`!?q41ZjZy>>wjjeQ(ezniwFY5%D4>6hgc782wyKVWG+)6Sm3 zs?Ygjr-z4q0XPfTxe=4Fc$PmNyamCm!4s5RNL=%r|Mu6{_p8s$zW(j6_v_D{Bl&dP z-TNcn>s?p&O=p!B7JO@?e|}wOSGK?G?|05wyq~o=Y{?u8qIenR<=fMGF9zK5CR&^? zL_B@t$m`W-|M>NfpF0%44E}WV?Q}S<%Y$4msONwC@BWvzvyI`yS-|7V#u1oaOXjo? zCST3ZAkf8dGYV3`DQvVlb8GHN}Ryo&)fC(U~_59YFhg)J07R& z*Y*zA``w{nd&}6Cm36ram+!%@XMn8iMcE2x!d>mwyWLy8(ZN>I&9rw2|C#-H-tR>q zeP7V_+ts0v2kYyvY>;nu_x?-g?e<0Uow$faL%nS5gY4SlJnb)@E_OUW`^D4C*r@#W zhu?m4@kFNGKCwcV8~@2U&V64U?`%9Qs#u2b=U{JDqp7G;N~CA5S;esQqFGugYdw|z zQe}z(gDz#h@`zN$<`*@j z=&h#d5UZ4vuTrfb)TbRaBLEy#{*gHRR8J5Zi(b9!n`-?=?J9G;X;*bwa^IwzGPvX* zN<{Ux-B57EqI_b_R9UW4nRMS%+By*1xEv3ZQj)F1G1C%lrIAYMv0xz&7GHu0D-tPW z=hzTUYba_I&=;V%m58oYi-xssH?1Ju`g(NjHZpJnQ+lac;1l_l+>QvUmQ5tG!6+s- zVy|+LxjFWPRx~TxS4PG!*0y!zaLzooh_8%86gumppLCbKwm&Xi3jrI#t!D`mHw^Mh zV?~@;qBF%>lo3#6o>JG+^~9j5wUdI%HyV=;2YBd)G>|~5g=L{jggMQ2NLD+rSa%D6 zKhB1JDDMo$#= zVtkJj%64MrClwhxi3BgCNy%b#fFmJ`FNkdNAd)*Q!o}E7OL9tP6suOBQqhx@L-sof z5YUwLjQy9)Eot?3jZX;K@KF^=;HQmhB1X1FHS1Pr)5dX_(k3K=#FR#K3`Z%DBE!R= zG6!X3C1OtqhE5`qU{J)ul_=3aNTvb3EW2?fb>yQcwkM=r7HqS|R5nsKtO(Z2!{(ZU z-v>9V10IRrZKCXyB&scE-RP13Qu@No@@o@?^eIVc7=q_SbMBj}LEcAF>1bk@>l0tf zwut07u>ukX@sh&CYZj5bTdK6kXcMJoMGOq#|u5i6Yiq%s7y=vsvZX z0ohD2m#rssi0H0|o~BA<@noA+fQWSxxddE8>MQD+B{3@ho>03gzz{p6)SDCeTB#dt z^Mb?U%NV@~DU*|stty1zQHj{KC$WRh^+pH-YhGK6u!2tVsHESKL#^LYssN&ShvJV? zwFnD}!Nq{VUzGr8Revc4SZ_#k*o*p@>g!TfBSf?)W7>-I!kxV!lqMkvY%%nRAQ1RC zA)T2R{4+M^$WIxO)jP>_5dDq~r9@+pLbj=NKz)cqpDQ4bFJC7pr%IMe2O~vq?!Z(* zkDCq)(W*|5ayH1^fD_sYdrgQW2|*Aw&ZsILE z-Jps}g==dHpp@FEI)~t3jW{YsL?-k`I>kt|tXKw8GD-DF${C2I39LtUHb+=O0)l1I z%?`GNM&!E+G7&w~rHV+EoJ|j4F<1;XSeYXIPHns3qS_%Nr4In23DZJq+r69)B;;2~ z4jfmpWLI*;zTcucUoh%Zyn?K0W}D5RQH*Zx{J=V$67;}o_b}FOQod9dvxTV?+4f@l zMV_^&IxYMl#vxRBk-`noRWPoy9NNB0AvHxN>Jwbq=@a=jwMX^X&1#`XAAzz!SjH~1 zK$kEE)KB7mXhpW=>mjs5#R=`U#=Lnv8KN; z&3(%d;?co~$d0e{H_@rd4JWV923H_nN7Ic!!Lk7nnFgT%2MU_ZA9yY?H z@nX{v1_EyiyUo^1O?44Dt1TuslSChM`U&NG?M*_}Fj;&Lab_S+$(#>Z!a)sDk`IB~ z+lJ^UIc^q2EaT+&slgMnR-*BM>QR($hct;9^4aCq3Z-fwvD%kA>M!a<%tk00NqQvY ziwmjt>wNV{jhVnTF*X=9h)9|oF(o`JYUFadk7}4i{QEeJ9Ui@Q{K$(znnE3ofa)B? zjDe!s$xi-?JI3#CTk7Mf3YT_@n{8l`Hsu$)2;J#o%|Rgni;4O!!trPOGm0|^*2AcV zaysS25K~o5^(L~~5J?6RTx0*Pb|*8N_f3sFYzh@(UsZ6Hldp?(siay0l}mznQm8Fc zc{~Za)V3)|P>(ZwEi;0vftDpVyp#&o*TSV7QJXWU<9LJ^OPzkU;fRh_0y{+fs?%*G z1JM?0B3i-1kXLKoUqzGzb%uO7ufO{bivgQ-(SfE*j?3t_Qb!j~xe zWWm{A1N|M|FsPvk!ZlIm^hna>5YnciBq;uT+Y(*4Y-3wRU&X1kpqrfV z^5R76kd#~{VmLVp*u&YYN{HkGA{v5-IEYs_!`>0e%}C>JZe~Olesy3_PA=I_2ggDr zA3YWPr4N8wVlEYp+ktBG3e!S}&;I=P18x7f{rz?owD)nUEi+@|LXke)qGtTQLmRD6x!d_)x6ogUB$os^>6?C zmkXZl>v?y(pC0ZW^)KJ-3J6sHydd@ttBw9xP<-3nUc|uLLUKE87S7ND0xqAvxSF?4 zhdb@B&u_P1ivaftEW10*f`T57V%yxDgsMHh-Px@+)AnXTSHAlnHuLI-X}vwZ_-1!{ zh63G8`&+x1e_lYiU#^euemaA2g$XX)yqg={L72=ZC~1LB`qTVEsM*u;PNS-?f3>~c ztPgiSvfFvH`s|x&_3b1m?&lhV>B_#mift>X=;||DUjmPQzPb^{_PC#?`$E^d+~#TQ z4iEO3Xiskh&9!@7?d+ac)Ase@gTc96{oej_+H5Y~|8_qMO8Z7Xd}aT1@qP$)Z~fJF zHahxZ+d12D2wh=2wugN)uMYFH+RXOz^W)rU5zc-a$2!|=*zfPwo6R!jf~L*|ZCVCX z+_Ssw@!rO8KMR>^4xHaA2{UWATGr54f6+9anS5=eHjnxjL8$#TzYvS9P1YB?(`IM8 zz~AKj#~(!mZDW2ozxuUvE^B4)-q@qw&(qo{oNXL^`+LHWj5zUUX)@1~pb%6g*tji_foBr0CU>=(Aa925Vi-F~Ho@s}rX^V3@j zX1YG^_99yrVX=Ast6NbG+in!o?fvdXJ4Oq$OyL;?t^3DDxb5rLZ`X(U#o=MSU(m_Z ze!sJAB-U;R6n}Pq+ODtH4>lfGyZ!ETdw2N!(stjjEOx)C20%}ihw}h6PXl!Eo{ZY< zgh+b#PRDFdihhM0^GJ|OX2CsejX=oRJ$yqSfoU@(eu5mMx07Q#?Q z1+fN^8l$q2km>0|Qqpw~h^`A(E>gLPEpliPFH?TL78cCbQVK2;UHo zCdQjomF#tsMlxntgB@yi0{4NVrlVS7$v!j;;Bpl!G=G7`6K> zvP=y%lD9(Efk=gr5_TKv1kQP^bbov?SkwdH^Q4Ft*|#Ln+Et=b9*{{Hg`!>cp=ChY zr~)ghZmd6>fhG=H)g9W8LnWW}(aX&Bn}l8oOt46{JxI&Z41*ayWN9-S)>T~KWh1C6 zj$G0)STIwySuX3e>J{*;OgD^_KQz(rEW{1pMJ9Ysm zeN+n*91mhqv9V&$t+rN$tng3le>zk$RyQG3@r946QdqvWno-C*%D(8up;!=F>Hk*H zYV#75r{e|{VM^>wKvZrTIuu}0OqNj@CUyPD*qt%_z_JfmxN5r1VK7;mC}c^iE~&Zu5b`$TkL6XF?Nxi zSeq#Uqw1UIlrmx%?B)T9Vg!_HYdz5RQc$9f?Z6;v1x)4--mHZ*WwHWbgLCUuacpz4 zortgpu`3{hjjf2f3hAHp{m>HxNQK!tf!cT};}iN&P6t8NeX(H^@T_aq0!7Y{*$pc0 zh;Rzv=4Rq`hQk;-5D<2v>#S1(Gui2iB@BU-D{SZz518Zj@h9OjHN7y1SRNGpTCQ|p zOL0da93r+Evmk6t+&VHbooWTxaFm3zt@KU(Og1;PN4mmJkOQRRi9+v#FBUT^IEycj zDzkQqk5nj8XoH<1wsm?W&{XSJOD`^IK5R=(S)Ceq>Z%w+zJ)o>=H)}aRZ`c~xM&uB zBvTW_!@$gbwr3lx33TEBK@L0HREXIZ2dk9jV4aB5B3J2x#-#@a5`U2|RMBG=YnFt? zMx)gRv3A3|NJN4#jhb6T@%F7&Fw})bDV-8ZA`D`_2eTyvGwOz$XDXH;;`nx{3hGa? zbqDVu^DBBJ1idx6mh)YrBBcR%yv81hv(qO)8m(03Bl$db_6+C;+ODoz6Ur?l%#172 zIZJ3w2ocbOu?dPTE2`ur_UT&G!ssNZPR^S;!Rh2}9H&JEH#N*gWEb2#+~aFKCl&;g=?=rXdORC9Np$S zlG?=}2BGTJix+Hrh)(4;VwbI%*-(jSmN^u*fC2*@aCVqNLDRer?3}gr+p^w(2tJWF z-0mO&C-CPx!JdiGE<33lH4&OW9qMQ)lxiPRkIE6H1cibOROudtXrv|I+Oo=o=n!tM z!=#u*un--S6eOdhRUCVJNy;W=9sLB4F;B^6nA#rgylk;sCdi~Aaxh4+1-*ihV~=m@lBlq;gaGRnD&RSn#IC_V#v(eP0b zSb(JwIVQ||8!DBAWw}^DOjZ7-gTBvl(hMQQFr3;JsDE%t5QSA7|uAkO%vEX$n zp#*qU+lWIT>PLZ$s!ph;r2cwJdzt|@&kBc^48wXaIC!+rX8h z;1Cyd1m;Rmw7znBiffZej4!B170O4_wKgZ14hHR6!i7o%F=LB^c}~jo(1OnrXGA6x zX|B-LP3kZw9u^C(b4BG?@+JbA(y3_AL~H{RXGp~7U`D_;A>rBWY1K7Nr;_rQlml;N z|7m=6yi=)0^4tw^*$A&%6*A1Y?;C8^5YAhvgE6?vZ~d+`eLTRdY`KB!5jZREjF zU%!P{V^3~MCn99x0vANBMZYmx$Z6&QdF$;&_;_XckuPTi45Q+Rp3`fi1tv&p5Il8F znI4fdASFmA0RU<(Q4KO-zo~iowTe@@zl|4$I7uZm;7Lic*u=S_o&dC&HLSNJHq8-9 z6ck7?vZ^Ea9mWn-G((ac2PU+M53&qmpHjuq;-{Y7oNPclDmTubG+w_ak4_nN*e9c; zfvC&NjviZEYi}x5AHpKk_ew)NW2Fetg)fgzhHDo2%kYB^L z;(+vZYOaO`(MJ7=mFB{7N@qG1QWnhvM2}{e+7_Ey%#`(!Y?Eq$V?!~b@=iHnu1bLl z?NDw%;RYDy_4 zm_@#6mmAI)&31hDt@DF@H*JpshidY|8O}K*c!fXgPV;sd!v!h*9>^^8t01M<^R!)G zKLMjfT)Ute|J}R)^WkCMzQ;^&ro-Wh7uS~5b+XBwHPgbaF2A$y z&8t`QbhJ4Xnsv9ayVyJI&pf%W_Vb@l!rtyDM{wV)gm}HyTO3}m&+mG_-uMc9JMHgR zhtvJNUEp%{z%$g-#^Uz<(B&B?X0D+U)jFc&-b&hSi9eTE*MUG zTz|~>5bCd&ZEQqFYsSFY zant!r7UC`6+HL(I{58ucY3eqQzMAYOf}IPGyb5McvwGOrJHU0U^rW1jyJw2e+owc;nrV};a`?uduUnclDH7eX>W z#=x43Nj+?sG>LX#V~dxxt>0b0JL{-WmYKANhg!IjT_u4pz(rN9D3plWcXZYsv~Dj#tRrtqDF)AG0np@FT4aR`NHeq%!| zUJ~$ToNnq+$VYl;iABBy_J<<2eB4BE2d_8`2vU$yBGr#x0(>pJpEO-U3&Bb-OG090 zQhKj-T7BCNYB8mXlH@QV#g9_@Md*asiZj7U1nZ3z+oHdSEfiYq$yNDaxwoAZ+WsD^EUO#p2Em6(jsJ+D90RX~K>$i56IS5ZR5vN-iio8lr6_tt;>?oC7oiB{ zD^eUsJSbWRUQ+o>hPNtLy5*GuJ=GDT1ZAtHiV~y>Pmb-JjZ%b&xV3Rk)Zrq`SFid8 zn0Q+*$h2Zn?HIAhXhBS{$eJvKG-5%yM*cx97U#EbZEVQ;r`&a%yKDt4fJ4d7s#>(! zFV%IFC3IcMsuEN3hax&AjAMWjhJTO(3TsM}I#AE^(1{6`pl)gep{ccdQ=PQ4G4IL5 z2XR`pB#GR4ix0`7nTIe$Dm`ZJ-J;BYqdslOZS}Sls@zr?65H>UigY>fJHcY$C#oY5 zj=oo;IdMvjP@IBAU^?KwkK#InJ$)CY7{ePOMO6;w2sRA zLD``fTM!^`qMTfC6moXiz4}6fK-&ZZv-M8>g(~?rRf$AzZGJs6RH&tbSPMm>ruKVC zT@Km2ZmO-ST!^V;F(0G2r4BMY^9LbC04yPMD^)%?2TDd{6qI4AJ zdU+_X9uZvOn8QFGjoRZPMkuZLg-QEdq$FaS1&4evBLK%lC$eG!eCi~|rVS}iL`oN{ z?I@W)03op`hFM+1_CKkssZ*z|Bw+mPOk+k6pDk*O!N{vH2U{#{2xS=BTGzOERnnk= zwuWBF1gggNOv@``8w8U{AVE?H@QE$NL|U2Zy&0=O(_f^^!oCB}Otc-~ zO~h4CvuNikqaMv?6h#qHr-mdT3H-#H*Px^>x~@-<=1QX;6_JuQ7mTolR%45+OD*Y@ znzw&o-zOZo*KQW4xj;(Vh+%NG|jEKGtvvDT`~xW zL@nbd<M207^@hgnxdH2SFxs#d&)i1n#t5quTD_sdw?8n*LEXDx}BI8p>0mR z&;YXJRjCGy-tNm`jUGjbbj>3{Nns z-vN#sW@pn)jfbn(s zfvJ(@Fy{*6V6R*t8-v9iIZr~*#D~iUz|2IxFh*U=p;c7VMJ=p%)p_q&f$i#RH^H+=%+pn;cV;Y zHSce%c7YE`Q0F_u9m*M`mQdfpJHUuJQvv~SfT}C5Lxu$NePdNkS`K}ShoD9w_!;SL zniuaukA+pG7hNR)2hP+$U!71heC;Fo}{Hrho5Tw z5SAoi<`tsf@E;8HrT790)tN5 zSd?T+DnJIva4GpC0Bt?Mecj-w(N$*=ozOygCzPPGtX@z1>-lo^t21g**DbiY z0&iRXxVdqd+wHt{uGPl{CMae3f3XK!AdRBP{SZ&N-<|AI?E=$l^Ang*7caLEB=@k~ z=+CFM;G?_yd%;F^$B#jZhj}9@`&A@~6h_6*#dw%d~v!p_(0hxPG^-?SH$ z`Ch2p?lS)Euj}ms*PQ<%?Bu+?etHz4SYO$2+Up#S`?bCAY+rjbZ$&10=VYb3WekMn ze8!IZdMg~GAdx<)_v_4SGN$E!U*6oWw|e_0_T(=`wR$UD?^X=1`qQt)A-lP^WwO#}E$HpdZhLEQbvW+! zj~9*{eK`7%6rk&sjrFTDT=k6V*7KHY3k$lovCwW2)_b;jJlOr5IQWTT__Z%?AE=#e z9lN-`w!WDQ;I@!BFMO^R>_5NDUg+za)qDEc>2%y}clVEHg55${Z6CZ8S?$&4v5f4i z^H^X+np^w z+i=(GX=77zvE1hTP-c&wn7-43%bj05`}B9GtvG#Sv#bv90Qm5w?f>5Y;Q8E@kKgQ$ zPiwvWvwkY1@Od(q0f?xwLoLZH9daV*l4?AWznQnxBbrlK(p2GBigOZbv~1(Wj-k@E_Irq|0ZEY* z>jI7-s6{-GkX0uYEdoKBqguIlrHmHM=S5M1ki4b=b>()Dv>b$6t?QYSbD{(wQsvp7 zsb{vAhzt1jy&BX((N-C!p<(TA3vyF~0!^JNI*<|`#M_9RDpFC1sCLoizQ&ul=Q}!$ zTy+x*=~yURXt+ds^o;xE(Fm?N=mN~1!E4eQtWL?shKcuA_o`U6O{+- z^?FKG%feSwyRv_Jzk!rqMJdZD&zx0I=U8^M(ITTMx!{VAzHzC_*dqymGy-f^s1tt7 zp}|y`jbmmm+SUb8@tH*pN2g*E6(knO2Pu?O0?9exkuy4uKAVtl^d41cb5KFDsfe^a zHkZ&;CcM4PKD~UfvFs{R6+f{gMg%J|c*_zMaK|AbDV@|Bu}bSQ?=A9&*dY+H7?K3` zj9`Pu(Q_7%KN_6>0WQ_cV*uQkn#hNbane3oDT#a^Zql|9%c1haw20`Gu>ghenT0jK z-VV|-#HyUL7lN7Xh0LuAe`RHt)kqe~d^fO;sUKRZbemlkc?un`t#Bulj~7-Mtz*vS zOcTi0j$NB`&|DJ+zafRBm~Aa$H9#x-V>{FoMrw&okuxNOFEX@rLj}h!X0XY9g}p8A zQ|hpOA;ng5bsHt!i6vG_^$ivSlE7KizBeeJNOdk7#7K+{=J-pT915#K(_$M#Y-+9( zKaHbkW$Q$0gffP!P06{+Y}YWlBdBm>0cK zrk*jg8D5~FyVyQNHMpd5DOKjYYqTRl;v|z&mFC<7w)NOz^PMzYTEYR1y>P}9+4U}L$!>3k{jD}#B!l|MAl4E8di$L<%8HVv97?;#0xD>xawFc zX@3c~*N$Q`0Rr`G~|Ff{V(>EMuFlmb2^M|4JM11-W8WyR11h|=|?ImZ?~+h zkmg0KWgIAJybzf@X0@Tx?OWHeL7S2)=w=D4T~8J{Ptnm8Xe^T2M6UybpzuES3=gy8 zjgOw3N0G#m-4eJ4&J5*MYdwUrS-Lh1kOJ@gz+$3o6s~sO_=bdJqSFr}veGHH3GF35 zQleJ1N6JVfr{J|h92Hlrsn5}J!kJC2SVbBK^GYoq8>ZE87D;1Rg6TCsuJ6>QR^W5{ zArK=09FvlsRieMWv_mn3(AAr(zqD;Spfcd7iVU?-dS42WW(rs2gbvEP#tPuRV2y|5v(*l=rwGk+u9|O=3oI9!d@iER%Mj^ zlObE6NJ@heyCyrEz1ts|-p!t(I+XJ>5)@Y4DZ)g8S0H^Ga`B`3HNnpwg*YN)yWAJd zFr{7;yu@HFRXgM(u|u(^HgdI`+U`MSRkQy1i7V>0m``cOS{Ai&LGg~Ml*b`s`=RE& zBjO?31zN#GH5rAahE%sbQ`c7qR`Emosz~J~+uj}4gV3WrD&8v?qe)L%z7SH|VHk*} z#7=ePUkRg`vg`&NZbI5(DrI1Yy%!Gvv8S516RX0u|3Xd(rFWw3T5k6-cPdh7vsDW0^(E`|+_hTxJCjP5SBuJ+8cO zh&#-@ccCI}DPEo}UX-E?7M6>SwLb0#8!T+bA6pAPs6-+LeJB`q5(lc|I-wx35mOyS zfw*6SJ#5O5D7S5$0=r$rBF;JSxf)~Xpqn`aTs?h4uRP< z>((J{wun`N)Y=e-e+)2rYD9+W%(ACwapd}}r1KzpfMj?6>^&1YznXCR{ zDn;}XWu;V*m#8U+oh9!_VTJS|k$5oG8d?59?tqG;?DOd4b@iG{+WrVk$f-s_mBYvu ze3i)@E|WkG0}3u?Pw-N!lerXCoxnXj3yVYl?9YEc!uRh5i#e<3huAfJf7)zz*OPE@ z(`tKioTbP>=jkZE-iLYXaK+QZZtJ+ohuy=1ZoMcxxuJQ2z6n1095N@$(&LNe7rMcR zz_|-y<=XVY&m9COsOHnJDm2^~#&;5^d3C+*)sZ|(h#Z+9;W zsQB=BIL`O)0lLRWzl89-+ijuYTuoPp-R5)@D(_Fby?~_)V{JhvmJ9vcfBeq^c239g zwsw<;&1pYvo_+l=U+)B2eEPnyocH!v*A9lZJFf1g&8y|R8h0B%r!9UyTxA^g^Zm5; z2lSsj&6M!Z&vC%dheEK-onv2@S9H{&Zu|7q-G6hQGl3s3ma#oP1O6_!<%5`&?{+r} z_wKwsI8^(1Hx;DudH5cDNF8dfvCs@mo43v$yWZO4z8}-wvlqW`{PJq;e|lWq>{i>| zu{Bh&nwlx!N|@#Q8;Tp-r7v+KkhV&g+2EP zLwa71=Oy~t^%?s4ix02&G2pYBVuF93QK6P`d3M{S>e@CeJ8?dK;PeVO+!?T7nW;9^?Ba}@qY%|%h-Rsr->XmT7ej#7c)B2{Y!Sk9cPr381i_llA&DPCvy*tT~FEN(S(HPSr&O4X&?+O^(WFRH(8gPuMVJNtbPA z3E8Zsf9{jbHDQ(S56Py2b1PXY*1Zwe8m5Ihi}J>b=+Z;EUd^&7Lf4H0cC!my6)1er zQ&l<;p*`_wj+P*xU;6m&$(*|wM=%@W;#08FsZu8Lxd(U7#x$HC-qsbVTd zI-Z*$cDg~Tb;MdHhyVno(t^y@#wm+OxH$?|B~OHXxHd^GR$0|BXzLfbRi%>%GnINN z`5`c;lX0NHsZ*~4)j)%5vyzOVYEEFXL~4_XJ8kqX)lbxH(yJ8+THSh}nMO^w+`8!fW@ zV-!O!Y9$J+3UfF%T(0iaSzFz`>_amth#^vNUx_vq*TII268TXqe|Sa@sujzXQteBF zSu|2S78OOMLR}1jDjB57JIZTANNTGbF=qu;Riic03KL7UdUbx87g$Zbl+}hxZmu~~ z%@Mi5Ai;RpCuSRMSf~$2^gsv$jA4ib>kz=G0=fJ7X11V0b~O%^3U$Lrshy2>yc0wv zNahrk!3CPj$PNgncIzv82_FlRq)4(o5d+@wX)nv=>`k& z2~^ODEER~XTM06v>6PXE*g@0O_G7J(xzz6ZNPP-wB^{`^6lxK8k2=iiebPZm0A&`y zx@VZ*=nB(F&}*m8NK(a3m_J+&M?4x_E8<&5@^MO{e|@vK#6@kGVqL=gVH?P6*3$Q8C(# z>aHRTe`n`yqRQ|fk#fpEm>m~0G^|mxO>}G9=p!=TD!_1pJK>jjKA1<)mAnT0U94#a$d&25B?aS%^pC5+t^ zwdGKuD2v}AsdDO1v5-ArE1p>>!AIaHqxs*o=VCjywbSZhil_1GkUU}cVcRaqztr8iduVpcK8q5`zU-sl4{7FV{;cdq#^M3xtniL1HD%cz?#i|RyO zJEC@b5@3@4YJsu`fSt* zM{q**n?>rRw2&Q@@nR=j4hGqZH8ae5(wtP}$R+C&g&BktgK89X_}Iwz*u^QW%UHuQ z&kt4+mNtcr2a&C!=MXv?&a0tLWGts;7%J^4Uyt4d4yne$TnLijRPi)IHdj3x$UG{F z5plwh1!@}$L>wC049ICt$dt_XMi9edFhqe4dG zX*KGkvGpq@2O3JTbD};->&<1bh{WE5*k!qov>C)BfZUwDxMk_A)V4xSnxr1VdQ^g{ z|8!Ulo)d;(Ukx1*3v-gZ%-AIyBvOQJmKG=l>dEn1X-))}P`(76%|3TsC2ml>^H7uN zHw$MJ7#r0KvEVXxYT88dHc^8R^}mHHaxEx7*sJ9V=FmL7zHYhlPdY&@u}QdLw%w#W z4&;~HuFiQChrR9F3c~B7>*7%LbVy~Hm>6vFnjNA-kIX43M=ZQB+P)M!!a04=90G*tH@tZ)l8LmC@b{?MWC#9 z0Oa(MbOqHOU9|~c-V+u`F{mOr6<3v(rlWF{meqe5-R~c_zum5WH*HVT-U(L=sBZO1 z$e6Q|9@h5{o5v6Td_CRIqKDnB=D(~D$GI?-7R2!OV-%hJ!|B?oU|%_G?#0fzVTG%^ znvZYixj+oRTtJ%zo2Q!z47i=Ay%45{)75Ezbtre)UT;n}^P&FPh48h91+jR4f7-sM zTRqRin>lTWsNkx% z+}usS0JpjCCt$uwKS`heh?=`c$QRX>7d@cKJ7l<-=^(?AGyLcO=gVw6gz>m4q5rS{ z{PSa6eUbEE$A9~;|9sqh>D04P-i%FzgI(9Jpqb7_d}EQ&>d~H2e|VbrPuuww$Lel= z|7z|vaLc-|$G#J}>iYRnHYZy)uP9LC3vT?%QNEwzX)A;C?Zf_PW4@5yE*RD4iLDo~ z%hnmWS$+MXA}+PA-mH(CTSv*xTE3O6dAofcMVl^rq^IrL5zAMrhkZpoYj&Ov`%}ei zUT!t--fa&1T?NB7d+eXqr^Dv<&#OP$K-mL{VfO+He!-4@ME+YJ|JB$m@1_Y9b#vXe z&-3W)(Vt9?Hja%~U6mitQN<|V>YAqttdCi2#<#ET{=47&(+dmnvI2AyTQGLfM z>#BaFAN>43WU}U$r+nUB9~+V5`QZ3^Zwc__usPX`UujW3Q#LQF==sLWa=iKoN*%lD z87ckTL!is*Hg}u1Z)ayQzI|SEzlel4$6OlWNcYtQBctvYJG)hEfNok*-BK|(DB$4I zNGys)0K8{jndsLDCKqzUFXUO05<4vUi~JG8DYLW&e zykI8qWtJ{5B10@(6MGY8#*&b=6vOxEx~nC3Y&egq8)cD3gXCUnNNtSiQI?`lrrwMK zR*-Wm85TlxsJ2dZgF(PSOaob<+$U6F6sZU{u1*#dV@Xv608M`0qg!=DQ~$I?BS|ye zJE&U@9L=vS_(q((b19JA6jN&GIjLnBT;VZ}9ql%8CA0!$HQQ|y(hvdGx`90y1n@~> z;#5yiKw-W$LI@=N4%NFP*el}mBrS@%KsLDr8*0_BH#GDEH8&=N9%P7uYu=FrjLa{A zQ-p&aQTu%oWD>_-DMlg^QWqmJ>~zHr1w_{-66%wLYX4ceEJ5uF>MF1q&VDwC0Mq zQ7E-5VMs2nse!$ffAjoI?z!jPMO4cO<$8gV`bMWpmOv_Vlz!{gs-8eENVXZZ#)c&A zb~Ip=f*vX~vUpuhU?-q|C=v;z ztd4qPa-$;MRcdxy6Vxb?oszV@xAa~y%NFfvV}s|bDcL+V5I{GKpF+g0mslTAEEhLp)K?JSxddcrTL z2s$B8NbfLRs#h0Vtk^MRK2nD+{%8|y_L2M=mC2gWwev0omjZxwuR{s)+Ch2v1iq{* z*-zMW7eoe!oVDf@6Mu9TG*D~t)yH5jFAP86I zPBVy>k^6j<5<=<1K?2W!@3ND+YkC{1yA`_zds-dZacJbTt2^0=vaua5iOmdDS}Pt2 zlO0y`)!1a6>M1^oJiVPpvl3kc1z8MnKfyU8Xk-PJ~RfPL451gx_>yZq3wpZS!sCjA^!li2>gAIw?J&-nI^Rg8<1;S`_F41=uHb9h1mr z3Fh)q#Z$sW2<%V6%l1p|3LOjFIXwdjt=n-S`X#b-wm}8Op--~dkFYaePB8hvr7vB>r|YL&ic^N^ z4Gukql_Gdm%V1TPihJSsTmc(M#F8HntDKbb1Ftp-CxL3W;wYV(W6@7#Gqr)ReP%VX zn$_6S4h^!zjh!U>D0>>Db4RHTJN8SV2F1aC)TnX!a7|gfX_V)-h($u_VA2#mL#UXN zs11p?R{gKU%&xW7B@I_sd;HWhUsV*UGTl!~&7a(cGYcUNBCRf;hls2Jr$=!Y@J7neRZNS^6MeFP_c@T> zU{abH;X-2{n#Od0^U_g2R&X9~Bn|4NJ%m&N<{5Tytf#EjJ+M|H@Pvk;P60+p~z-?Gf$L|9+j4%UX!Q{QsXLDPb;J#x(zh$W;l@R)%@;>@R~X6Rz&h` zVQX%ZUhX-SB-ag3gkFjr?btiDJuTpVn3_ydzESEG@$%k`K&;gQI1=rPoRHjTtjTDp z!xR$!K^1t;2)phZq?ha@ny^<&wV|G>9qP6~CZ{CCBKc*K#B40UlK*Vwxt4nj#BnV_ z$BA-uYMPFXv*;_XsAf++Ty{iq@NMBQN?6$6%>j%mgm`5=NS`Nup z`_!&C=@LgO#cDosLt5*v?%Y~ntX)%us-&v^kL1Pm%B@p{AlMvIh^@?MyL9b^C5QlF zp&eXtVDUveTYBs%?o%>n4XRc4P>rOY9|!%YPI*{Y9ZA$V%QR8QNu(h)uUG2>0dhK> zleNWWbNF&Izo-L`Q=KoR(W3LWFmD1mk4 zEADe$nRD%8xAuISyY=Y>Py46cEBxd4Lig$WFNCi8k@z)r0lnNoU%7sNp?~#v+SYgZ zm-Y7NpAMUO^-n+lbZgIfwDH-A|MI;Ea2E#B!{udPI9l)Kx1#hED)ZwT|FFN^tSguTE^pJ{bpKK%93M!mV*&FZ%cOmqIre0vt6yQ0QV@As?Qhs`XW;kTPxy}R?genL*_ zA8wp-+3~Hnn)r%eUT=>VuG#hR@)N(-GT8t0DJJ%}wq)&x-_K9yH`~qaa?Rsv;TQh- zKeiu^KmR0N+wEGA^m>tvjp26-^7z-s`qrE0AN|A6|NCj)uD-KZzu$?5^>@$jzFKhX zLI;aO`ND<0**Sgf{(SVc{JbY_H>VF*tH<^3gTrs}@^#+6-k%;EQ~a`_mUpuW+HTf< z_t&>vcER@o)cy*+tsgBr{#y)Cei-;ah^}lD0Gd7z46_7^@sCcY@zPY zhx_{b$Gx!GkHU$s?Uvj9`;Ra8yuiK^6YZPr2fp&xcEvXr4%=qhF3La8;s5VaP_<}E>Ah1Hm6tbE>qAZtlc@D)%Kd zURsTnLTtG2-T$}9U?9-}r8yd)WariXq%tcdAi7o*&Qn4z8b(YXW6MD2Ne1)rsqZ$p zoG>LB!_=GC4s|gb-Kk+z^VF6Lm{24rBylEKr#c5GM0$-OmZs^f`f%n1lmBl)*Dzs6 zu6oePQIw=Bk0_0hs;*~Fr_U6vmlP(ORhIW-s| z|D?AGZal~uQd>9*`C7+N1E3-(o3q zw<&fgVN8XP#wn&8q1bP?E|HR|X&QL>O#9AVA9`d+Q@DS3_GF5 zwa|Y7!wvDE^q@VP!=U zPz~&}*2hHW17(RIdgTBpu1Ut2&;zz8l=nmm%}Iw46*{FMm>?gK#J${*%~n$^H9IOB zsuv%AAhkuz>Zl8}YuH(;aS@;g^|j4(x2pDaS=w39!>$<2CUgkRiIPNb^u^pJWa-uc zW2G{(%#GuahxXbG;)qE7 zE~A#!I0zUD$S?250tKpIK2%3c{C36;)U9NTH<8FyXUoTKPJnzFX>u?ov%>?q9z}@~ zumWvAbj?}`c}c!aNF@>pp^d80$H>;|1c9^#^?p=vE@NrPCF)TkMha+y7$u^B2a}s0 zcwResMU2X07CgxRzBPqvOv`nNHTReaSJ0#&52*$w}}$tdUEo5UGtNhDe1B z`8`9zT$>2&NCY$3g=jS3? z&0~I4i24@ImDa5W(^6ebL*f*6=U!J|97yvAYDvSmr2pCQ6 zWGPJqPqpZ<^o^#fyKL$kK;!LfU<5=Nigql__gJKnL153^(lq)G&!R>}E}W4%am?*; zhITO0q8vw><0LE*fy>QJlSnBtZrh)!`*swEBb@!cxVT}{s3=idN-V!e77qoLy3DsO znVR+Nm<<&16r1!!q9Nj+!?`twKT$td3G9k@;(u=d?7o-ekw76f3#m+fdSWQwVs?&;7CxRAz(2kCP5{uIDrMj7# zOlb;-5rv+GGRzIzp|3Y}Ie@F1)acU)2>U`5zbSToS+Ef4?)Hg5fP{(~`p`J8)iYP% zg_Lc!OdCQ#KO}4^A|H5A_7BuDJ5Mh&6~eV zeNj0}X$;aTy&?v}*4lAhn6FsYih7m`OGDUv_YF%zO%T|6hMZND)D00b_hLgpJd0gF zaY|B$$4Dh@DSmedeCWNEl|C56REjKiF~x=)vTpPe^Jn0*>e;0IJW8U6np7RiIE?xB~mJe{~rX5I%b})6{@M zmtZ)M;FsCzf-!P0SDWZW)F_*YN;5`nQA+Y?vtvqB zT=Y_?U)};|Js4%&oLm!ygw$F#P{W~l_8n^^D$$=wdQ*a&2>BD)MIofFPO-6}J*|$= zSWw4EXCg#^ah#-CM+cl8`elG{$D=$1o=LH;r)z1MbOh=at!ySSmLkj>mQu1P8gZrc z4W6>5rR-u8{Gyk@IG~(tDh6tM3SvEOhN^Rby|Sj1hYSv#CJ)}$334s!4j|4rCVMJq zC6O9CN@YyNC3C=s7E3gu`c!_UvtFVfp^1oV*k-XIc4ZDZda??JR}?lB^(z=?L^<6j z6)v36fC7{z`pF(uRr7_pni&tSA z3wE3N2#I}fQSQf)o1UUp1Tp&yS}1V?wEwmw=o9JbgoYx#o>%s0t?7`%D7FUbEc1qI z*mm;gh$maRevM>sjfd=j8AwdHe2$ShNO2bF@@;*K&5CXxB1)y$2{Sd?esyPC*iQNX zQML~91!_bKQ-Ot)M1}*21PkmYB4qBJf)2Y%a_k#wk|kT;M@&%_;kq#fmiA}Cz7|#5 zkm6q21x%Y)6geX=y}5Q+#20CBGw=0iu+Wl|JBmrLBSM)M&T>Yxoz#y}Y^SzGVIO2s zolqGrBfo^2B6_M6Thv}eO!(^We>?v7m*K>OoBKvz5XSAB_3^l_7*~POKDxjK()O?C zyNa0GAMWPE1yXsK-)-ji$JOm&^SBXgPngo%&AZKZb-O+;IO11#e~opUSO4vNJZ*00 z3)t|2S>CQi(fRXFm#5qv=Z)xUfBuit`gFPT(_#P4A$QyPUiZCwrzH2=X*}b3$-|X);f9)Nwzu$kT7(@Ln2$_61?00t? zVH+LmdU?Uq=5#)N0m|!KqAl3m@(4>nt3n z+x7bjN`67u+7RC!Wu$ja~byVKLWJ33{o-g^N*F4N_&vOp3)hLA51=7;^U0$d;F z_1(Xn*N2lJx^@xanh!n<_LpbG@BVPVwlVjAeneFKr+@tRhj0E^SK=REeaSJ>-`Y#= zA8qKrc}?rAC;5%tWxIdU3)bKM{-fJ{^RRZHxmNrB@loWnSIg_^UB20#-_(DLD)wo% zvH#&oFwL*-=iTiGy5hIH`EdW?qc42)xF5alWnKFw-9J6-cboZ>`&E4Q@0U5Oh^P@edL(4C;x&O4^YTMbTxATQB_jq~eXL4=(KaaL?wjUPA`|-)<+#cbwNEguU z+qqIV>v#3KFFf08aa!M*VL84+u~(4w;rSZ=^qc*`S-TyaYu{gZyVcn6jD3Da887#F z|FB;@txt#j_8C|GJS3IKd#AA#ELThP6|46%4Y{!8&u_M^(^qYjeqr1F;0wj^=Nkv5 zzbyHeb@FnmYM_Q*zqX=^)rBR5hgD|?3 ze^bwS%Msc+0YYf%pB`5aYMw{iaUz`PL@Tr25|6C6Z7LxsyUOe*z2ze*J589N}%YN5i}C^0dB%Bd;6 z(SoZ>b!xH%9&&%CggdMtv!C~ZzWI>qdHLtkBMQoV9xV1NhF2f#M%ZK@jdNj)+| zm_v_cRz6M>Dgwo>v2`_LCK=lWs4c{XQ^1$5?KZ59YFmwCQ>*ql7XcaLe%1Lv`5#q3 zNR(@IL^3htE^P2~*pH{7@osmA+$};p*2YUg8WP=H+}RZN5{7vgm!`b))Q9o2{PICYKidsh|C25k8d zxEV?BY|j+z)HHd+Ff}IOvLlEWzFDBrQ#I{vVK)PSy z3r2+Qnj4JMI|dgDO|}LVVou_(r>|6j~d@b3Bspp%4NVlQ59(s!m6FfoW(Y} zECij9I7jhKYVnS&ary?ipN-wTvni4Z-7KvkPR&pg@@)yi%mY%}JA*!c=5IRM8Sv<2!Fa9ZV z?9AAt2BhcKE;d2o;yubaZ_JIjYQ_o4??feKCk>@eg1tXjD52h{B4)>rcmZL3W%V^d zkX2+O2xU4<{u9`xYLJ+ywx@hU8(?;h&km@*tJF}-VOq@%GUX^Y(o@1J55XK!%3i8p zZ6F>^Eg=$v76oo0)J`z(zceR_zq!SVnLq}@#CMH!PYqaNk763r#O;6|#%CXZ|o1!sV)&3E>uto{U5^^F2}{nk&+LYVgLJ zNlcAx?BMfa|1)4BQ=)mc${_`3@dlmcK_Uy-ahg(S&_FdmHFlp7x3LvrOPM@w$!QV1 zBxe=Ef$DEm6i?WBskgEES06;1PbXR`O5niAd!qp*yP|q`2k1vJKPCofU<;F-j7ZTd zLSxzv%HIY88HiWhA|OgSsO?RhkTfk~dF$}3F^7U^4~AYZPqE7&&Q^#f&8{{?y*`Vg zG)0t@oy!8QkuR)%C&Xw84oSa|k)+&>`pzKs4U_1AsU^jJAtfE`LYU;Nef&YVQG4)+ zMF1V9xj=`R9dc4tCtpnn9wx*S5eD%{+LUY6wXfJKH$@olsaK^3axp>tGSEOXNy`wm zmdb?5LJ@(dM!aM3sTtU*z0)_wJlyoc8x^#4y=rCXr4HkisI4f8wkT6v=7&7(1RXQ- z9;u;QjKhtANHW?MQqYR9uiFYn_t65HhLqA5gaTIdg@e2}Oy8InC5B({3^F?05)K&} zztRUXS+GH+BsET%TMJQzPB=n}Dn)}y5A$lpJ%eHfqW0-w>}_oijf>&G5XHzur-_|@ zQ&V88$Ba!R3f&BYgTFCCl1e=^{;267HSMN8U4z7Fu^+QCJHRL2n}Y0^Hu+RJQYpBJ z_3Dl^PZbNF;*}mJ>aI+qcut6C#GJzPnS;hcv=LjSa^fCGCFuw(YayLuq1#nlTz{Bu zk=d%6&oiW^=gPw%GcQy-Pa=_s#O-V`O&fyM09fVob{M7ZiEJWj4#tjyED|+nGqwTg zN79$bW8;nZWGM26#H58MHnk|$X4Vub!&MuJ4M4W5iBt*Akn`A~

rVsROZ{a8(LJ zo9R+Zbf%i8C^&K=OoaMUnu@)KycLA3$pZy)V13Qc>sdETTAWOGZ>Dz`A%pA$D$3oo z^ExT(%xw3m*FuD~o-p?7ozjLZytoo5-dAgG53?~!Du>#9vjjr9K`afS>9KPUFtm8= zYLatCB6@)fx~6E*-`GB~fDfGku_$B=RHII}h{AKWP=<+)kvPeR!6~umqoX8W5}h8# z9m>T5{kWp_iFL#Wu{`$8Ah8?YUm>nQ&yV9cF7iQJ- z?ryGpqEFAbyowTBuikFw*;y^TXzL`!TfbsRHb7*9YNE_ow~g z_Cx*7TS47Uhx6^9?H_OU>jk*@FxwCIPlCHucto~)ab9|xwYqh=?Hqb}*?P316v;Au;=-<4%plzM2 z_yyp&djBx{2g`ii&5uI!R`6vVnB`fWBzPH`WkmFYquTZd=cKpylYmx0t&*3rCiIOoY` zX}fyoKfm=~o_8XhuITpt`ToHcsy%MK-6sS6xVHDWwV8c+VvYS?GjkcN`h_o`+jED19MrnO_LCf0m6xLS6 z+`{<3yrPZOyj^88qL zvOUtXl@MImruxmiKHJ)R+O9w7rR#bUFjzOZ-@JPvTd%0+r}Jq+hS!lj?eEsMZoWEI zb9UI=Z)}Oancr5ks-1Orw{-~m@j|3+gM9pmb^RG!f85$L)(1V>|6?oQ&kpUk9Y62x zUYL>1fc-L-V715f$kcDKcIq0wrFtX}xiqvR?ur$m(ny3VOsVC1sfp+yT@}dv2xbI< zO&){|g7km|f~kxMI#x-HlseVUj5v`Y%3mh7N71BEerZT@-B9AuIgfxxS9Gdv3?9Tq zhS~*DG`bBzWLL|h*dW%cmOOxKENX^U!hR{Fh)oCiYK5+YW?ZR=wNlizUoas9rb5s_ zs92$yD4JDueVRV~g=fQDafKXt3jn9jh{i^8+oJcM!Gd#9O{7#VQ0jBX-_|><=|L@l z6n&0CH3W)OMvLg0oT*<*9zH=7MCV3H)}+vz6l;1R^}&sf$*Y_YuueUaDoQAp5NZ|< zqo}j(^#np^Js(e9IR)o>jFFK^A%0fER zP=zE;N(fAgIH9S8sxD8mpOZ?pSk0HnbpY6Va5gfMe-^_YF`9_=Gc{ce^(HAbHP>nl z1;8Td?=g%7D3dDHn26U}yoQkI54JySQBxtsqT4di^2zGcEf`1V&q#&$Q8i+KD2cZX z(OF@Ls!{@&E{T7hx(dqFVV|;tzex_v4CsVBq;{I7Ot>_m!W&(4KhSJ@Qk}mfM>1o% z(Y68QZE~5U!bYWGv1%Y`ofHoNDL)ju79oMeiq*)v*78{vW&u`pavQ*jZk={S2dX{~ zB+(g1b*IxNnTq)gFv7j)K2X_v(mT@bk?QHt^j5ir7gYojlTZF^D8iXbY$zNx!>TVq z4%&RRJitzr;sJ9&Ww9ia6_5gPqWWFX50HaajWpnXi1os@0=33>W|UI~8*is)o*E_c zr4ZU-&wMFzXOQ}I6crOzw&I$9PMPqWF3phiV-bnCtL&5?<&8# zGz7rVO0J7|qWf`-$keR!!tO#xdq1`efWCypH$v9Y4wAkF0p}ZwPQ=sC=}+ZkQC+}h z#9T1))72x^DB`+V>$?Ujxttn>wdO@_*%9Z_So&t({j7F&_jZIQSNz2_WDfj*c&8wg z2>D1+`nM6g!j;mNJC=gCL~Ir2bKJYZd(QD($G%UDvOYl9?^8_q-mKw>k!+{CXdgeMjiez zl39gN%z9fCVdntrR;1Vwn^;QfJJ{=rql53IN6OEQmOe{|KB6X$(riLv7%t^AN zpm=<8>?Lx_2v8LpvM66ip@{4+%`-`(Og@~O!e3D1&ZLn< zFK1BTf`#@`M0(BJ2QgQs2545j?u7k=7Q;ryI~6|0K<9Yh2xMOKVgg)5Dp;%)uVQ~*ZEf3Xr{ zQ<|o5Oo+U2?I>*7vSUj?dP-D`65U3NBf?2y!xBb_*09Ls5yIK4r4;+!&G91&eW@I< z1iC|!7*U?(D^o8_Q6h#k=3`7YXV~DN6CK9UEe;}Xif|LC>Jb#vK?22&KjZ-JN7bXr z276wb05rBwq}l>Rf=aSA_<_zdT9e@E^t^pajkun>OBR_!!u|5{W?e+4ce} z1jat521K*0#-Y(=ud7jH)sSM8U)zw+yKk9X?-w4cl+aK<`tf6vPH^#LXGUy#)#~~+ ziw5CQ7SNe+*%DbS(4{Bq)Q_0%F>jPHwNhWs+}swsmf5lTTyYrDPc`>Epz@Bkx^&FI z88%;-uvsYx{!GhZ6eJP0v- zhT}kq9nsxR;5|tVKqmvFSNlAT*f^Rn>S)^8hjw^W=Dyb5~WChG!oO) zMnE<6G^5clku0gfKzYy+&-R|afwgL@e!!VjF=xWIup$N?I+(J$_~^V;XHdol z74;#GNV}edaEIMBsT-mR;&l@{{z@~%zPof_ewwB6fwccY{Igk=(ozL$yh<-hV2sG6 z>Px+_Tn7*FbK4y3D0* zh$gzLG+7Wrb;Mq@5h!3AC!u2DO}wg0L!4wSntZD7)FOC6dpdcq7$EXPls!T|1F1*? z#59SWhl=uIx{2gz$6VSVtA-iUyrABKK|(}5K}~EmP9!9kt3EIp$1cETo6y5s9{=^dhJ?s1pTr0hSgaZbKs`$yT}d` zzm@!(24~=tXa@?ryD>H$cfPjl05lWv>hFI$9QgkW9NHiCFT2Fm>hxapnPLg8xVXQs zRF*&3A5Pbm>heO!+0q1mWq;os?i|v%!0m!X)2sF2w0Y~S#oG_7+r#WwKmYmq{Fl`? zAMSVag~;@~{oQ=HKtA{H=i}<}Tw$^rl)L@=-ARDsr|o+8DXh}rg@?!e?t>t$|G0kI zZ)Y*N{{HtB&}WO0>w|soAJ*sN@o~M|-K@TzPw$1ZRCBTa*7iiFUwp8! z5&ZD}_zPnm&mT5hn z)`AY-t=_J0*P>cpZ}$sZ>B~X{SKKb(QcbnsBv#Y>OJ^SPJ z{lVVv$6wW9Z{OPJ>`tq|-xx^J&mKlp#g&Vmc= zPXTam_qLjZef5EPn77X;@;|Iszxny6xAV^KxzOF#r}^#XAYN(Pl8(t1XZf4?-F|aH zKL5l1Y&X5vfBt@_yOm0L>~lB&#Tl*N%^xj_I+QO6Z(*`!y#(JqI3io~<=bWTIQuTs zFnslH-kmE?_S%MDX!_I2qWCn%)qFnekInCHR`roWeWcy_LP5Pe&F%cmzWvQdgvq*H zxDrK{g-yQ55kS1BTVL;6O!9pLE)WzI?B)_j=Ipy_I2Wt^~DXHLr4ak;%TOog5(n5Ik* zYkr;DQkkOIJSfzB@^nDMo$14soJ)d2^`hDvk)w_x@S^Zbl#?An>c-HqGboK)k$RHI zhD&k5(sq(c%8N?os1T+-Nh=E5EwSx4+9DEqjr`y;Oo&U7>qL@_!!kX0#hyR6Q1^Z9;yJuIPUekLzb9z|i_EMR=N*AfO|lwCtcs6RAfEMMA!U zv<7IF1AV`hv@GqcACWcLM%)I(ss?!+TRSqmB)lwUfX9m zVGmEtwjK(s-7K{fD#!Fw6mk)Amm+Q??t>`e>xSG#y?EmLfl`4Ocu_mg)%L(#NF)I{ zM6(&O9{HV1=qIX=335dm&9*z5a*NwMDb&CQ97mCaA`YSRj!D>>llSS+2ew9tM>v|j zEe*1%o*YX1W0!-R8Pq8aDoIl@QC2CA2#B`%?pZ83n`e(ly+h;(y&#gshOF7WmeR4; z&`Z1((L%+u-!%F|iR~wTat_jwq4Qb&2@w8!+JpJ0MuAfR0W-O(1qKD%}{lp7ImUi6k*07GwIq`dUYRyoha>*RA0VN zl)aL&&Pc&#UzDgqoQ<76NFHmi(UNXfw|5emKZYHXi!Vdmk$s9u&sbzcji85H79m+b zeSqmE+GC}FEjhuSmwHK8p$P=<{SVmMhi31<7jvWh)YLgpzk;YSCY|n(D zyhxwNj?N2n8cU;|y~O1Om5TinqaH?HR|<2$pOzm zl6?@AqsglItV?$WFsw-wUStI9oE)hXF!UisA|xArxJ5E@b6sgG(cqztQ+qmlYAMA4~!MuDPl zB)Hvcw7(`Yu|a~`jYLn4hHOhC&9;08#GnZYTsYGioXQ2lO_B5yPDr;elmrygst{d^ znh2ZDWVdGOTp`3@Qa>bt9~D_O-Pi7}n;+jJy^O@FklaybE;iAluV5(}(Yp_37;_`S zSx*=N;3!6py?``?C{yCB>G^c8N>WIyaHw`|B7O{Vnkk2GK^6gWqmWG#)VZ`B5=vg0 z2iQ?BwQ@MA;htH3l2K4En)A|A_jj!MgtjkA%+1_^oTJ1~z!e&>OMVb`m^QBv*hoEI zfG764y7BhnaFa2pNdlZ-W&b)Xi7A2Hl2F&}rk*oQEeDY#<6)xB(4!+HX5EfLPNjs^ z=mhpoq|%7LGZWP_z)^T>3b>1!BFL!rkws|-;eDYQFgxaz#@KB-rI4ct?`bjidJ9^N z=CD%MDLOO`yvg2i%hZ#k4#h}=E77f-;pCM>OTvCIK?a5%jZltLYMju;{Z}4~QbtKk zN$6deeMGk|OTQR1!|HGt8=QYv`#@XT4F81~(1DpKiPZ)+mzs|yV~J^7Gn-9C&39@p z9hI7Tl~*yY?!_uWk)?D@Mu_5^U7o5oY9zqwqa>zK!%^l8<-n$K3e?{in= ze$V3Rjw7^PjM9>(*AQ0SRvy#Z6$uegy+a*K;0n5qRzut1F2Nvrh<3|K%c%j@=v+_{ z7w!2Z$A(%?g14NIm)OgJqfCz3EPF|0wL=BED4sMdVOlU~+lK`n)eO>91nY5WQ;))5 zlA%Ur7s(1tbXq5|f@I__@J2miVF+rI0kFK;Q>lVcFG?JmQ>JrFOt!-j2r#k?dY@nY z`EQ5l{@;Tp`|5E)aSC@EuGE)e3u5&}$v2ZAY(gP^xq3f4K+-Pwa`~&39kphzR$mM^ z<&&=-=i9x|q8~o`{spsJp}j%~jyLJ@%lgCl@#*wo0rUN^`OE3UwXndKuh>hx@NV9T z{?w6~>MX8MM~5k2t=?}=4>mZ@zpKd6Hy1$dd~r>ehSzZDL# z{^+0I@2?$Ze0&Wh{k09Pz0u=lyA{Ur)l@iJ?&*BD+1F99pC9)2=xc#HA1jIP!Z2IG zhq{s`VE?qO5X9$;iMMx|?iXHGAJN&ljRn*_y@HemGln+!?;k{BYB$)<$73aG7Wn!4 zZoYm-aK@YD^KsnouI>N3e!l1Bscu$(+-*OsQhGgPPY1i#=4risdB*xK8z(`2zgtLJ zAJ%U+mGsnJ;ohG2cv|)2^OeNde8DXrK6y5~^1c1;X7!zeaF=QN^=m}019hL-VC!RC zAeZCS>bQ4qXu*anb@D@niq`!wy)5?2rC%n`t|HX;YxL~9wX@OcQMZDu9>gTPKQBMN zs>?Fo46lbt?{>NAvNJ2w?1z=$zU#xmw#?>uoGX;~v|pvykG1?HOwD(f?P9O`+xgAm ztX(H@ORbZ-r2g@Df7HUbJKs9ttuxr}&Ndoavknu#-Jk12KI|W7Um$fA92e-de{R1` zSA!XcPrvjGp1sl~-rL8vz~}9|`QyF)-hSp&cp(aZSa0o)>$?R2e7UL?!V8=Ddi7xz zmvFo)CD#jWE1m8#A{D@UTwM_1S1UXI{lmk0Q}Ox2-|mHAUwD}Hw=oa;QU!F|Eg$Pn zHUqRjXb6uF^Vase24Jz8Yw>jprG9j-V(!jjcH)7ErDNI~bMVZ;MM*Ssf^wUUS3O-`Q#(#_Gtg;B%DSM6nkZXUw`x8S zuCGudpE&w@mO>X~v@{;r;$jxDTD0#ns5;fQzS8OUlweim0(zDX1=Qw zCr#IQ^?j#4HN3brr0tQQDNk`Eq6-oGHPn=zcaSg_<-vmRIlwm5_adi&hpmKJ4MW8B z61|j>2vlWF)MtS5#RUkd|VmijCsAk4aY90ZIa@#i~IWg5W{T(a=wwY}6y6 z$|YBs0KeA@fJ*Xt;tVcrR-$IS#MVW90C^M!73pPSX<#9p%RKg!ukOVofOOh2S`F_( zO>qyU0gNt{xr`iv(Y9zDkcOyaFYUl5eay^eGTD$c>T)&PYQbivPqHYGv0`XO7=^4u zOQdyuF%p?Ha;v--7zst@4Ix0yEG5)#a%31m07g{{q0nZhh>~~A0@zV`(@eS6DXK!- zDDn=~*is!T+G)|D)O;#xil@?o9if35m(=yqsH)a9l`?unC@kS2#OX+Pa|G)=2%$tY z5J_YUNM$K+pa4Ba5?TY!5*w(}9vSti%#inxpmR!n)W0>qmoiSaJCxiZhIhf{fz4u2 z`#OQo?A#+=da+W7+YCN&u>|lcr00Z-#P;|Y8$u%=bm@>|-MrYErX$Q8y}qekyH8`Q za3XU#iYBwkRM%vtGV)|Ibm}0xOc)09Q!t3IiSHY7&-(U-q^2BXMs-pt+!8Crs17ec zpqg2jh?_tF859jDQd*@(0ilDyj0ro^HLMptwu8!nkW)9Op~rfS9gjM^2=pNe1oDo| zOSjz#+h2kVRA39PTMd2tX5jkJAZFe5lzbV%=%ounw-!|{rb$!s5;#n^=+B5%Uu|FQ zsVy+MrWMcHejsNx3uKOL@S>)s-lyPX+Gt^-1eK9vG!4^$94vCiksLed*o-8YZRZNN zDVW=7aRtU_Q4$YWj~j;;g)`oWb7G2!5La`^&@di$&bO#TgdmN=%2DB|7ib8pXvuFv zZh!*EMhe;AMMZ1K3OIE$T`7zDmFVXzkVm2Tq>!%RT5(UqmLqB%n7c^O%-FjYWji1e z%kGJh-A)$OZRr*kuLDn?8K5yW0wpz&76g(i(?J}N0kBs#529hDsKVBiIbRUF2oDUskuv!hPl}tjXYwYYtpjNa51m*CSnc}X^jYnM9g#AyTU{-}&rk?YLDVW9L#MJGAuzBhS zxJ6QXcoWChz*G(G*vRP=jnI|VYcnA4+@y*%@uv$3jS>XCD5Wq+u|I#N@1z$5ALWsU zqRAovtQp8e%U%jjOr6nap1?D-hLz7m3S_}%%~0S`I%f4RLT-N)w44H;tZsY41xE==Q}C$KinLQkvrtF| zuUwq+57g&iX3hK#aC~})Ab<{#I!~HrgT)^-n*af_g%X}NfZd{Qun{s(a;?;F3f{9F zjzq3&3qKIz(@cRgZ@}hOxdlTLQMqCJ=ICT+@b^$8ga$8|OX-?4zMtAsUWV99K8#Hy zWQ2MFXvnUODP)vmRwRrSA_2vfCkc_m6ncVrj;OT}$Wf(g z)N)s!FlJ)x#XHOcIh{75=%bQ1s2pG)1VEgs*Q0SH+;Z9uD#%jrO&WNX770UuAJLCG z&OZ4?Sa8;9pl-fwhyO^DAZ8w2@CSCc8Yvs***cNxKj-G~g3x|j-gai<@;1}bvitf@ zQH+8u=)EW5U?|P=8=Dfh)~b9As+?+80;}pOW!Y54eHj(uBxvEA7lnRYAC&k1RnfG^#eE{8ajNdyQfbOm z{(_=NZX~vRpBD8^0)AuaL~{-UpT!?W7G0W2w5`=YOCn41db*~m&Trk@7J@{o0IKBd zO97R5wbTi0cSrUVb6p7!pqR8E|8;aq3E~k^e;l;2{^ZriW-=k4A(CSU*cUxYc^?dBIQ z+=}*ln75m!{o!==!qR#7xY-H5xOSq|?dDM!K<6ZV29BP0w-4*xy@*~bYp2D8;?li$ zM%lxs*KpEWdx{Hu^##UvzkfF$b}w@GdN|$gSP^l{<@FaQt935Y-747ODEiXM?<(x5 z_(T0C>)mUZr|$E(zS9+)pjE`EVgz+`^2q^zc29eq&A*-Jl{h|6A8uCvyw~6!D)QAH z>glk5vz;I7fZ99g4SxL9Z#D}rYvmI@{QQ$Zr~2HUM??MNuU23D`tI_rWtQq|R)FfS zzjUPFVcs9^MgJ)EL5<@FHj5$-ncY@yvfgP!%P)5Bq&n`bzl4^E-$kjhi#g8ggnxV+)cp7YKI?PH*> zjgV$j@=(icY6-k~bC}<4oSFDS?OU&9U)Ob3&h@+5Zp9+~xEI9utg&de&l%a*CYI9zIQa}>c6db7r^ZxjP)|o-~Ig4>97~~^Xl%r`rSc~ zSTE(%>uX~Eiw&})ls{$;wz0q6+hViP|Mb3B{*UeY_Rr3;ec2wi40cCt`2}3P*}dWn zzIt9$HoT|3ZG@*~!TZ%;X7{6!vFEry?BAar1WP|}SHGFJ`}4cQ#w>yDoySd`9f73x z!udX5<@z#0k3au(oHedCh(0XhiLMXwFt6`$Vfo4#f}`slj4UQ*0lwFdZ)6zHzH$H6 zM&|g87XN-(lNTCm2f!cBx0ikKR@Ulyq<;L>yNyVSS37%~S^uRi{o}8+Cn}BekH7lz zd6#|v?f1SQ&oY1&O}w-1v_JfLnX+F3?G-6NK$1~ub4N^m_Pr|&kl#E>0uCa%A(TF1 zGhCJ&6)Ft6YB_Z64;HIWNaB^dLS1TNjcO?zbzTRBm~f@8>k4CE$DnE?y7`mc9aHGy z+=cYEdH)@Oc`O>1A_j}s*i<3KYM%v|B{|n%&K(p_%=;EYKQ_6Bger3y^$wCvd2pFI z@PLP8!6n5JS&9t=E3vm9h^5d07Q{Xdj+?wMPdm~ikG*RYOQ!*`D$TwnH+Xg*idZe$IdT8DmUSQukWv5z$l8qL4YxeUDWtF&0h@Ke~^*@K3azm2e}< zjxXirvA*m@iv(k$2iLJ$!mbn_(DRxb!xsI=RlVCyg+0nSL5$u#A z(1qC9z*^=$s^cGyFZnEqa7JY+9rDrzal@g?tTWW((i#C>lo#Jn~{4~E`6Hl{w6 zY(@LQC}$O+zydRY0#oh>W=hU^nkI6MhCIrgz>e~|7RuiaV?$3_bJ$ZyO#LG23!rnZ zO)!qcp+AkGA@J**Ey5Bi9K$E8SrC?7>imK{5>e`MCH+^CQA=hV^#4$%;J3CLo_9PJ~ z69KWj=cb(D2dzbZO*>oNv}!RVcwUKZnbPy6gA%z2=xd3DYr$rmn&P~__Q?F=z4ku6 zh0W?j1s0nl^-U3USBYAdy)f5@T3a*~Un1tf|J(GhdmRb!3ZFdK}SzF-bO>(4FE+BU+1I zjOY^`cAYNUwr{3SN4LL`(6X%`rtP+s zh3Mek&Vo`$cE>z?9>ngy94VIPGEt|>+G<| zqkKJ$WD%4O;pL+(;Q;g4>sqK#WlKIZvKQWSAV&jS;b^DrK<8A>W8f|Y5gSRZ6vf~} ziI@X&r3_*Zv?$hgJb*Y#Fb4rn5usGWL=06?biIT`@(vB`wH+;zW4~nz9Zj$D4K%x| zblT!>{FdGUFS#ci!8oY%8Y!jByG87<1^rF5TH`;9SHTUDhIyvUo6uZcOGfREh{XpD zH<=#Sjy>F@L7)@jy&s}Zd|2@c*f2UVQfku6;G+v)tn>SB2a!-XcP&@ z=46shdv!g?EDB*x!639enU3NxOhV*oT@(Lg5_%Nzi=x=a!m*uuLG)bUG)j4c4mor6 z8tEhV+LExjoa(VVP|7s~SD>JF>8Z51%25gl_Pn$eEF$g(N=-@LXT(FUOlq_vmSKVT z3ng$R+%OWISCM`^5N2d?438s%Gg1)C8c3hq3`XPNZeWo zHta-~bU9{dlbzEd3V{inq?Us~-f;xq$&Qu@mCPoKosdS-97DO9pDvzoW9OhnLZ9hq z6PrNe33)hTdmR&~B8|bgSV+JKQsARg{lN!=bn1{ULZVp&j&blfxY$xbBvR4QmHTL- zq}bR&YSY_sFoPh~=Zr+Cl3Cy>5wm2(<*x2rQs1%Z?NN{}%Ax6~o5D;fX(L#Krml7} z8J#(-B!3kwBx!guYLzA<&`~?EAOTEL#+SxKMHlL4v)IPsR4iqaJU8f)l~oW#~k85E}< z$BuzfL>2JBlTsD96@n6_*orPxGty2LduC~ZfEAi-TakfRbt9lWRutZjBoI`BQvgZN z+@83;;>|6kp76;wTjRG}Dk5gH=@nIcplhjXfb4ac8XMqS$F9^s1DBJ#YxH$%ej^Dd z+b*C^E)Wc$ZDz?r(9lYA1CiE2Bpp!w)1i|Za;oYtPDqh7P9+jaaZ*v82!#7d+l{;& z=fW6~tj3%Tc_+By)y9b361K6LK5pHnsP;fW&R(jtv>K*h$2Z{hMHfqWdE03%iNkdj z?Gkk}43|108fLC~wn+&Ug*q%QrZ&wyiC7*<=Y@)=Ho`0gz{c^d(-=nTz7_^SSb8!X z!gvoXP~BUYPc5f*zD*ruK6D;7$z|USQn`hbXArY6b_aVKL;D^Ufm(zQa~rzQ^2%S1 zDnRK-?lMzYERyYl3`yc6Z#9amo31hrw+4&u2@(=YDULlhQ;8L0XoBF*Wi=D>^r{Mo zdn72*Lo#unS*oF1kS$yjoMSd!ZK_lIZBSZxdiCdDg5dpbz5B4xMn2=pUJJ@!u#?Bt z7xx1G9aqw^e|)o@k6*r~lYNcM6vf5y!bvN(*3({4H$ga8>(zGu{`yupxZ{3X`A3C* zTdxl5r_J5=gM{tZ6~Flclyw~EaennQm6+FQQ0+I5o4?F=(yV{GIUG+Ftoe3+|NJKQ zG`FX-e|7&}0O5thb^kb5Qc%YS>Yk$d-9LV)s9Yx!wx8b4$K%Fc`GUcG+}z#Szizkd z<8eN&9OC&5g7$azi`pHY_V#Wcq4@sI9zzty3oOr>WS<_^|Gqv5U3;7l3yAoFsnkQ& zX{iuKU9D2LR?uR-#@&4RLg&}b)+*7cv6bsZS}!S8KjczzN#cD>Q0Eu^cC|Gxe~nCt7`%sWSjmKLL+8)4-QPNp`1OOV-q}K( zKRELB=l^(juq|;~eX~EjnK#?(Z;!{l!{onM{_9Ks-gjralyaj{rv|ar{AoPx{4DKuN|^2lHljA0>94R zf5bPu)hc`DAQr$@?`do1LH~Mznb%!=ecE3O^s2YLY~W>8zO0L%)}mU)SMx>+|8-w)ORI z_s8SUKOH|>ZhF3NclQsQd3SBA^7?D>PhYC#@kRoOgya2@T zf)_z38Z!4@ZX4?Q74a(&KTWhJLrI2Ui=lzN*Jx!NKlS{(N-h_eUfA9pi}SIRLE%z+ zEpfM?&n)7uM1_;l`H2zGZBH z@^uL*EzMe;X0)gR)C&}aWsq4H8mhNTv^~&>cX%$X&7_eG*2N= zGiQ?eI+B-)v51IB`aYCE;>UI+LBkormQbiv5~4&7lkMBBRJ|W#LBS}<;@Ddx$~1(2 zP#8{aBlac{(z&`t87RfmtNFCy`mU3x)(uMj^x0NWf#6*v*=C^_<}fw3px!^Jr<~^G zGb%!&H)=@ne498s3>Yr4pb?mR4C?=<*4L=~Ks#C?*r=BVFyWTS3~*~|{c+VKf#mPS zE>#GXI*M4e;gea4VMwC!wt%TSLA$WG!iI?g9#BGcx(Aej; zAHY|NEo^C@wx5WM0&7IMxKV=-bRX>G7hgckw1mPC8)vQ_*w)6+8qV+)l1EkCy`w3eSc;tSj3xb(o9g6Fr zZy@V_mm&cKqQD3ekznE=b1@^@9%Z(kjMgxMd)mf4KxopI>iT^m2SG=ux%tQ(m~BkJq}!?+Xy9XV0%DZo)OpF0 zxmlcV*fcp7>Qx=2Y(<7!6!$&!KdQe=c||ZcW|^@sV-f_6oDnHLQ^+woeHR_Eqf1aw zm}GY$_azGpLWw-wNBlC*QVOIjdj+VbMXp9nasP7Ca&j{+(+ zkLDj(G*0KNxy*^%T!y}Dk*IZ+1!pEJF(`3@y)-TtMLBqT;DCjRC~h^hMU%i_D6^!D z26jr9sljLX5j04!gc51$nn;w@Hpk06VqypGYjaoY`?+=-fjy#lyBcEodMyE?NxMT@ zvG68pej%0NAYnCv*{H=ALA^FuNG;wc1zyBmk?#hG=iU%OA7e|A>(4c)y_0cQ9Y&E#G9mqOH;4iSS@_2s8_&9PwirRx+C4IfQG^ib zy);Fj?b;l!#&Y@2vF(ABj26+@S%yLE=ti|&CsJI*zC>}FBfE?oK9nGdML__cm=Qgd zQBsBUxFQo2lnH*(HG~0fvhDh~szvbMw9UDV?YQmPD?UxW-lszyT}Xa7s$4;y?v#6x zfKW7H9v9)$Qk3J0XOC1c$sH0FxAepSuYMH^bLh$Bq7vzjoe41q0+u;<5wtv+?F$n% zj+u3EvS?Zla*5E5>D4m=UW`jig5zO9R$^vmfiu&Gqpf#-|9ox z;8D9uus&gGC=>w*D3VBuYbo>+g;|pV>1sz%0ZhD_d^5s>C-rln^a|l{N43#WGCoNf zm^|np^fCBr*>*9K*sA1;`lWq0D1`@Uak4?Mp+xvN!wWOYuq=+Wo;ny$eX~_y73Ax8GLA z+TV)b^q~scA1j1#eQ$sIaI^Y)K^5C(P7Zpjw>*9P)NkkQRyalT9A@f#_iU|?P_5HwpVo!?fpS7aC}1XtKZDW zCsDKN=XZzw)73MH=wrq7JyfjWO1GT1AFh}8wx`kEG}7Om?RWE4g&oEmU!k7u)mIPm z?P)FG?{0ru-R{pjf3~Of;m?jz7TDE(@3XuUr}^+GR^HQkyO5*q z&-Sgy*W|JSi|X2M?IU|Io40#`KOfJ#&F$u?f~*%%XC0ySZuO^~Q_8O2iUalQ`LNyW z1ev`%Z0`J7zu!O3H~w|4#?{l-X6?Al$QQ#+w)fm_oke-xKW*nPS8wKrjs5Nq``w-W z^^0=(LZw|@U&vwWtyW*8mrHzkC8(>7^nSN+XNUM|QTr&p*D8E|8lhs3w*D$;c749v zXg&|~GuP`4?dSUNm3!F!qt2x*Z>M2?jYPEpaw6V%Gd;hibJ)gEx6adHv-6Q%3CF5O z6cBd3I&SWFz8Q8W-#~&e+YdFx^TU2C4(@-y5F6WCTR+;f-P$egU-+J1R?K1^UZ{zy z0k9w-UOEc$&FZ)7TYK!|Wqx0-Wly+0i~QI&sZES_?#7nFap4kv*xWz(x~faqtc-nr z*~h+))ef;Sn{ar>t=F$QE3vJ_$LFtZH}9O*+jq9(_4mSe&&T?z&s5`Y-)_%N;9LRM z7wqxdwGHw{?FSdC*sw8efip*pVd8Txp`I-T>Y}ByVlcb2viaW00QA#xp z(*+8*h-pEs$x~B7@wKNa8>d8aOH^C)QD)wo47x9|=)8;&qJ_T6V0YgAosn2TbhOkg zDIiO3_Poz;$#EhSjp#%R;<M#>|hhJGqfS^H0?2PFrP|LW3TyCQ<^wWQltF zek1;~1okN*bz@E{Bt!3VKiS0sU_CUHebazbGyjt8rtW3cQL(o{5r1-d;~P&Jy?;HX+;UT{Bbe_H+r6H$wf;A`zo> z{&ba0H@%wBT5Mh&EAvNCdDOhIrpcw2Z^h9-rHdHM-$xx&9WWiic_jlfP_E1YPvAWz zY0*JK+NM0KWf7c?pyC`cALsZiSEvuLmX;4~U z8Y@$wPc&0fnQmWXoJs?2tp!GNS&1}EfFGjYwS74>Lc&gDnTy415ES%k+9TggTy&8C zHSabJXeme4wtzWM3>pm~QZ20&br`k_!D5cF0Y@zniG{zYZN&Nurv`bZ)>Rg?;A4#>y&M%?iiRFJ##sU#{od{Y9JV~^z7PY8#0fMb4 zKrjVPN(p3YnhME9fB?9ssx-3wxeO=xWW}W&pdW7#D;n~K_Bh{al`cdDZ??LO&GA)p z2+EOReO(Z$rub%F++;@!@&`|Na`O%)PG zTI}gi1u!TQY$8mDxSNr9POO+?!gxdUQ4u!KV1C&|3Kmjwm~iE#L3Cc9p>lWXV>+pG zP93ls2)@o>mmv8E(%dM*vO?2t5T7g}qQ&4r;(Jmxwg{9G5`+3*W2;Hx#u`ZEr^Ofp%?Q#wHcfL-)Q#vRHYVi*LiP^ZCA_P=3-WZ1zbw&p#fU@C#r;GY`l(Unn^0fp^gptXpL_M=94sI zJ`&E~LRSg4LZA2~!eq`9AvJ-S5{)9&PJ~saei*a;QoIC*FAq3Y%06QEQu4Ye0Q>A@ z6^RD~Byxr@udjw_vL~H-v!;ebqo(5Br>28X6Gcwykds1WvZa-@hzy7K7|?IDvZ-Hh z>T|=L>#nx5@KO!i=0uH;u&W1gSaTzT*szgJ(_Vx!(&kqcY*(Mulun6fGp4a4Vc%)6 zPq3ja#?mb8n&c5z2Fe$^xFn>*#gPRWx8S5&lm(DHjTr*cVDJnVcy2l5rkl+-k6Ir} zb_VlEko!i(W(aFu?Zr5b?3fqBk|X_E#CyBzV6I8Hg>)m6$`*9 z*D#IdwgP34t;U3=E^0ZVY?1iGRYPu10Ie<32^Jj|ifm+IkBJRd!8=eJ$VC~k0F+cC z9;ldP-mF^hzz^%?+E8na`l4Yg^ig&ITg)Vbx0;J$3Kw7 ztLI)wsayI&*lC-lp)U<$xaOmjYovCuAdR8V^kr!^pxS~>2=aj_QW`U@tZas6M6u

@v{RLZ%>h0M{d!27~{V`m7qq+OF;KS?6a(p*maJ@hN>g{>wf7BQ4T0j2k>XV;8&ZmdH z-}z~MxX>1Bh)(vXqLp1<;mXIyChuV_DCs8ylCIQ&ZyWk$v4rVn_`A!p{rKy@Jyg!s z3qtuBKK$uTUys|r?{^nY+uvVS?nlV>3s$$z@ z-!zl|EPm%oj2*+3*_8#Q>?rV~;H>lN&l|0&)1lHlFBkHQ)`edm{2%}H&p%wJj~`y+ z{{m?Ky+dRd*z{Wg!TTNJ)J#o*L8pG zd9PNsTpYGrLA=iM-Fxa#gEOkZauib{e-H77c9%1w7$Y>kMUkC@DLeB}6-UT-) zYOp_N^ErtuQ2O#DZ7DVwM4=6y(;z}bB`^VDA~~n7P0a+AIulAIifJH*7DK9lj2d)9 z&5wiEo}{*mUY*5>BZ=+S4t*#=%7uXpskuoiL6A2zHU;-O7m6hiU^uC%pXBJ(kc8dR zA-NC*dBzr|6)2?+(za?V8A=vW7DuzxH;#V|kcAebT6|Q9N@^m`z{X`<$#CPEF3Ler zTeRr#8p`BuX}Y3JRf-{wg_e(vZ6t_hDrG1QlcxKVB8y_(txSTKpLHp)DCPcI&5`Z3HK%LTLscGG>fve~+V8OfS{77>c zCwo7vNc5iNfi%8nrg0McZla1nku22&?p=`teM~*brgfewh0*>?3tC!LG^F-X^{OA9wKzuX)V%3>few+ ze9ld^xn|YX^FlozgD`ifDcXxTnzCGK9wEYmn0qXo+v%GMlKp2pumdyuijciIqUxz; z=Qg%Z+1vMRa>XsX(x{{bEgD@CGcR@$^hA!`E3-`1DslG^N+tGooHrz%zABt>l(Hm( zUpH#Voj1r#iHI4yQaqf3Q;@+|Vq@QjJqvV-^u6!IY1^kh-*dMd#TnFVqK!aROg3s>iYcWw~cFDPSWzQ`F3Y zIvF-55vje*9h-<5&diniEXhfbWG-NNhH>nW*jb3I0m^zou29Q*RhQTAXMnlrunv+9N>B}KKv zwwaDPx@I3f-Jy#+pGp5ZNUf$P@M#vzoWOob;XB7yrRNSf6}Yh#!mJ6xv(?PlQR=|v zgNwhS_CpYyoJE@2$;=PYO$m)VH6wCeZp7yWO-H^vr4UUq`XGo6His8ErQqTPv!qE1 zH|>l>dvR)1@FE<<3GyWxA$Z^a>L*7v9H|jdIHtqJ3^Ra0!~w*TU_uIGH0H+<$)E=5 z_!2NrpeNdHYO1wsf$wSCrsXpd95{+Wib8i!F%gVMq@(6fk+~dNTKQxls-4dP7m8$F zJ7QKJq{M(-^n+UEQA#U@AqVJA9-I}EATB|SwusgL9JE$M$EoM8-bvXfq+ApNo|c3J zSet}rU3fC_V9G_ZHj8L61Lag#4xNaH# zs&8&1j~%>4DM^aEk~vlhsofQ*5lDG3m%wZ>qWQ<6YiMt)cQYpP_(b|ffZSphCxf(J z()yv0)}5pVp6zQ<2`7|!lwKHf8`#O%Lt)Ddy4ZKfV-J$6K(1_&Wuv+2M`2kpU=cYA zsRM}Q1l5z2scPiOeRgfP6K%58-x~(Nx_8w!O6*3`A^cRx%-2>gGD>WVw-^fZv%-7@ zvJdJCKu66SjNB7;P%x^3Vx4gi*D`pul13<`@xvVQhU}w7-OAj< z8nyrrp}*Nd*JK7Iz=h2?qGRYWrJh>`U~PcB0Z&SVK=Z8qB z<8NYd9&5r$s|3QM=uAF9ZJ~C8rY*xqF~>;C&%FW~NGwRC)#5y|sli;5CSG)m!nPI)3om`wvg2t5v@0@>IY0&*#IR=MPsaGazvq>)*aQ zZq}>)`J{(_{z58t_40DtuO4Szb|D1~_Kf-E!yoL|5B7Gq;vjtimcC$szn|}&w=_@P zm7PGZM(D2&hUyP?`1}onp^{)q)88^0!(`tq9}kB-E{ie$%6Ql+w|7pT8<2Pvhwy(n zAQfqQdcn8aAct=C^}!b3@nsFxA?~is9}^~4P|P1r>ys_o=Wi^etqbt=xxGH^7R|X?*I5-|2IL~ zuNE%r;S=&XIqY|yS)=jFT}eSv+#)-`Qy9!a1w1UDs}ZiMLA5v@sCCGNqQOnb!iiBhG+2a)FKB}(#T?3`nd>$* zqFtS{Qna+pH9;O)b`!sQ8ixkkT&LFJV@NVd-lA)|0c*W6D7T0yqa;K!VX>vMNeJVT zC98BBr9}LAHuC3aEgny)xR8Bv??1L`~*#a3=Y)p{RriAB@>zRHSpY$rxjLiK4FQ zHD%Z6H(8n}6QwyiApjdZVdo-ir~&3w9a!E;-i`@r8OfR>4$bEgT60y^?TA<~l{_|S zQlH;3CrHyt&@Uzs1+7m-Im%wtRt=#+ecm(`VsbKjno(!NhU;Xr4T9a(VtW>LXVU8WMGQNCBW zV%%|(Egp$3N@!%^p~(~#(XvL|(pCo%Dvy_`1bTI|5yYVcQGq0h5mcd$mx3TUk(o76_Ij}Q zf}|02MAY1k7K78POp%41}U!q z8cltniV@qN2qM>%GX40rPv|7J*v%YoQkRrEcGCmmgPz2}MZ{I34t2yHiCG#c_o8#v z!M=&ewrJ6V5XaPZo6S4HSWKKls9kENy@guTazK)HAOcnd!EBMkOr z1v=zFSe?woRb*al2ho)x`6!hygB3SlzR+(a|+O&e&r9@KA+z|d?~mb7VtoWD$f z2EF-+_&DKf1EotwUkmiSX64P#1Gu$XQvjo7)&wCQ$u86OMxIk~tw^&EqI>-i)C?ly zZg{}W1Wrl8c`Gi~c^6(uO}o9j@Nyq0J`=*WsrS33vsK-maKW6DFc6YZyi2(`EdAY! z@G9Id+&9r_J+4NnHh^mu6t_}qR~D`dRYD|i0dN9A$C^#(BQiAP*)m1#0YHS}E%O+I&-G zzZ;7(UE6?^j*~%yIU^)`qFAACr1$)Uv31?34!Da{)H)5Bo%+1kM><*{3Sjym)J?;@ zU%eF9Ngy{&291oif<4MNc^q zev>_1vTEbV>KMvQG!hDKQpKoCZTsrMSMC5$avL@hzm`(XMDdTEC$`AQc`Hf;Qgjd{ zD@wetQJufoP+wHRB8Xg6lkG^&@5`pJ}Bq?6OYxZ{5QytQbF}Q=~HzyXk%Y$e7;U2-c<4NCU5`-75 zy;6%!?zH;bNhHi*kDPadq6V=zM6PK{d5~hvQN&DSbkda7K?EO|o$Sf|QISE;Uu#)3 zkkmFS+KYm1#*;wL$oDINHu@RKyUXE|af(XUf#+u-DRwyz(mtTBbG05~El#RN1r71o zcbL!?M0&zBZ59(Rb^7+=6Ghq!wNw$WhZ%yV2iSM0IjYq7p;GcIRY38rLkiqtppMmr z)hUvx)@YuvvwmA)^f9L)T@LcT53IFYw2zyl{)RILFuFf9NHm8ibR6^i`(zw?|(ja`2WXHw@Y%;`f^vRk#GF!HNQI@x10Op)vAB`q5bvy!~StT zJ__e}wThRY{MGzX_)>?Q=F1NRxV}If?iPIHSC_oAKJI`0yKlY`lk(%odEVJ=b*cOW z8eN9@1%g-r%_~HIE||~y1!w$yILx>E`_0ZCe*0;8zXcd><0I16&CO}MKG=s}ZQsv_ z&H8Hf-Ttt*2lZ$9{mJHWv%X$^wOeWS&+EqdaJ$JK?O}g-^nnzEt_?Flmyh$m^&Fd7 z!}>gY!iS#6=YD_G!aL0G>qB0z`sc;;;KaqpI*U(3T@k>kzqp>p%s!n3Xtlw%dAZ%( zI##q^;%5Hr3ci*c>|U|Nc2tO>S@=|22Ali!`F$_Z&Fifb%bNLdUfN|PA!zw(6|SG> z_{BFrU;lb@$FJIh^cocWUH$H7(CfaE0xt{n--a2#e6zic)=C@>_99oGL2_^QI}PdK z=~8z6<7d#^CoH-6hL4B!ynSM8{tR)wJnT1i#r^#!{i8U^=lZYQN?`we>m=p=G`5QD z5svfWgH2yWc8djDbAG-0P5pf;-Ibk#f^pA>2fN1X54%jl)LjqHdsaB>-TU=>yZ&bP zbSZm%8{4DS_+rd-quY%`qo3|>Chq!q{rb<%o*qs&&p%1m)8!X7yW5j}CK#}lMzM80wsX#&bJheFB?Wm2M&Fmv`-)D0J1 zK!V_~NH3}B|MTcj(k@gowNR%~4ffN!7ao%GE0Yii;7$dLs6q!xw^8y-lt~wLYfqJX zN2H*mZh?*klYo_sT`67TWL95NZPgT)+sLc!37$B3<`^7lZ#;!CS2;I?*$p;U9Wj4O zK4!vcSNAfhJt%jXnpmq&R^NA|{gEvUDo~M77k|mrz-p^HO+Aqr_W~7A4NmQ=%!r^0 z6_UdQ+lP&CkY2?0E%y6LDl#{`i#1ac;wvCoZFVZVOp5BLYfwtOR0s>bIrmLO(0N<2 zlDf!QGm2UchTcLw6_{b6N~8>7Lcv+XkqlE=ep20)ni1*DNfQOqU|Gw7KwYu}UDME9 zGi~;RL>Z@;`YCo)S`r8h<*zN0?bUIz z;TOjwC>TVDW`qKTdp3>@!H=IWN@7sRHF@fpjuAzIR1#3)fflLLvN9o#kT+<{EA0@H zj|rhgnMq|JM+&(_cwkl=sn15D;}Hg{A>6K~aS>Axok{}IQ3_qKjEI!`Lgh87HoZsh zw31J}E3Y*S zl#H@kPe>?rP$dxwg3O_5YCP3j&aE``o9b0C$#o~kFi}U0=oHX%u@|2j;F7zBc5ftf z5SA~DSaP4lb4Vt0lw2v{v4^o2l`OfYqPRlX{^VTRU>#Jf1G}7^S%>t-#KGJgpj~Z3 z>t15lJ8#r+CfW7}SvUk(yS-2u5OIvj>vV8u@yikvr%E{!WxBC~7vSGAKvPT-k%9 zwNW-PjxDcbJwJkq(6AI#{3V2>q**ffICiL*D~ArW*%+kBE#BLQT)d;C>gewFpGF+F zt0Y|`0txI)>)DF-^7xTGS1cA((3Ha@1z<8+vg2g-E&hv=@dKZKF#AKkT+p$Ovz08O z#ohpE`-x)^9hVNN-n34UhS~hWk(R2;EE#*iLSGalAca;#%<5%+8#w!OU|E7 zZf?#Ow*uzDPXuhV2{CU6u?+gjXclqQ>a9$zh=_Y< z@>?jLPSUmp3l*nH7)msPc5}GLC?FC-8`QLgub2fkYiicjowNW0Y?L|PYH@=fhzpBE zb$bb`HgIVlc{O=K2x?;}`ecB$SRfR*6-j%J=x3wAM$N(GPKrfO^oC-SjXec>f<3oD zv%0ZMCfKWRa_%fE~Rgp9+`k0a&foLjsB%r4=fFeq>v_2&<$mpF8yuO;2ANqw6@g)6)a zr52QSO<;MmTGt6=YC&|$4f}$x5<8`^)*zLw4#f$R+M}p_QM-`hlw#p`v!it-q0KV( zK3I4$5Yi+F>rX4loR$vDEFyes7)m;a3{DL>eSPsP4~&upSqOD%Oc29y!??bf8DTLX z{*9BlY;+`L5ir8!ih5^KRsvq$T;;J*`>URzRJ{rE-8p9h`9P{yVY_Zn(uZ8u-s@=C zX>T@@mau5@dYa%QRVeIq(0pYs=L`eJs1_@TOqv)K?R*_ElN!YyPZOCHoZ}&<=C_O? zk)XY@+~i6NehFJb;)MuR8^v+~9%#1XQfC~z-5Q}TOiok)Ib62+?y|mfYlhJMMpbd(j{zl z%%L5ciboCOGPYdV!Kh9m3X`)rgGdTxyI93JD#)52L5W3eZd#G%$Cn9lzfss`aGbSM zu;7uh8V0GKIz*?Kg0p6{qZ!NBVn@bwu5t$)@`l1|Sap~wqnJZBC%lcwEWA-{jkFX& z&K(*9>A_OUN{VxIUaKgGrbz8b;H`8eS zl8^~~PFz&IYeT(!yYX6*7G^N1*-CtP2}^Gxu~E4_~SC^Mu)?;LEkrB=eoV-lq$P!(C|Bt+$B zi=v1qNJxziMY6L{0VvK=GMeN#WhWvBlkB~J?cZ&CG(^UA(~2zr;?I9RAou@6NHqcU zj*t7pCn52UoBPdneK=$P{!I+;e_V~j^=DAr%h!D6ef(LTAGur6a!~D43Y&F)mHss}c z^zS?z_V4E7(Jr{&UagKdn~E>Ksn7j5-`wpsH|y=^)9@cwFG)%l4C6^(UZB&<*FK+H z|K>*~*`E`ztRG4Kh{$CH1&zP;bscl+I&cCfqa-aoxN3DkWz zueXmCR(;&xY;52@?hik#1l9epd33yRSsrphpkKb&^02q_yV+k}|8lm}P=)h(Z_-ya0Hw(Fen zy1#$9B!%kEw$#r+Ut4?jvF(oe{$acSG*{DL>%4QkwXM(lRlJT*-}fc`jMS~$DwWm4 zkN@XZ+`m8b>)8GFht*rHA6vO*DC~hZH)8w#{RQB+0&$PCqseX0-aBf4ce;OTv#gDL zexxfIg&%hNkGl#;cHH_x==-0){>tI%|8NrA+jhK% z!^3uc<9OmTwz?gVGu?9PF0D*$2cW^L<6W7jKmW!4I5QIar*Y|uEw@MJht!u5^W=7@ zkC^007EBW`$KrLAmYNi@0k_bkG6OM_BuVo%f0yDRT|Z3X`UD(Cj^^h#0+G72`aK~q z$&=`~u+L1*OASV%t4X_wloeG4olss>I8}Q|^rPZvu_uBuCs(^i<|Yd$C|(nug))O# zE!bGp6+mNXiYA(p$-FYaJmij=z4~6&PD+DI;sv55$Fl|U1_iNh9m`tGwbj!omer6y z3@p}X(E+8%leutB$5YLS>z;?|LB{JT@Pi7tscT?^lFgj?10L8*dL+TfPbS_=G-?z z@4m0Z(gszTASMxdvS7w+=mD2KC+t*t-VY z)Dor?4FVa>lBFz&WF{8=!TK^2PYe{&AO|u(i@R-UCk(q4-{qJIE$PK!s7dMcB(8%j zmL7_41-vR_qi}Q1RH}gk#>=){LW*rOo1rVnNh9$yA`KDZ#MTOZpeFhP_?)PedmsW1 z!AKiU&H7;K)2G8)a6{yFACy0Chu9X;2rD>@v8c6`{FaHaF5;zVYS4a^j7i2B9;(RD z(+W=gGzQcyCTV!-XGi8lw9|?*-ASwvSU${xypo(%HwkzSMIoEJ=@q3#a|N(y>w;ti zZBW}gl43_^lmiDTh&hHz|E{z;ygxL0NDM!{^U#F@KF#$=Fl1_!GOTa9z%KNuwa zOE(vAmZ(;;(w&rgSJ#?m1s1idA&_G@jLj+O`^Fv^+6JVAO{jK3R}1yBsk^UGwRljt zf!tZ6 zJL(T;N+Dg9E{Kjvpi_t{0};qh9G6g)oV<`qepbx733worK%%g&%rP4yDK1Kh z;1#y5K|&0u0`B-(${LTZ%tuF#5IZF_SQ9_KY{VkdUXLx^Z+(AzW*kX-o6H?Vk`%XK z1?!(S*psFn#ZE1RX;gOoRERxt%-d!c?u?8_KZ%|V5til!0$#~HDl@^@!ld#T ztRs)wsl2Z4lLSnyDl$lxn5P)|wdJlxl0-JoPz|1NgqHM*FuNF{8eE0{!ZtMtwyV8x zYIJAny7Rs#DpQFXl&TM96c(5PZz9%(!>?CthZPpVkC=~E?W?INm3tJUGAp%c^y(a@ zQmUv_5ePDvqerKlYBdu?P=dL$?ytPI5&BncQBTzKWQT(tGjQ~Yc@+dRf*ACOW-HjV z3(60}A(kZ4lMuoaU~s)U4`1FPTw5$n7~nbBOs1sQtvlKVJR}UM~R9KRhG=l9Giya8XOi|5D>U2O6nU*;WydK ztT}^tj?beWn%u|iJb5%_;`Acyo_NUHK%!o&^o6o<_Sb<>0%rNm&Xf11gObny@W2ie|sW8VW?9K<#sH# zDrnMrSHhsz4o+V{TgxH#u<3$Cxkx!W$pa?dJ99IWdb7!Eb(uEAPpR5MA4H3u&Edn| zE2Rjasw2VsgncVgu96I;Q$L1!R0direHiUvrre0Vp8DS@u_nPGBpFGzkSySbqb~IFsIikka9yN! z7p9()8tRl|Rut(HpzfMnY9u>$8N3ER-nllp9V$H;3BhX-sU4Ia>PY0>c2Vw#CcS2lEiJu@=?>*H72fETW=3LTL+GcGvS~}+$83PKQsH>=hVKLK^8oX%d0uwD5xh~eO zp*^p;R>jxgQ){i3MxClol~hgOFw3=C0WY3(2GEd<7D96r8ANu3nF4$9mK&<3Fmqwn zcXiWqql1c0fT=f^P;h@b1(ec3%-x{AS+eSO2S^Wzgi~c4YKa-unNLOc-fUFH%%hrX zXaNt3FQMVbTlgsA7Y7oTN=A`|lY=8iA~o#98s1I~2$cJmw*+B`wuo&X2vl`6^wf%& zr$}%ErEOx%8|rgw&)SHs`Z5*cEt%S37%;STHenRzvS=zJ9gZURz9VMT;(Y;)K~Yb@ z5D?XDmdxHAz#KM)^nzpWbgh&zQsX|fL)#f1k#fx5ylXhGeb^P%6CCpE5YZe3IxVDp(0DfPzIcxy@^le#xH3WsD{SgF=GvPSO{j>W7U6=VC`qQ5 zP9w8QnK;NrJYOi$9ppN8(4C@omXToqVm?4!n3$zpO&LcOSJl$fzN0)LtocY75Sj6^ zl(`WzE(KId)Ty5`A`E&cp@GZ$CM;@nzxeY{qR#z)H9YMSws)Lw9`^^w>MfwjPv@VV zv3Cvfui^#~K)U)O44=VwFCSJHJzwKX|J~E2F94oTpyMy=lRn*bv%A@z90*vSzU3Ib zokdf7uX_uAtl>P~?H?W-hc-~b&%ikbvp<6^q>J@)#(Ab^Y_@(uI;ey zy*<>YOXgiOx&E8S{lggsd$YfPSnmWU-r9@RIn!5uX=8n|;rw>He``C^?8nN6bal7> z;C#fJgMiStj+&hh2w%Tm{ciI|ANYTFcN3-VmA&EV;J?4w%Cs!3%QqMH;xE^y$D`~_ zo%&xqZAQK3VV=)-znI86xAx_YKQzspJ^229ejb@C8=4T{bA9iX;!BEj))ePX6m zX5E5r82QkHj)sQ`LLNgh(E|Gf8%w}&dkd%p#D2u_5J7~EITjU!{UXeI>a`dEKy8bs zZ!piEx$x$G7!iT*q0D7r8PZ<5lo%?`ipr;JTHE~-$tpFRUNu6onI)nYQ7*|2wWHW6 zEJW?7aY|Teo-7(i389c2FH0#li@~TalqKCeVE?>@Cy=3!k=0r3GP$GFj8Lpi70rJk zcq^)R%VIU`bDfiHB*f4Py=(V@$X;~Mfgr5*-czGG=d0J%WT-7hr5|kR7s6CBh45fD z@&QprzDlUJ8Eu=ZB-nmZoV*c`mMMq|he~zQy(ZX>Q{<5OkENaVI>KVkp*%IUhce`^U00nApN1cX*{}fiW!5h|tQgkIq38jW`#EvNYh0WS_>QEZ3S7IEg zU~B@x0K{HI=H!+IQTh`g)<#4{LRi)@>0xP=+S%0;h%SwCT?h#%$#}|;3UeDkYFD_9 zR^xuDF6Du#W{dn)xR!{5DZiEnxq7gp%P7PuilL=8M_JQ0VWb%mh`z@PhV+${IQsmP zAUV`DzSa?t%mEw-*do*p(hx@1v?9MdmjO_yzJ4cFtkFLSwy9|zC)*?q_|_j;nCG6R zMM@TdfYNsBNLD_K+zugn^`V1vJu77gJ$;iz7JxNZZ15SlSXDjgo6e-J@&Ufs6;Jx!*yLN@$9?AmI)L(Z3IU`CJoB{ zBR7HPx)E}q-YeSe7M{f>8 zIN2#vs4_Z5HD*F}P9msdA;gEY{z}skR@X}60%$b%bE1Yybtht_&jRyKmG}06g)SgE zaT{QJXv*gYGqMxci>JUnWJQGLF!54Az-~D8MkfkRMIjilVk&k>Y@Jlw%!n#R$&5U@ zLaMOzECL@Da$yVc)QkBZCS`JafT}%a?+CsSHH`%|ktMv{oMk@4acWSj)ffb!CukXs z3cJYU^kT;Yq~f5HsUtI-ZJ#iHLE7F8uEQ-$YNNi+g&S;VVf&()qM@mYEpD@pQsONW zu{k!rl#&|cv7aX?_D#BInXM#2Zt_VcI|*7p-$J;eP?1)i_lC6tJjvpcFQ|?NEa6Ml zoW{M_;%c;2d`JYWL;`VhpxR=xe_*cQNDa#(0qdJoS?7`P6me-w$b(>dG7#-C9^^d@ zL@*~xan!HUfrSLb>YzhUCFCFi`WLS^T=&Werv|CrRZic`(g`904%jpxf+g@G9RC?9 z%P~w6(?v*!8VxA-tQOCN9@Lmh#ypaEoSU$y2815O(I=r;L<$F^uilB+@kIkBeFIEn znT6<)@>~r@$l0pI3`E=#4IUa<>B2&Iw=;iG{P&Mx3Dmoth}HHDCRC(L*V_W6CO|~f~Kkw9ue$_ z`qqLoGf(ZT)6|j5P>hl%2kh$XnjpDGk6MM$8fAyk&w^ojb4;N~&<)!aD5jsT(oxM>J+zmB1+Z-ah zUiQ>9Ot045xa_oqw(2h2XxoNI~{If>AM}sB1mBFhDf^l|OL5`A`K?-^)IT6W?cse5T>8mI~ zH1JXYkwh6R)hxw2qW+P~+Gwh|wG*2@B`<NiAIgJ zg~tV;MN-BJs&=+Sx`xWXzs?pxq-i4hYyfINmA~@~C3;57q(_HbnbR^rlA%|iONjv| zy}*KH7@VlB^{#`uxF`TGuJa3LNK$(`>;X}>6W(|j)t-vPMJ|!};)8r`?p%iic40dW(l#L<%OY8OYt-^FZ=dP96uX#nPpdse?=3)ZOOXf=@`|X@30X8I zF+!7FvON_^Ml{%o7aL-0KXxQ<10!SmO#P{Jf(^aO-WPxW^Rc@Boxo)cRj2(^h2Q?{ zxZ2KZ2Pz)wN5T$PIeXEQE{MXb)yIwShpTUY@x4%Z|N2k=)lA^(ZDsCUV4je0;oK{q5amJ6`}1>m}{|ANCKY?LxnL zbF)9~gmly>gz17&*zWw<6V>5c2u;fsU09Fv`4viG3v_ltkG@1aJ9q4QWtUj* z-rL%FJ3oG$=ZY@<&4c#g>f80fHtON3L6Pb^e{iPQl|Oj9{?3zW|9EQ;vvb1Rx1V0n z6knpWUy}J&u)ME^+xbQ)VjGaV_2FUrsjkOmM<3?<^=2otaDkKl{8Duwt{ibJn&0(d zTT#TC)}2j3o#WHv{(f(J`}}l&_ZRjSfA<#`3+fqByl`5N!h3ITKW=Vi*y;dpZMyaR zr^AQ&liu0)be$CY#;^6^AtGDapENHGx$MUQD^{wx|-^~w?v$Oh& zn(FGj;;Kzq?7(sUVDEQVm*4!y2O+&be2y0OhrD`*U^|ljl!sU2`AC>4Wv3bkq|Tl| zlLT|pTQr!H8WIXIibeaDFv{P1ewlux&FMH{W@}+7~Beue%Zb@Jeuw^|Von=-WLRpPo_(%jC z7T+VJye_CaO=B?&K-wO|D83=?ZOejo{};ba8IdecWfFAY9UN)m$a@`BBz*W3rS0UYHXBeI7-0+ zl!#Ml!UQ@gjWm2+!$G=(l!OZ+@&K24elO`VFkfLb*F|#c+Bv}0%VM%z+8k_^QH_=)I{uT5wtKjJO!P%yo89Y({{k&LF%9g1TxrC$kZK> zY_At2*cNIiBN?gLHGBU;ewyGm>=2m?QJTDEt(%yy^3oF5u@@kndJXIs1BwxjlkJm0 zYugpHd(wXs*>s^&k8L~iJ|-lg>S9AJzl<5S7TI5IDya;i5*v6cqYN~WtII*ACz4VG z47>O}nn1!wJtirKq2jOfO1ZOS@c$1#VeCs}?ZCn;-SB;$ zh>d&!slZCUqaA+Qz$9yHfdN*$rPW&KXi`Mlk=BCklA^c|+!6I~HU#AL!Ia9PyJ$U5 zsQVoxnMS*Glx_<)&?Z4u2r%f3QrK27DcwS6%4jp0Um-sn(0Z5`Dn(rIh|CQwJ)u$* zYnXEY{&9$-nsh09E&Y~OM!cRxnUH~Y1Wt%^6ERrJ8t8tBSTHEW)FD)w^rJtQdMatV zbW)cZkd)Hq1TvZGc8_Jd$D6g7)%1-rG!gr2;`du2qD2f3ww-{Q zJ@wF;E$m2R%Xz;k?JsfyNPMUs&|&AM`m3owbX51%%W%4@!dQjuMUwyA(^+ngNy)@g zWaFfiG~|JTNTJ~r7}c{%BogM8p%CUEv0HL|gyBba(MdGJWO$sN2nj7ql3-qY6Unj0 zhIGQGLsN@%ujVT^YiDtyL`6=_0ON!N4?$$$NPW|}|4#NT3;Ry=t)X+sCK7n7eV!Ox zrNj}@yN+sDLQR?>v`MOE@Xa=WxlEF@f_p>}n-f`%S88EYib5wO$I7BN z10r1YEV;%6If^>iDNHasj_D-}Y|4bhN`s^@(X;Le$Zx@Y+X%18bxL!}AX`jMHaSWd z6nKK9RWnkSQ4|2mV8dSMrZ6_$5cSBD6eJD1d7)x`qdcP>I;1e0ix8k>dwW?RxJQYS z;Dd}6KU^|; z+J%8q_}yl^wLhe>5)_{wc)noh{rmk}d(vm_-Dj54-Tv@+_eP*jaj8y^cVd?g!@p{*0>l3|4)9jFwxuMD=X}53o*}8u%zHR4VDGTdLRUoWGk^18 zeq3+1D|_49`FizBv7*kOU)!&?S?`V)T))h8Tp3rLJNIcuANRtOZa!>;Z$1M`|Jo)V zZ*#F)bh9VwTdj3}mW$=s7PanXfJuroYh5#xAjWx4F^m&FjO>-38fkor2?; z|GAE5-G1}BK9)bto8PTByT>=bn0K;2r`yf`!{&JMZT5s@HXE_f7Qfqwsd^z3_t}v> zxVOEaU!E7nd8$6gGjEPh3w(cbI@VEFw?GAqpZB`oaedgI>?U?JYqfuVou?gfzd)C3 zm%{~du`KpR{TMe}TgN`z+j;Fv?)`p$y8xw+cl){SnBy5NoUvzM5|{EBN`GN?mQ{Xx z-j#LZt?ul}53z zL13OG#br`m9hz3bt}+u8=t~;}Ool93P>hjL;%pcrHbKfsMgGY?522wi^++OG_(5s4 zA_aNIpotWYXpEGMCZ9bNmA|kH-kaCKaW-#oY}{l2iMilZJeP=q#3U(-DpPJGU~3p9 znHq?erYwD4a8k3f0EAIh07)*Slecm_7*aBa5R%Rob;hQhs|KCstszcz?3HsPq**Us zEeZmny4J8S*0tsECl-M~jz>GBydwWA6brVZtH4QMI*E`(dilX(_tYfym)e-Z0ZPcJ zwVOnig>j#SnC+x4Z*eo|RdNmJFr1V7KFu`arJ8a9LgV9p9bsS!T zmpo|`LvEPlYS@-Xvjvo!l12$p5Qyy>jLM|g5Tdy(T}I$hC>CcmI$?DJT`iJuon;WE zA`{;*wNnpr`6pxk9DTVG3=wTPxV!BsD*AJD+uuV^y#dX*!Kckj58bE2Ytmi*+1CvW8?C zg98*=dm|AeDE8ao9udN1)1JsQOYmSAD9aYbqXs1$k!}Ynba(Ox8ht7stfBN+X-}h< z`Kb_AYwctn#bE(lGg3*w(d|V)KTw^>j^iZd(u;(k1L2c9>uoL#Ff_WX0@KYLTQ_AH8fyU&+$uQ0-08dEgq;>+Lld(_qG@b z5xeFdq=v3tABY?lYi(KFOhma6)eeKWsE|8qo)YuoL_v}=<{fdV+eRDPa;yedSgA}v zDjhh1kc;Ax=^7&8>dScnna*1m+9nR}LA*^+p9*Rl!-7GU>~Bw}x*c71g3?AW;=`nB zW^Zc_@$Ol|8!unKE+7j=NU%^~VrXJfQFP)uLoY$imqspMTTo%fx*@)-V=vu%BEYiM z&asdhnMq{SHUTa=30lVnW82jdj_BaP@J|-(k}`vZ(}e<|Bq@NAd=oS7)DI#FSNk)q9VdJ#zFLP$TFM7e~6B>?Ja81&1#N=I*R&4P_;o`WayeOr9M|DZE!3+ zIU6?68{60q%^F(HR{H^VhlyREygVUEh{-3bosEjPy{+r6A)R*1WoCuKs!84e(9eka zc4BE}wK|X~w`CndL%-vT#nD4leKo_`=z?njTWBoxbF!#inHo=hytX6Sb~`kbSumjf zV0zvhM|&F}*-aA3f+DNgMadh%YMh2l#eQ|5bxqQ;#xClOBlM@37AEWh^Kv8hAxNpc z1ygpejggS-N@utPtiZz^Myjnx{Q%3t-px(S!$AQLt@zQb7Ir6y>!zosEn9L?JB|iL z$wta{n&q+yrp_=k@(IxBYMM?6$+Li3c?Nv>q=*j%A9_2lnf_LI0Bk>>=*W=RGHS+( zZZ(qbQw-hoivF!Sr!g-r!VJXYCdnD0UU#2Z9;0{WVG4#EB--d)`?Al{%>Yv`{E0B+ zbfClxOp9k{e!#Avpgb_)_w5)LYdZK{=5aRNW8peV6ucj))k3V-1+#%Ju}20$xw@jH zcKcwHaO#v#52N%D!7)ftLTu!!iHd_dFCC)S3|QR~5{t)Rtx)8i#85$o#ezh{X(!Qh z6w1`uF>VJ3GENV}h#g!Ow+nMe(0y~zBa(8~-euasMJ%;oChTYmO|0)`LS_rZRMxvM%GK zW>iSMC-ZoS=x4AauIX#4>AupnB&BZWgLzh%&KO&cbH9neFX*94Bml`zvYPm*RcH!g z7o}fBbv==~jO*tR%Zcs(M7#!qKbD4y&<(H!UPu)$ER6Ds?G%buyU=N3u27ts)cBHz zlm-QCcC|({Nglnt1Xl2fSG|cUjS5F~G=Zz6Y!NkMsqsFMkcypF0{4=iq~Zc38^&>L zt(*@q=wiI7R#~kXH1Dp+LX*|2&b+}BQ9?t`3$psirOVCjeRPGd$26Kb zg4~EZ6!qS~qEmLH5!US-xvZ8dFhXWB!E;ejup>=Wk;--rG}~9N+T|wd@)9?A*XYF7 z)S?_Wq`tBPfCK|Ait1pmoAtB92dux~!&8e~@GTv?mLS15PM@3bO0O=}UM%T|Y{&iw zBilT1JE(2j3F!!I{p2RJb+ep2M@)M}@b+`85*Tp_bLEj+d1;JU{kG% zNrjTov6y$nk^^HuU}jw46N}=&kyV2T$yqq#G}L{E)VEFC)kzT1E;V5$JF1XLs<(~X zXkOL#Ff)!PtF#y~oo%rNCd_RYl#I=(67g#meD$e>&?e11GXjt&RJ}P+P2Ik&K=|U% ze?H{+zagINyY=qVGi=lmv(7L2gnj&dn`i z$HV@||6?a2R|i1;{%-RbK6bs`@8%~QqWQTFk*h$vz5?x@-b0YT)yKQNz1#8h?%nL~ zy8sNg`{Nn@>Y!fxcfkPdUC!V5>a^XhKfO8bPscYu-`i6>J=TxVEKv+&8~`*zbjczB;2+_w~`vPvhX;3nt?8QjS6iUqYaT zFrID7Z!c)JXDx}_)mF%UfA4?&r~g{UQ9rZW?F28@b)7m~DCLK}&G_S|)z)c>fB(@g z{prm+^%q>+$ez}oNr>$;_8friTlJD}f3gWWukqj8n{STy`p2IF!W{&n>?sI0pC%!P zZLEq5h~bn(4IIdmSe)4Xokr}jR&ga#2WFp1g;QRQl3ER`Z>`{r4oHPVQJrwW@T#^p zjBWZPCJ_q7B+yRfv*_Sb5->4e57pLVR#KM&rKxLmApGXNIMb-NY*4gE*cRmtDJf$y z$J7E5YIM<|`gB$#2$|tY7=;Of1)IKr$t%fEw#dIF1Q&TRNni-RLp|mea!3`GqV`c# z(&QNTTG*8B6IE40@vtbO%bxPm=Il*S#P%jOBtlTzcBDu&WS6w3eDx?vJcPsyVz{OF zzb_Hc?NOmKDSp|P$1p;=cn}!A;j%vGGYg5=B&~^t-m1nuF3kz}+%ieX8%dx`-y?2p zR1K#`DZSKBZDq@YeT%mn5-YvFE$#-a=+@DqG`Nlc{Foe&c03~W)%AxRkIwBEXl-}V z{Cvs9v@1fy5h#)v1bbyJk0Ry)!UX1y$PH7GZX{UKL185^eB0b26GIV7ipY|NBpp4l zfG(1Q1!>_l$u3(ML#ubPhiwiXU%Nv>Mn$Q|W<*@6^WOdoYd7UFSd<96O`RS@4%15+ z8TH~+0V)xZSd#^i3Gs;HN5(i&d9sEB49DM&4RaH2c1TlSjF z*JGpAH*3C5@X%qdQzUED37(=zy2ze7#1TuTLsIyT!F!%^)9GH%j3M=O@QdG%kz#sz zY=ju>{2|t8?-ruAeEX~T%_;poiG!KUqpPtu8f-Oppmz5c>u9X0~Z+3`M z{7DB5>6;JA0i*Lc3|&h`-&Kx|#!2%5_F^q$RaHig4SQARb^1sy*C=Ec6i!rdf3IpTgYhYxd+0&%z1U>~bd#H{30 z4$@U8t_C&8A(UD`04V!P{du7LTAW10L*gKlQk2B6AQ~C)wuXd~TaFiex!BzQjiJr! z`71;$&q&sf3-D7A?HScm%oRY`nkHoS<= z9eVHkfS-(=u0_6!s)VTK+lLS+PapM?Bk5yED~1}8Bszn%$hkB3q|jQJZUFj`!J9CA}0 zhKSszIXWdZO4z8{0zSDJcE?xJ{F;BC=?s~K3v38!d{!pa&S7=;T3rO~52|;g-=f$8 zN%YiDT#yfE5R(;2E|{A+Qq^^sBtlCG0O5e!;R<&1G7`!_8H)g}zhG=V6pw@X451Nc z>Rh40314AEhAFCt9VXsfslK7h^ZlcqPr^g0$1jaU5i0@Ah>Gr#1rBHk)y~rD?r6LO zfTq&x|n+Cd@1VQ#E~JKzJH|dlGwul!Xn7Ygd$h=_d>--aheM@Ody>> zK{fS}$6^N-opK710#f5no?QahQ2Igml4{7e_atD35<-z&n@Q|^uVGtq!$;s5Qo%n4 z7L}SriNKK^LtzD5KnPD#J1DU%XWR3b(C7y@>11qsvFcG5v~tr3B1}>;gSav%LZ(CB zB1CnQ!=T8uYtwP%+HTSh$VEP>EuAc$o3D}?X+lT{JWKGNfKaH^ zhlZHgSA#m!k@??3sIF;3T=J;`vJ>t<67Xd(uct{hdK7975wROejzPtW_9X;e!W}(2 zp)jK2bn0VBE7p3Ex|4HP>^2SLRC~k@jfP^s`UIlFqm#Uo1U>Ot$SWXy_}H7T+h}p< zCDo@-ViZEwlaN{V{ydImEJ|%u>+P6RkkL;q>fN`fCI!!Tma~uAhv?N6g|B3_q(DCc z5X*uO0X6X^+MzZmQoIS^beRLE zF_QF_8N|LrEU{AAxv^$-{R&A)kXnHusYarNIf}D}p*ESbt39^4?D}NRt3yv_kpbG2 zUg&ew_RM9HXe}n$l^v##Y4A2$C|#A&hYg(SI5Zi`B^#iIMhc9Ia{uJ0$g&LBf77Xh zx^Hr0ev&HC0{fxHhJd|@%Gj&U(e5u-VF<6WbYgUpUxXy7s>p#uW%D=3Mp3ZZwGyVrdY*SA(rug1VaE;n zBU$9bL{~bt%+@+FmHMP+jG)60xyqyw2Y?b`Cjy2bLXl!(#Vldde(Jw)%@)-EO_j*2 z-9K7abk4YXgo>c#gaCoYua zD3U#n3(U8p41d%lLSSE<{txc07@C-*I#r7ssY?Hedm}jNos!+u{u6VRYiXZY;1HxZ z7*w*QB#=XuC=H{GA*7RRvx%$=X3DmdMWmD;4>}tAi@*Q*aNH+sn8-sPH@nk(D`wQg zd?Ucy8G&{Q&(z-})GO}Jo5Q?zNT>az@BihW?>F16zVcyn+&u0N&PsXynj_ds{+EA# zg8S(PuHmtIdlL7o{dg79^B06{J3L`3m2Y3*Y~R}T1gTqiNd=%gRBF)+h~}fKm;o>pu<8SA~y+G4nlqvknKR*iAX=C-J z57^gto85Z##TYI>dBVspJkgg!pr`rfWY4wV>FHPBZuf83TLF~?-u{KXfnCs9aDTPC zo&WffWcp}BzPo)RvSq#OdUM!TCfVoTU2;yJ*@BDt0kqKG&i4GbtDmoT?|<0;U;`7c z$LHDn(wEr!ZeHJN8UEVtY`^}(uGW?3FP-tP!_`W;eDhGANO%kk{rdfaOs^a6&0*d;QEz3X6=$p{Uylogu`O%e&6!>w59{6W zVSlJd_3gaT9DGJhYZrZ3-JG_MClURwAMF=68p7jwx7Mf7^Zj~%`e6U~-RU@=Y=w;1 zV>_H*)|L3p=G{A6<-hlR_(i^k-LZVruJ+@fZ1R5go5OK!>nmMHx@-lU+xNeAp#JK6 z+qs+D{mFg+-~Mdz+5Y=%0#0@6%>t%jyvVvJ2 zbE;N{{r>)mF1P41tM%>e##f-(wMx5OFVQ~1P$A`a4k4F;6xR2Q<^5`Nc)NRhZM*W; zezaJD?fRok&+bvn>ZyC-w3ETQKiN)rSbw@Oy31-=J+A*)amTek;*))CbNhtcUcHw! z-8mL{yuKLB**56egbmApezm$cLw~g8BD+!-ik{@9Ia>M_s_m&ybNsZswSUv9zMFk_ zZVt{AfAEHaVc@D!u#X1XwN1{F% zEP88H>TBSdL+St-ti~o{1vMY5Yf8WJ;?y0+{j^cU9FRhZPL)UCD5(}hw^d3c>Fzov zm=943qk}wkl2UAwDfZb2gP4ZlEzDM;CQK6D9@d7^)SFXCPLA+$6yc_E>h1=fZPK`|EaXHEOWV+WkH}CbCZKS*pS0ougYWQ5Akb8l zY&(=dJ)_eKF_iVsvb5#oGwK>BTmUB{Yl1J7V^e9u+|*rB^>NkBzol z0`O@N`V{#%#R`ucM#1UK2}WUtqaY)W)=2}%t2v62(km*L#O{#I_M$+WfU5M1X>jrx zO0UHsponsBKWcP=Jqr@lvJa<=xedj3DA}gsn(o+PTskBIvBSGXJNo2GhdTnKsTT+e zCDoK1!Ql$xPJ&A%y$6y9rIIP1BX6}~)$oT^)*QiV0%j!uVWy}K3ZgU`vxr*}tEy>) zR-Ym`K*6k-GIb5ur+N?O{{?LF8kG10Cr>^YfGfmzk9h7uqH(A%Xt$mkf65(!qG5%$d_2L*m0J3W$~vb2#$ zDpDzjK7ki$E8X@~b9|mn^3*}|PZXlDrw%f)$_+%$q{8Tk)3hiAf+eB3N+@FH~~H zg36#Lv6;|87=?!_lu92(qC!ZItco;P%OJ^REOAO+7;(ZDyt6*ZAf^~}J#DO}hRUh> zKqB8I8CsjJAwmhP_lXAw-Zpb9ld;52_5aO{jfdxVB z{v%Rx+T+Fs>|Ixv*~O8LgsJL?!cC^wz`cAUO0tC#J$4@0)sRkU6ao^W7j+CwAWW>T zhlmC5N$rnhQk;ngPJ7i0wX;b_RFMUSLJH~#Y%X=v*f=id-9vJ(GQAP2?aHhaJd6i{ z>EVHe{lo|@*+)7_!1k;8pTZhjKK+Q)d?M`YsduHk7QOVs(2@|h8l*d7zA>LaO$n)e z#DG@_FDW#xT@S)gLD8U~XA9nJ7j#*gQRErWL#n>tagWDZ~ zdd5-vd=$kK_<_c@N(m1#c^{W+OaL7I{Fw-<_4AabX(gzhaME~X3AYfTYa8VL{?goq~yOOQwqWvJ{p942BXHj82N!5w`$1C3v@ z`EHcK{g@0L)5rQPNU~rMdIpBo=Ai-H`*5q~0qe#B^`Q+O>N#woQ-i%)LSLm@$jeIH z9_vLN%1#e45E;10X2S%Yau9)DqNe+7aXNC>W}CYSwdu?r+nGvUw2r%W`|E+tUKI+$ z)X1U}hfGOO%NK2b38}+7iVr&>H(cnNjE-Amb1>;ZDB_1CgmICn!cbBkHwj@%tq(k@ zprkTxd>h>EH69qLWMTifxwZ-Es|RHZu_q)klaL+5iE`tU==JFA5CaYYzGdt&*_-9Z zfjb&RuhF#jRG(My+m}93yjGFTRyIWlYHLqDX(kPFRR`)5x7U~&9K64xbbtVU+V|f6 zh~dUGwrswg15-QRZpv-Q+`xKQ?;-V92$)J{G|KHQbP8HrQApP-d;AG*oE?CDqm5Me zt!mU*=u>|%wcgvG_qBlk~Wdq{;@KI+~&!Bc$CQA#vmrVkQm@Rvso%kzkxv zyBxp)P#GSDE41V&0xQw0nQ%hdN&skp*|E%lq*xJP*G6?d(4&q{k^uu~vuYj*<7Y)q z0Qcy)*6xx;P2R8ObId3b&|CbakYQKzVs0P?HIwetT$F>6s7*m?d7<1JQnb2lEgT-8 z+|?J?NTi=6e@9NxL?V$0=Il508NLSxk@?WZnF65=*f#V1O)20aHAM8zKf4=nj^DhR7hu@;RDT3MMw><*Qhaof)8=0*i++>93JWNmvmNRlMd!RfEW#O;Vp zA_+{>#h-r)wD@ZWB+eD4xDmX#lD`Ui_Hh2>m-~bB_MXJsdUZj8x}X?^&pkYDj*rI* zM7!YcbztcA>Z_w8c#n3s&whXN52wvxf%iU~DiCe8pilqqFTNIRZvSre)#35){^EM| ztH>#&?n0t^JAatB`-l4l`YDdsmjs|;5aQ5&97HyH z@sa`|Z)_0OhwZ1z$GZ%OfBP_RH}kvW^@;~=^>%hhu+x$rE+||VfZ_dqw?5q5eR=|0 zE`LAlKdrZ`ckBDj)>)SCH^-xm$SmI8-KYBGmm~1FU#)NVhcn9WVJ)uR%Gny%Az@KCG`-zw@WQgtvaXIjj%2``y**>;0$W{9*sc`osO{__#kk z&)V~x9nX{YPGruHPY<|^a(%Cx{i)7?#`;Yi>rYRNso!trgFvsF_ck`Smm~OYb8yV% zMu1Gcv#`?w$=WpjkH7lE^XuBb3+A}Lb*N~N42ECetDE)hkAJ$~EDxfEVPD+re5&S0 z{p<78eeqj=Sp8{cvnIB>l@A5{#Hz{@Aikf3J>dh z8>E|+Kfgw2vpcUv2Zyd7Hny@C&e-F8aJ>6*8JM3P9XPD-A8+pV`|avv53{u~Ea?(o z>`!A?6h^gv&DeHd3B1n}dilQR=hrS> zD3_lfQ!C=Qzu7o`+M%|WtjEhTS?2VKZuwD6y#I0am-Fth(X9M_f7*)I_|31^r^my5 zbG7>Mf4SYCv_8ID@5Ezmi}_?<{BH06_VfARQ0Ki3$lfi@X}kJ%eR_Y|*kZ0L?9Es9 zug6EH7T?>Z(UUuSx$oxlNIFfqErSJJ{KAs_c5nOfvF@Sz)(dv|>4l!Q(vN>SY;Hth zZfo-X;V}D#w%=XJs;&NdfBG=5PgkqoeY$_U-}>KwxjudHfB5?JRyWWC>bt*~Z}w(6 zJ~$Dw4exflvaPxOP-ns){d)CN(7TfAYTH$62r|6VSewiZ?_Oy>lr%SRru4Oy4kbw1 zW>F99&CehzxvXxlwvcDBzC>Wj=M}*Tz+;LMhtLzQzHbXvA(1mlofS3LQ}|Wgjw2xf0Tz`8b*EiGZxt{)jwt*%P*Z;1e^x4lMjQ_OODQO~6B1JUdWGb)O& zcCALQnZZQbpfp@;op<@hhN5SQ5PC!=MUf_x)|{(LqVl*>y3KxY17s@&$sPr`D@53t zqzjI`SK`}g4MY%sNrB?6gv}d$j%qLCD4u87y;)ix)RH|Ya-rZ;Axj~+pY%jBs_dyW z)Q|wz^=n4CQQ2EkLld&0*t-<03=U0>{*0yO7w?seCg| zLvGm1+%TqCpc6xo^@Pm4?r=0`XdQJs+X6M4sP-`8@uD7rhB3j{k5FMyNC?9IBLl%C z$PMHjZAUjmhQuiXFF;Tl_BvBfBAYSsvH76Ri zq@MuAqUjxRPy?n6IDyHN$I8Rv~hwBwDKm8(OoJL&=TSWzG0bV%mhIC4gdS z+%`S29qWzAh<+f=3@VDm*bgnKR}Bf6y(1UN_)$-Q@COY~Mm;5EnxL)}M{yAnls-9D zjnHNNg!JHp2tv@49kMh0klSR=A<>Np)Ho2nBS^pzR#36|%Si4d&xN_2#JD|TbD3?1 z=G<8-eK9n$lGQ;O5 zl~oU}=;c*P##3=nftzfUdGTdz$l=^{YHtxp)6Q5)s1un_=YBS&o)mt-VzHQZM5Sw4 z``E-+HEq&IVu#bBYsG513Oi~zf_>pR^qE{3HFzM9xCqG$KE53#qVb~>$Sy{NmXiv8 z1JTO&LWEO?qA(R?Bg8k{$b58_htj-!L@7!lnX+iYznI&M(oi}$i0)CfICQKOpa+_& zY|lmdQ+8fX6Q>-5trbXKPQ*#>$HpWLb9(u#P7A21dV5_{v+gSMT^sszONM8HrxNX8 z4h^Ys^-s;IC8IJH@!g{ghM8?_2kcavrej$g+lZM_9+pFARnl!4V^d6#019Mp1;{|8 zm=w-SXb9t~Q5QcbCS%3+jIByZ^xw3D#TnP=0Q&uVHAYjvy};Lapiw%o+8djGxRk7U zhw@Ffh15(#oVPMjunsOsmZ!k)1MO}bd$O_8`;vj6uXXTT3;a)Q6eKX zGCiF``ht=vaIp8HB>-zBbRcCRekGJx&1xms_pO$D6CbgR6+Ye3_?j!JG8moM}Mu zS$=XAi;^H2uVyjKGMK4MSm?GlQLI2cA!_|^+`r|qVs1JJyUM?${5qZ!p~o=Xsr^wc zR+2}^VH4HS^wMvt_Oh^`rn>Hk)l_{uM9v`!boCX0O&+*TnZ- zS1(p;Y)7{RNg;%&DhR+UW(26vDAuEfQt=XS@DgH?tT08Vk|Q7?59R3S-7E?l;0M`* zDr$#L!rPFGFenR%orqD?(bLN@pRS8UT84}%FDIWz&L0sS;DMDCu_FPV1Ow65M00l< zb(PMksLkak`d{%#kQoT#rpu5~wd2L?M!xv-pAYK&e*$Lqs|x)SJL+e+!mA2YeF6Vm zK&TIg{fCWHQ0iWe4E`u=?7Qv$=7)-(JQHfJ=0DzT-fkWjV5_KFpRTq$-Oq>p>9{(s zw~jm&l<|)b`{QOoqV5Ej{Xx*+o%8Zm9OnITwwJJH|7v--GeqzE_1mr8d2=Jc;?)8^ z{BCVuI?h+CKj?{fkE;*s-Rj%@`@02H`1Ox}`p5NdXJ7cM**^Jhe*DMH?)`kQ@!r0j zZ5Y4Z&YOP}khos&xAwqCpMvk!hnv;cckA~yvi9x!`F7r~zT4k!-V5BkSsq`gO9v&M zfv_*plyxF1092f#_j}`LGi#*rrhP z_T!&!9l*FcgC>7&Q+v1H-fmW3h~N0+h|XVc^va*Uyn6cDZ|!{q9~P2%J1;0rz3%RD zFACwG?@tH&!**VV^Vjy)`;W&T?6SwF%YJ=lkLF8Y^;;VyTaeCn{oBo6*wDXP@9b^% zUoHz*fL8%Q*LK7Gw(_ukYY#f_R$sk;!kzxELTDY-tM{IFqJn-qKYpAmSL*8VqT=KA`1Gy%VRyCivE03P*zS1=Yf)Cr z^ZI_j;J7utZ~WQ~B*&g?+A9nkXJ{~`}z4O)<&3i$VH@0y_e?1>BnT`M2*YtY3 zy8W5#*mu5`XMgU;yUpQt_1RZhe9j*nU?c0X%H~sZs!K8&K2yM0nK_wmwAi9Ynd|Pru zq3x{n=@c<{9W*L5UXz4GF&jKhf|Ub=&w|;E;zr{jMIPcu^3FKuIaMbgu>YxDuStjK z``N-buhY{s{PnsG1XV8Q{ZgVi=LJeiRG&3>D{|-B`I%uOM)7*H0HpHssAL%h z{mL}CcI=_defH8IHINLF37n9&MOeM2hE_8~`WbN?My!Aetu{2&Fr9Zx#ir@P?O*?j z5pcSdCcRMWvSLytB$=JbWx!g%B>pL=vkZ2QB+M?GkCy{#-Ach5g^uk_rV(ozX39&$ zlCET;^w>ad)*EN0M)I$wy9NSTiK?q_KpJoA9#Op-TnO=@A-c|C{z$-_bQ{Xnj^gkp zUe5}9#_X9e`!vClr|6JmR@~l+mQfn)#0rQwQ6D|=beeM))t*GlK5IN7l`3wx1}o4x z8&o1RWE3A;0@{om?9?D`GaHizJ>tfr?^@MUH*`UqkOZxclKdtFbspK&iJ792sIC5Y zCc>m16w1$%%OpWk$*oW_+Dv>O38ZFTkb?+d z)a^h-YBe1^l)<+BZ`+MY-Q)}azBiSrBZRnlrY>0sA%-%=2y9S?AOJKb(X%7TsV&1u zy`+Qi;aHazBA3o1m)W3jWI%YFbpC8%y#_d7QVi*1rbcF?dO@bZ37B7ElDOp%c*nD#w)Sw6ud}->Cw~<0Y4kuM@0|GXk^o=U!ssjonPO1ml8A{vx zYm){l`3)(D5ll)S%A|O+T@#{!L9V~P7m5`riUM;krqZ(3J30qu45m|Hs~YAScMyiCHfSsYLJ+RLKxXupHqVQh4EJd)G36P;k&@AP|Ku)Un}{c1Ksh83Gav!jaOe7m2e`6TC&6 z4Fsniud+4(w49Qe0YVYP&_Fgj$8eR>qOx6!9Wj=@rZ3PwOkkqw7eKe1!P`6D3`e*yBh~QX`UuOGP(mN^mHs7i@$ig195h4Z_VS zY+@s4QeR!kI5JtIriA)_)o(abv!#qr13&kJ#HIseWYh#QBqo?KzbR8{U1WRMvNNem z57KZ-?pokQ-6>M4!S>AL>RO0Cn(Tlf>d(+)UdN_TX<2HkmM00_RWbqv& z^M$02R3mTfMpwJ+JK}*z>a`=!Wt>!^rT4P~q)=sAod5&Tm1Hd|WLs5e0bXwrD3ff7 zdCviDejXJFp?s=DugEfF^Xy5M#AaMtBD6hiJS^-0lvwSypa$Z>wnsy`;6^~_Fp@@1 zQbwUvX4bJvIx0~w0`ZP2CLO$p7_B4OlgIn%i!9~>%1Tlf%YoBiF(^hbYM_93oI(c;mU$B81mr%=6K}M& zs_z%E;nj?mg?h9{Y!guK$}y%O>gX?=|>ZhjLq~;d}$t;mys1{B*4n;W8U_0DmMrGB>I@tP+ zkn|W7N00I1&p!dx``-^#W|tQ5?VbH#|50$rPnWQ|+n@dBa9n@7TD_YEffFq7b0*Ql z<4YRbO9ZkWX+i9*Z&&->41l?X5O^zOfB9cmeHiz}iyT zKDWXB!h8XLod4nSalT$3cIyx8t&P+dXGr5275kGZ zFP4~p_j}FV%cWN_u^LlHe+pH6!5I6qV_jsczg>%3_}825X4Z_`{yElVu{`kZ30r!y zUw)c*s~`XAy*;??!>_le-KPa8dgTag$IX8?-)?3bU=7oPMm}xsZMe_hw4XK?pM9=H zox8EWX>nXRPd03d%k8b@aOQ=6-W`5djidfggSXp@yZ3n?-OeBOj}>>X#r9!yd#ca- za_BDB%@a)i;%^T2ewwbAFZIpo{g3~F`86GhH>vfhhl+|cDw!l`QuP+ni~=)tM4u+rX($|I zQGozq3<^XeoL;9738WNU@pa&{96b7n^B%xutTS^f3b;Ny=Qt zsZVKyM6MVI(ukrtAyU;yr6J@1QdhcURw2XUDFuvgl&41Uq7p()s;Gv%qE9C1b?imE zM)?78Q#2Hd?$Hb4MPTVDa|XW;v&2ff7@$VlLwmJPDS$5Eu< znEJ676dy5KXO(=Lo0LhVECe-{9-C`Z6uy{n_)?o3NPb(yAQ1DZ#lFL0*&7F{(N-=QM&!8wvf9YFnr z-EJ_vU8R|m!D_E0sv_hLOTu^(DYaa)hU=pGa)PcP8)TnD0Z4+AD@yjZokKBOy&{dM z3|;#sBGf^}P3`jmoA||qRfrO$h(?ggHs&V8NLZgiEak{aJ$VL{pgx0e7X*UrMX)j= zH)<5$JAt%a1*|bB`b7G9J<(D1L4_kkG}!?=QO0q|f)G-=d{9$2_NOaD$~kBeHld`W z_Im>&^a(3{ATjHqXB1FhN+7Kb(e#c88Wm9@P(NjhImU+V-M2%QF-Pi((g-cPofCx> zIftWTqqu2H%xy$llq`ZlI<;h8W0Pg}Q8znBeD?BB8t7N`Ce3p~0(=We8nObd1q+@U zjmNGsTM_xv|4rSyEJt!(YnrbTrU@l80Ofo8{bYx#Bf5j#VE8aJgrT3L-B|-8dPuqC}Qv&IzSjKLW>#o$YM6Iw@(gf zk3?LhBtjW4D@u1FMw37w;Kc<{CQn1O?KyNX)er?4BD`I1(-%;~&3nSg$Lq^7ld4P; zVo?=wm9!lbAy)EYv7DpWEo-Pr%cNKRn_f})S`6}wApYZA_>tb6({#E74ffMznKE znwV5Rv0a6Fx#l0)?!lfvTPc`Ev6EWO*aY%d$qrjPf*E*ZXH@;VCo1)KX9flv*PVR-&Ny9vgDOqPhSnw^ONm z8^TG^XS3jX-wKaL1KUbH8BWQlQa2RZrJdS0n2fSJL}JmDr%c?X()>jje`o|(fN9nD zE83gAWEu#xQ;J!-r<9u34R99zwY?Hoe#4~VrB3#YrrNi&fRsdv+-l%y<49GlMnz&X%elJC zl5$b_4SsAH#Qa3YuPHJ>hEJm?q^SX2@TZKE)5oh8pE}CP$p=J!8 zc6MhBhJGH|`s(P!we>6M3bM%J9LRsqOjw-moDl@_itPjz6<$5TBMGTF6*uEb2Q}b(Y~>Q z$Jz*eQ@1(nAW>l25<`I`MS`^Q)YVf08L!J)R#}7gWpZ?zz!^$4evpk4Gk{30Y5CP5 zB4)T!#tU(!%KDe=VVDQrfzdG^wJnf@rS8-sg*ef=X|pl&8Pf?xN3#?&)Y&$R-y3b8nX7{HV0^4C&=(8bP*-8gVgbz`s%3{ zT|^|%hOa|jgrhKx>ZOxNDfnbjo;|vdU7NRgq(q^FMX7hUWMr%4kZNjKmc9*^5_J5N zK2{|p3FXqGI_V&sDI3he+Ma3^3&Z3((E!jWNEx-uG@8;Tu{rEA0V6-vEJE+eNGY>O z9!_xOU1|u5D)^xKS=mdj2F`y`BV-KNzTwPrW|q($oGO6ISEZ~0#*xYf_Sli9pe1Bv z7=^2>FgQU~YSqDD5>=)l1cXg5DkMokt_8IKjfX80OzBodM$5KjWh_dkJ_NyTAoVVW zH9NEsAS-!Ls^(Qnq5+b^C9RPGp>Hy9k1t+Thae?n#?Sou{_Nv_960+w0%!L1b~W7G zB=XKhG5#60D^LU7RfB-fLL>)md8Y-t`t7>)|l0=lkpZ^HY2hllZWk9z`bnOl_)Zdp+B%+&|1aP2J}i-1I}(?H}jk zg2%lOkaD`+rU@N(>$~YNt`9!rKd!zU@5EDVW4le1#(%cAxgK{bbA754jp?g7z5c8D zxS8!rFPO#Qup3YC#S<3O=jp^q`aFBXf)f^6`tpR9{`158U+lYI+O$n}&)=qLn%1^Z z(y!7N607Y($GiOI)v!Nko)V7t?Yt8o?qS^Ch%ouxu)A4^V(qr;&9L83>(#aJ-U~d| zA9DW!=-mtlVP2h-*@36FOMbEu*^Tza!#wSLMo(9Id7NkR=lkjAX7h|o{g4Iv;Iq6y zrFY{AUOlW2(_=#R+PZic4_e0OqxVk<@#`;q`Fg$=26pw^gdg{}UyVNro%*{ATOjX$ zdH=uLY+QbQuzfft1nTprt2a-(^IzMWKvYW3aSw7DC0_p{C4*^W75Pu@S>Kit`$*Y=KPVQksniA4KlBfc!XZ-(trYulDy zeYN_0yMWM-dtVS6d!GAY_oHl-u7CbWLX8ji5Btma|GIh7zIK@X37VeRgKb>3N(P6- z+wH%YhFg33@$s|0cB-$8??p?k1vLM-GP@^(xV5ap6e*U??rr^lylWBY5w0`RYp2(`_hok*ci$ZYx!+bM5y%>hQ+3S6>)812X812EQ z{jX-hb%6kaFe$M^O(jyP3#hLlU$3U)SB`5#F;ipC0_p~|soxGFHubhrZbRgd#G)jQ zBfAOo47KKgW_2n*eM6ONN$R{-c$xJE$`6zhQ^o!k|7 zB_z%yo!w9cL>22cEB~hWwJVhhL#aQ(u8@+%LX$evFG;R@EtVxp9`p@GQubjWcwp04 zeMfplZ4=s-pd*6%Wu1U2(u@Nr8!LYtTuo`Q7s2=v}a>Kn7&N~#l1UE)EA6f}cDC?Y#u zBo_~Fp>hlsidxA=p)$A#(?M!FENdi3pi^TL>Dx-2(~KCrVk_|^+BS_-jsQr({*a^p zl0no4p+;-bhvZKs|0iU0oIdWAlurrnS%nrcnvyEVo|{S-C1R$rqZSQ`ttngA#1k(% znHoDOlu2y}k|-TIWT&=a>JimYN*^jKdL+vrMFT8g)Tn}2^{Ty6jmoanXT0w|zEZ`e zBw(%GMv_@}o*|P>-v&4V^@ozEv2DZ)Vv?zBHUhCA+vdp19DbCVcY!_qjky;U(QZYX zy9SY%-u6e-p=#Buk*OzE=BXff+`U0e=!=A(8h4`T31~>vdjSDZT?uyvE}yi~QK`_3 zNQVmSV((?8z$TPXr|m*r(fMZcA^GhF6|zN$95tePWDx!#Lt5M5VRK*?G?nP)yP(%slWk~yUa0@JrxG`6L z21l{@M2KnYuBs_@A;!m^fPbP+rh-XT_ozBpONTLol^33eD7aEpIa~wb*tUWT1l?3Z zb8+3GuS=vIQW#(25#pjTMJ9@nCb3c2M2IqR&r%&kxsZ%2O%EpGDsm}oJ{SWgMlzBz z5V@``IU*Aj+dU1J*|~@cNz(Nu)ODw>U_L(c908=i7;5v8VB%K+e)q4Fdso;yl|3Kauy2`ge`MCVDehp25a)NGLg6h z&HS5(K*phnez1p1yXgw^qJIV+Hr**8SA-SpyQ|4aZCu!MxY1Oe6T243bOfQ zm1@^O;#+0w$g}TMkSEw-5YV)uR;uWorj#vgnPAaE>lm_3T76XuYr?46;7vue;C1R$ zD4QkzFY*XvWUzBniG?oXoJ%+9)hj@qa+LNW<0MQ|-H2F^##ppc$blf8AT^nQyWXKhhmZ&6sz3?ATwUT-wzwu{ z_Sbi*qHIi+c8)+oY?ka0MQ@IRA^*h|qym!sbLBlD>1n&PL_vYjIR*wMDyWYbwR*(q z2e=23gY^=i$*(2E5>ecmk?*qhtiqE(d5e(ud#koXsH9rnDPd#vMA-J>fHZQD%n=Hi zfLgjqQz@$t9+IuA3=8VhVSy=STm)Uu+v%4xG*%h4od3m~q$W$Dq{zGx!6Zf>9rDmS zB{@|(<@Gf@$DkDZ%Jk2HpofTf5l-3H2R}+dNlb%RU zRnW`ltIkHT#vb{~qV|%GQ3rGq6i(~PhLUiI!b_nBRac7Xg~$=Bwk(;vFSgXuT1k$a z8mq-$EbpI+on|G-pp~wiN2(%ii0e_lwC-j zJoz6=ofsIzRaM*MYDsCA3a3p$vQl-7VyH_yvS4eAmnTpBr2xb{F^Im*)T*}>Dn06E zfn(9Uj<&?3*N*s}2zc!l7MC#hyJJn^%#vztQ^T31^F3tbQEA6p6xpEeN7pkQo_eS% z>N~YiwNps~&Oe!3V3VlbqLbR7NX{|H(IG%e1frZq5!kNH1Ooh7YaI^QFX^qjV4VG6 zKuT{Nj7Tx(jpwYh-Afi3^Oxd%CdWsfQd3YZwiO9kLrKS~7LNz>zquy^$ODUU8ap{4 zDQKe^vf8!AOrmgP(beu|mpc3SS3!FJ=L4lViR|V^c+dovw0~LXdK0qqusrpG=6gNg zzyIZC8ZQM)egA(9)83xxtw5{^8yL(`U#-4>0S$jSKaRWY{kT26z?U!V$xk@H^aeH% z%fl_u&1WR@4}$!yzMs~2<7OI+Z+# zY?N2~NmRJ|)wk1LnAqcZNUP}mzuE%7TYWbTZ=XlxVVJf;?*2vC~Vdh{=9j*f0%~np?OSbczx@3r`y_ePP_H6 zAN683yY_e+L3d9))cd>n-TvYX)B8?Ot-svvY|2*xcW*8h)@0|{eKqcG-~ZcA52fGx zzFHQfRJfj(+y`vN+hv@5DE#?%t8eE0_;5FE$JHwv)7{nTji`JtK6<<^K zKRgfY#p>H(w;4rIJB?bNclOD&v4{L(cobguo8!%s5ZhX_&$IBQkoj+qo7ETNaC5c* zUk~<-^D{E|?9Og$mN{P*#xm{ap#HNJpa(nJ^Vmi2L?t{eZ~hnE;QQG&_ecBnnXr1< z0xui)js3srX1+Le2jt-4v(?^L&CC9Ky&HahT76~4&@IR;c>N1ewaB#}E)U}%RycP$yS>!ISAc8-kj{3*}-S>?xo9OJsqBw z!GALy)GcvL@cdr`5pL_Yp~$9a%`tY# zQkvxxX>Rc=6BCWtP>49EEN%A^mw0FImnks%9cp!N(rZYlGh-@F(-9hEnGD(t@P10R zidiLp@mRIFnvdt=s>p()ZHS;FlIqv??nE=xDhrA|wxS*>kf=bRGeUJ635ftG*49JY zu>N9c>jdfx(b)i~Uf}V3Ndzt`1k4)(x$bD%i3@md+@_Qw(#R))a|7 zHS1XM35}hHC?M*GOuj{;=aAqyZexSxRh3mT#)6cMX^7}c^&tXlG-4qn^g(Q6hVF11 z9zA_Sut@=XpiC(_!4|o~OOaBx1Yfa3O5C`xtZHUP?X~?8n<=66notosrXX+BmC{7V z?N^z}Cy-lhMhxpYB9ABuDPmTnH!BKTun}aA8IQ#rikvmB2nSZ7H|tX`1-`li{*`oV^G~SER0_ifXQ$-jk&8h@#DpJIlMBN2 z9kN~`ah6~V9c6>1mtavE6}hTL%BfVE>C9UuNNcAgIVFUna0!5w=%U&ZD55j>lE@YM zAj(9fkaG@6>R@sLMsmk%@pEJ^qPwW;2_gGpFYZY%vMocEQG@x4tE;TP$j`{C&QR#7 z+$)Z6!cK~9xy@362)rJ__7+aj~rsm7xRW)f;?2qUD^ys3x{ zYVsnnHWR#?lbDhlk$Mn;j%eJG)6~qrcroD^Z3FZA1!c}lR2vL-xWi9EV$cAOxavFR z#+EJu2~wzCvn8l@YaRzVvT6VbAJ;XD#Py$nTs=mJb;iE!lio?eoSF$bJ981pD^}?i za~o0#t&suCrd&O=aQ{{=Du4m8h(;OHD92lHE+hj7B?h)TJiByc0ic8-67PuwVVi-u^?7G z&STkj0(DWti1%nwc-}|h%qV1WngNtlP(LCwFUaR&rR-i!npEz!3z2-comlb+>0ML_ zrd*YxLR*xiejz?}NUfQDvS6R)vKBPD1}1wc>U1bzpw`38KBh!RQ4|@D#1C1eEvVEk zTHQ!0cgg~{r1__gVaop4&1>9UB0%g=G)|#`d63X-D@ANnh_Lvh&Xk7S7ghH^31Ukp zrNWLGj}HLfDI(Ksu9m%BFb28o1Y-f_^jhpF`5<~%7kIn25Mzssb3(PE(2p=~8T>Bo zt1jkzq06k7b=Eq&Orjblu??havqtL6_NrB+ULuodQ1@!){Pb7Reqb-jEZ%KUGayx# zdg0-*z+%2-OKwTse?p)`t-Gwiq?{_zVp>?51%^dxj--yWx{=a6fP^~jYH~bkX(>@W z(BCQu`%RE16pj`8s9NB&UQ&hVvjmtxfIL zG*RYO*BxvI$!3&UJW;rqN1sczB)8fEKqTbpRa1ilOPrMzkwK};g`)JT7tb!Z;o@^B zGNk@xTB8ND(x_n_g$F8u2s2&*w`Y}u`(S@`CCgc=SD%O7y!KwHds;!Fq=t1Z4TXwS zRNFc-qirR27G#c<6i3%GL4(5`vgQK6p4gg^y5539Cs(55K@H#NqxVh7gT{1T+FyvN z!7NIZQ_K1aGmn7~0aaA%6=8}i={Vw`<&Wz-tKi&5R8l+A$}{J<>ZqSnv>aI!lpG>E z9*{tyaPoXit%QOJlhuG42v?wJt|2slo$-T=qWW6hD=bDX zj9z(X>R?c}U#5Lnc*vx*C|0GF`~|mBa;TPVXmTmL@BHU8W7$a7Duoh*)qT z*7k#I5R#E>l}5(^Fmy=Q=Uyeej2&9cT4o1qY|cLY$AQBMqL}G*_XrqVX&VZ6FKa-yB+T5)o)K9&#N2JwC;YM ze*Sr|UkF9|aJ=43`@3|H6Tot12Sahf^$H=lUyXYa;ihleAHJFIhV6Fs)%djCOxvr~ z50kKU_Vw+>auvayug5Xr`9!?Bdq$!z=)&dJ!s$)ho1+Ma69)5czW49$m;cpg@8;d+ z<}+P#f4EE_)_1$fQKUljJAuv z#p?CE+m7OIy;!{&R)3tghhf^{stb0gm?zGw+Fr_%ybkuZNrWzuZsuw`Z*PGmUQOJ|fOt z-goz8|MGTx;S=7CkMrhH*!G?7kmh}O`~H_SL;GPfZ}-0!AYAM2M$bDeq}G3(H$w2* z3w|>W+ski;^}ihLEx(_|{;CoBZn774i1oLFz`Dnu{F=+huk1Vb;!(B-U(l9cB>ZR2 z{F|Qj-7*%w(){U;*MAoWvlh&0*`*!#*#Z(4aDG{F=XluDjJz6-H{&L~#1Dt}zdYFf z(9>=gz0oqg8TaembiLeiKd&b(Yo|e8wzw@k2j|;N-rL^!z+&k) zqfN!rN`0Gl|37b^ZN~?l6ZV-f<;&iEp3!fPn#I40bo{~}Q49I7K>Sl<#EG%_WMtC4 z%^2I@jX$L)706vvCjk2-vT9quwlIBN-KB5ugfCw%{ihQP`mI6Oc2w>0W!Ml@MnNM%kt~)dG8kLR z_d9bktEZ<@gMXeomzn~4JDc=;-(Rptz_~PQN=})v#~AI zgBzgSd6kKLr6b1dSw%cay(nwx&~?rBtffEYI|@Rm3!ZMzMcE;V!1mKmdshN}(UznLIFcf%)G)jBD;r8RokIa>qLACzNDSTJtWD4ILk75 zOg~+FVX5I^?U^!~o9ar^V>atlz0~ux$XX+InyfwDKetmU!>*>>o0u}l_wA~_L@I;I zY!GwGHsUiQSYFwf>-`D1gpEC&W>se|2~-2M-MYwX_Js?}MWiEb1d0L2!iy{!*J9y9 z?pmeD*6G}z5R3SUl`nUtfeTo>waXtukx9Jf+}cCvmK>{04aMrq+^e#md+Y1)UwWAdufw2xLVbxvE5cmM`~xt_lv(?PX||6xHZB z$aqvzDC9>4f`Y~7l@$a1v1~*2u-pYo>N3`8H3^!lh5lzLHfHE2koqsS0*MKfXRCmBN?aN5&rB$ouCh|>75!Lm)Sfd(FNz0+B)GnUCh)y3$ z&zbv@vAm~&mkd*hC?IL;WEd|u=&fU~$#d(aBTa6u9dVGMD;&xOZyp@RMw_L#TW8Ko z;ot_l3$|2<36+jl0Z~E;&?Iq&iD;4rOgw5u#!TjChhxGh-w=f}fM-^7NIvswq)Ma0!~eZjXg7_ro}8k;E&e!Ffo8u!roD%rL2n9C9JXD zj(YHS&8_VtA+zjEGm`@#e#r>g15@NDwp|jVDFrg_IL?W6!N8X&JqU=?OXXFud03+2 zgriUpK+cza8>J=81JHgX9D^L&0{saW$sst9!$66Ny>yT>5m8F5fv|c_8Cr~!Z4i4V z=PcFeNNqEXlI;jMYRewzXloBW^Q3MlgvbMwh_bB_T4kHIDSK>qv!%s~&>r@J*VRiA zjhO1}&1zbqjmH2m<7HD~!-{mlsBOAaCL5x}DA^b6pi-Z6m*KViFTE7TNG(Ndj)*6< ztz088wu}nZATv>$?bev`hS8iJXo=4WLoXrD(v&BmHPl9hgKA* zSeoiW?gpe0iFgQ!aXF`7SrVYC7C@L7YArZqWVcadHheK0!r}?yzNNlvvr-sgbV99> z1dd%E6{TWdIB+fZzXXb;a-?~YWri+YPIBpB!ZfM6R#4!qh9XD>#V*&0O~IQoT@+Xa zJDQM##QXh_ebt#NkU!WjN0Eqm=@IpE6jU_GNTK!+IJ2Z{SV489bSbvnxG7Jg*qDe^ z%@?kbgWpzkD8PnU)Fh6V{PR}i#V}Ull2Vf|#?uyRL8{gXY%9d*ihBZ!y|p0InE_9S zX%#DC39{v@{za_LNKk-$kES%|jyMSHXtvD*=_sk*B6EQKB$G^VH^R=~f@*?t7Yd~B z=_Lq3OGu3f4Z<3>F|_eE8kAHM{(qy zP^YiwjX=W}tIxMLLI-|1-PnI5lHb{`>71U!`uXqQ4x8O@{7E=Ofelx$hx=nf+S)(f zis<#91)BI`_81~&@gY|PtpGbp!793q|H(%RZCWQWe+-!tF zzka5*-VE=4pZ;{Q8Xs+OkC)4Tt%q#_mOo#@*MNw^?*_a0^f*3`xGkmmou1kD&IzKw z`)qGd^Js7QAlBVwcE1%*t^Q@1y^Xd;i01}lSnbz$^X!=Qhk5h#a5rz0r8r0R4~H`# z@73z_`_riXWj7xmg+q7X{MI(2J}tyg`AqFXEQ=$$XJjAH zfMwtSe`1TdJ|PAlGc{HZhHI)NIq8`Z=R{Oe zjb$ z3ILdiCZ%L%9Wu;;hrDJL0@YF4-j%_@VQB3zfCXx&bIe?Wl%kM!8x5}G_K+0H4x=g) z5=hy(zRHqHscv8=3%N4hMg#|r9R=71q6VCt8N%HOv)`N^YMlWHfE%TVy{Ud)nW2sa zQV!d3TSD7$>?A)9t3iF;_KviEng-mWzRYCEQo>A9vGj*fcc`S=ma(Lor}8*m3+&{b zE%oB#&o7gM)1pgM*W&dC#De>34JmX5F)R>OtmqIMR8JR|JL$YNjdTTQNjF_Y46w*o z3fzE7E>do(6Z5V@~ty4S6V9a{)lJ4lZgUn zLtUIDmELt?e1grnww)UmX>(~mXeb(n+H#Ee$a|4@mU8r(Tmfr!zul0XK92?8W!?&^|3 z&{+;`j*^f?2%nIU)MO(uE@C`VvW1z2D#>jS=TBv4Sp(@ywLk|49F{_f~YhxA#2_u6feQ6SF^+{!sO?f4E zq^&A!EE890pyO2F*cNqiV$lov0k%N1n28{z5pxQ4?9Z0E1IEp*E4b6uI)%rS465zP zT5N%@g=UB04AP~Eq(mI*T<#=ge$7+s&_qiCfiS?){KO34kj|!FkPs-Ct7D4fw|fgg zs8P++V+gxdy8&sB!!D2w9FEe$rabDKq69X zd?meODMoM#iq%;{P{i2vq&p<|4@z;gWuHkKC2MXgkwRbPYO5?uo(e-{PK28hnmT}( z+1)}AWSHs9%tTc2DTO0PeTPz|0PrJ(qoL|y+4O>+llhgTXI>AK!a;>DwVc|PsA@vR zr6=Ppkf=TO@h-CTIYy`J=of;Ln>Ej>JBy3 z)Bu?g0_Y_wtr7BByU<|AL?ezIjt0$;k%EHlWLu&rx?o{tOHI5I)9a{jFDO~o5s+Ox z;B6+EBZ@e zlpdvO?{toYJgn}oY~vuCx@yTaVk6DVYp|`MQL_`UlegjrL*X$q?shd4L$IBOonxIk zn~8KMi0})$02`NxpoJ=lI_mJ3GzsxN$c_c}PgIcO?NIi14+={}nH!L`xD;b<0o%7! zcSu3FeT^So;(!sGOapSuR_98{6_|3d{k@6>ic7il1P*GnY!33^M7R zsM5%CH$iyWAxxNc=Xk2s_4%6O7=go1md0Mz%zq^|@ z)5C)erVW^=U-dXIgZ+9w+9Qi`Si|*t-r98^hr>KA?6CH*qUb!)kosdfCN&WptM%w= z^-3h6ZSlfXd~;&^9kypsib{j)feMt zx*ijNHxZY9JM8!4R#dWs<2D@$yT4j}InKLV(bG;pkB86p0yy7~TX8zy+j6o&DV=-v zP6J{iv;0}y&3k8bUdCnq1evNH;O%a-A>ST0PnR|zHd?lj(%3u$cNg|$@wpD$d3z2n z{rcYdrERTE3)nUd*t@%VwRYa!o7HxFw-7gLE*=tudbX#w1*m1_H@gs5?EUq)OVg2N z$nn6Z6|6_I5wt~{Aj%Je?Wwii!v=dJ{$3l{>*IcJ)BK0s^rMZu?K!`YgW}g(l#k=) zYW2fHx@=dJJ-ED{ciWd|zj(&l|8e*JmmBf^u5EVbhiRLD_CK%QO#7d0u1-Y9e}4OR z++GjsyNlJIruBN>y#K##wcHNdX*;DI@U`vZ)erWzkJEPlqdnj`a`(&*IN!Cs$le}n zdTXz(bv)ZTp01DD&;mRk?h+q$!r`WYUGU?RY}4C~P3L-iL2;**{c6};+EA)_Ac4vE zho{YW#)JJb?rv;^YCfFE%rpD<0dcQ(V|oUQf{Oj`1*g`(oo!GjWfm+vUHeYR?q3JV zZDO#{7S$6};3okp;re?_EQ+Wh7Z@!P4ON>}d)=#6e>RCjML8vqSTqNbI39!dFk6+bB`_f%Dz+f$-+hR!6EY{uC$(os!NedKwiiq01FvO2d{1F4192z z(mxhsVW+)Hls(n*ZHx9ft5$oZKFyX1NJ<^|EkYI~C>$Xu?pj-vD1;>5g_>|iN<;$m zF**nq5mt)IGIg`GfgALWB>{!HV6hjEN`8+ZJ0D<}4%$(Gu*;Go2x}&^_5O^IS zkg7(^<}kLBwIS75v!i8J5-7aS)Qh&TL8=0a#mW*QGFO1OpTBItu)XgyT9#yB#e0rrHkIsPhqfwxQXS6>98(H}&XvlL#Y(NZ z=+)FnCZrr8GrvgFo81cACPa>)!Vw9LiU}-~2nAD06%O3Sj=S#TRh24f>Nclba!_vv zpp3n*g?cz_g%b0kC|sJJlI2Ysaw5g0d|XjxOL($~XzA$9U!>xy{+Nnn6;;K{}I?|I!;v3hG?OE2&vegi%dTfqwUm+a5c19?ojINc} z1OG!Po=lF=O`l-PLW&cp>tbk()pJ(3D#qY4y4j?^}U)Rh)g^#B^P zGosIte2a5ZGnBikvbvPNxtvad8Vmf^l51jHy(W}=olu%E>cT5fB}>RVWC0VWS5_m? z!$~s1c$y=$L1wxLU56qRb+))O|gZn71r8qA9h)B-d$f6sY#%(Z`JTa!JHm8Am?D+h*FBsIIX$}s4k}y zpF6s$#^#0ilN$4+sU4`PEw+eM?r(%5!R}0H98fLX7Np2($&c>6zZtnELDt-Xnm`F@to&Dbe)}8bMViv8ZHBpO^ z7rgb@sN0j8L|k719WD_>WdytaTy5w90Uex%3h=0M+j0VOH(#aDa_DtP{hGXT3v3z+ z3C!K+bm-y7q za9BX*Op3oAR!I|cFjtCayROXoh>HY4w#`coXLic81rj}V0wL3>?1P(e)Fx|gHi#N} zY>fo!wG-4=dO_GcCstoN&7^OZiGHVrVoRF_#tv+XJjC%8VCyJlsXlmUKJJCiAlPhd zyl)5+1Er>zC%y~hya{J3hBL6)l8ga2+oFZCMshjjV_I$y5x*8xqU}@V5|BF3B1@7p z))~`h-9J9MAWBs$(uy+ItnJt0ig;!{%WY9~S@Ep(Bgvl_YU zLX;GX3N@SmA-Nr$+CRzb@x&;!dvtY?buH^-b?9N+x!m16acHM^8FQSzJ1{O#1&6EH zCCEG}USKe~CZuAOlt;RZ$;&nGBvXlz{U#I=t?1Vknv5D zDmj9)aME*&3h5~6WesS{_3}g^n_D&RA_IuBRc9Y@>YGyoDa}p||JVpu^o=u=ATgzd z^%+(uO?^;$B!4VB6p`eiw)3Y#T6YVKB*RGD(`Ifc7uSfw8#3GqvND6_wjtTI)T;5Dp7#h(#dYSb36t572pe2)VfA5~uY_DNsmck4t=*rBeG8jx~)G zTBwCa-SyNhBH(C+)yP&FJE$Ti(n-?NvOU#MuYMy^Dl%rJGK6?0bjtWKBad|2=%}h3 zR!`oRIA)N$NtT?F!aCd#1}o(d@;6%Bo|#&ZpWbbOUH=L@I^xQ)qo^%2(iV>g8cDz+ zi?{jY)K(*pjfx zwFF}zZD9MYilp=>wlYd`s~TBn3d~=-swnL8w%S={5VlFDG zlt?UXCw{D%(?SgxJ%BfZsBW^K!8rIR&BThr7i$QC3VO-&ejqD)dE zEfb9{=JqU3vm=8@9MqpsqI6#;AqA>~tH-|2+BkiPl|zK;wS}VMiGs~2maz&pql)lM zZEV4IgDqu^EK5g1I_AnVili&O`k4PhCsSxCE!3!EuZm}%|KkAN{~4ia;(Qg}&Oc1# ztk?5#w|oDK-G27}9@fGP4%_K|Az5{-mkOo8KkeAe_QS5qh0gvxVw3}ywyK{_l(XQ z-oF23VS^P3?-|$Z)fuIyP>6?Nqks3|w0}u3&411f4wa#y`F3wZ*4h0rFVUS{W`Anahf-i!+C!=y#M9lZrF~iFL%SkgDu7B3~Blwj??zw zkAHJqeKj9euWcR#-A+ue&pXhZhh+!a>35o(ukY=CqJb7d-f5`yU9llM(XV5szZ`e> z_Ai$=u)f6}{s@L1{d?=?5y&&S&_{r6v|@%Yo_>v=QZUr*z* zn4WRhHiNU+x8IDbzlbT>Ven7(6lQB~jODRx;hh5p(Swh}HOvLWODIbrQ3K0xd+AwHcpb%CV_9i&3HYo3hQ8 zDi8u*kR077RB!8A#9$=TB^6U5&8k(lq<}^#M-;0TCbB?CM6Ygo*n*OOZ=jAZ5s{j3 zpLKL1N#x115T1x_?OCxmUIsDi|8xUh2~|fsvu!glkz2k>UE{C{g)GP@WYf7~OB9q< zi&Cs>*zw$yf|o;6gMD8Zh7bP`uf&_wvY#vJ690w@oH0 z>(3ja*H>VxxsVkewoQXt!-Cv5S-Yn1sa~srOj)lqI|;qnZtF8hs4^#Z98h^%3_Te@ z*EdeXMptn0f*76~WeV|(GK+|iiblCCSfk=uirDcQI@vyC=WCU~rPM`G_KR>E!7Orv z?^yMX7jbA*s*D)K4Qbcp0O*wZGFy{LNv5BKyfo^^YEuc%{PCh5&5tBHWN}_ z(zJa`Lm$PQNAiA6;i_MhbuGOypANPbiJ?6)gQX8O%mW;&6T?x3ETtlA$(XO*ZIHSk zR9BqC3s(d*M~ zno2a{h(3=h_L&EX2p|455#R4Jm(}R26ixmZ^ zL{&Bq8Czc3YN7jtns9IDNI`S!;6v2&6rv$^+=tTDfz-T3wb~ODSv+Am zsM+r#Ka+@cylb$6)g|)kB12GJCMO?mPzWo;N2}c$yM+Yv#zu)~D}oqim5U(+XVW3o zaCr2{o&AgS^yYQDoCE@5Pa;?MMwePp}XV#cGEJaE=-;$zio zSV%~z6L~YiS3^xQJ8GJ(5RM};YE*<$Z%h+OEvc_KEhi19COZz6Gbe**`AnHdob}DR zM^QHw_3Q?5h?kK6vV~K(5HQ#F<{{H>@n;hG90~V!eP3gbs8ADT$0)(*2#-myC+?CFFlNy1CQJBmfMraYN*SN43&hyVL1+#_%o9E$wSOQNhufKD-MU_#6m}M zDWxYDGpgaVwag1G5a{T_(3Axg8$otpCyOf>6@&}DO;yImnXHtZS`=#wyl2M^41Ogd zS!y%yq^d+tWnUsaa%-EKN(3^3T~=K4U2SW+pt6a*LtSMA)P*We!8T<$R4c6^jaOwsG)ednRKaGqQI<#PbhGa;z35hl73=m4OVw7Y7r*{0rkw6 z>98lR)iNjR-MP;~?W3X~M=b{NBlW}z5oA<*+W0bs6(O1B6Sow^H|oYG%(=x>HD#cp zz|zQ_6s2V2o4Do<0BlyDg;@)|-Rv?nzu*AkTj&#P{_ zHfsY&nJ($8K$N0@=M%kb^3yY-5MPdcRrdur0KsX+u@Rtc`aaVtOg1b8)L^nV@2MqG z?nh1XQ?)sY9sIz84SD3ygk`4+C4w7CZ%~b;r6KGD+edkvwxZ&oax?Y|0BEbJo2BoDXmpBnXa8X6oF`)UlT!yQRdO#63G= z+is&M;vkTYt(kr(OH@L`Sh#xj`9BW3{r?1e_N{PmCy1&8H`kkaZ+E|b^3TR$w|QC# zP$LxS{+a#rFzgTGezm`w-a0OIemCrHR=auqqu*vXZ#E0G@P`H9nLwG_(e8IYZ0F

tFiQ_|8UPx3#M~3h;;HcCT@JGdRQO=-9*WY&H^N^3{C&m*aR4!17P??&gKL z^eGL~TYI_baoE_54L|yGug?*pAEFx*9`+kO*&mnRJ);wS98XtUeXsu-w{|xhzC_jg z=Y_8IBW~VvK(P3E`=-5GeKD>_XL7akzxJ;=4zv&)cGzwwdxib5dxqCuPdodA{l_0} z1V8<58s7f5|MI`Sn{I>+-Ho?4&w@$s7hLIQ`xkQK_1&;NjK}@xOXb}tkgyQPdgX=l z_W4J*&PKZz9@bXz^Cj#Bb{`^!?;In0J>9PEriYcy%4)wJ(>P4adi)r5e)GaGE$aP! z8Z$BDIuols#;~&Q-Hxlz--^#U!LeW4?XIuK!MSJek6W#+yK(a{J|mkKkg;IA7pvRJ z@xBl9W;*zmaOm`G%h})Ja7_HVj&!}Bwzf$R(|CHd*EW;)gZ}L=zE&S6EtH$1P3ewph`@ETt_VJtJ{q+9dv_p3D-^cX{AfBMl_p^p{^)Rgu#|7a% ztRB)%GV?L{P}ttlFTUKR0kFMg6Sa!n%c8LpuQoHW;II90Z1&c({d#x!cV`%Vn{T_# z!LGDl;LA3%vu#nm>GBG`F$FaKc*d^B+i9z9_yJ7$>iK>?TVgc!tJ(gT?`=|UhNsm_ zZ$WyA@5i6?UxW6BJmYO1e^m%%xUD%Nuo^92ErrLO{jEaCS0EE(Xb zsb2mcXU1$?sOn~JbV-UxqAx~0vk+YjksSY0y-U=dFR~SAiZeIX_9Eg|>^&MpMFnYA z5Mxki&0{4E9#!^{)z5;x?h`yH!5)kjW6>%PpFstRgj?)Ulvk9k6`{NvbEK)7EJ!G- zt2BMfd810*{-ii=PIg~qbaB3Qz4wmt4Yf2(h;FQOunPfV#bnu__MN~R0KaLa*+Dw! zHd!?im_`LrCE;G>Y2zj>>n@|nO0^l=)FgkZqy_OpNC8PwW2lXmwV`Uk3${Zk_@I7P zG6E8+#rA~8+uG?ZlqdZ-T8LLsx$Z9KJN6jL)-}C^Xv9$+(i6u$z}H%lAR;qtsos68 zgIbJDT)oo&3Pgt|re#<<7Qi$;mbNTJjOvkx_mBilK(=yE3jlB=RFUFPX?yoryljFQ zQq;n8LiUe~bZ$-52taUnVWS(oSlf41&J5Oz#jR}Htj3;dA390sM1~7oQVqBh-94(x3&DjccvXN~VneE=8-@VWroqmex)RR;oQGoT=ImS) z*?mSgSh!HBnNonX39gjkR(!M?Q3{aZP<0g2!s=1zu@SwHh7$aukB;riC*9mXOS} zn=FUq%Ry-`S|)w8U4neXwn~wusI*go0J?(h)hydyDJKg_Sv7Ce^oD{MMLw)L<-Wl$ z4Q7~XxRgp~QnQDEK)_`v&oyEZr|L}|WzDHsrG>DzVh0OqIerS1uup8~lAQunnlqYr5@jl|!H>qCOe zc;Z?hPy?2tQz;Jy*TR`PPu%-3spk;pH8wUL1#}3A{G+Z3gJ6c*h6GQL!ujlxOhp`R z)cex)4bmXhqDe;$g$T8b#ThOmqMKR)ASm3BL)GRoC;q2Kf{J7S%?qlrp{;k>klgrs z(@}{u&9)5{QlZ%r93KTQY5@xfkZ+DtOF)xW1X0AWW1C80>_`_uE&Vi`sS6s06W-#0QG3D<@e%3!cbmrsx;Aacc*Gk|XI#`?9SNf2KdHQbe|_+rDa1u%Us%IRyxMQf-d)|Mj@$BK`iATyE=vgd^WN-azU z)dl5Lm4Q-Dp^G)L4w-kVtSzmsUQD|%^MyL9VWEnX4LwJB{|L6Tx$5%dXpus`NQ*1j z$~IRQy=gm_Gk*Qcb5myrDHSi&!o>`3Hc})ss5CV_RIe^LKMa!Q~^S2SF*xV#{$unYOMf@~V*^{KD0=A0JqyhQ;}d$qD+*84GZ#ciV99*gh5Dq%J!cUi-d*o`qqbftf!2iJ z6yHMov1*@2(ZbaJ6Nhm}*@s%qqcd}0ZmY)mmuTNIr%y5xA+OlT=jCc82~#)!b%aM1x)?fuu!FXQX<6O zHcHjgDhjn|exdExEVd|O^Go$b#?U2TNAruAaoSB{Ji1VqAeODl@f=Y!2 z?c9Q)jnQmYiG4A#7Ktzrhl^C_3}ns`}cZtAS;;^wZ!g$@yKktMRAd-j>YO>JP`={4g$f)6bp6RVa5eqMST`$3|gT^MgKRxfZ`U#yAz)N!H0 zw(dT<>fLa(D=gQ(u&>QGtNYPOdF^NRi1t_y_SiS+DR+mr$4&ZluSvYQT7ACS9Dfqm z?#rbeS^XE=FT3ry`h0u4876^+ZA`WcV!7>_6P|O}tq0++pJ{*f?aS}yCtGZ`hCbgO zY;le@Q^WCb+H9tAxBtDp${EA)Y-8y!`1a@+*fauXZ<>I?FCPd4o?xVZegDhb`M5hh z%Lg~V5pX&2AU|)HhjBN}H|cuI$Yysty-DcLr{!bLTVZfdpzaf??fP!q>BHBWkSqu}|wZed&9V@m?f^u9$CaqC|VGZ70%R|Klsy3tYf_Xo|#qIqrf81)hp1%(kj?tC1+ z9e$ebKl2Cq{;1b~@^^kS-po(q_B169MBhFS>gzk(WBbeZ|GIhF+W_oew(egvk2jxv zZst$C;OW=@;W$hh(aqq0-|AVG{q<|WzDg|Q>~1NRs#KYwfK{p**u)e>z6QQ+ZzpwD znRdS~v_&Q;rgV!URqc6s)HnL-F7t zC56Q-Y9XcUxcJQw{$nn6m9;l}>ZFf|o0D1#@rbG8yTDYMa^qblN>X?Nvt-=Lb`2Mu&G53yvJJXu zA8?UGrPqj$sP~*I8pa@O0VdeQi%3`+(ZJ)yBr+W{j?Q8!3R6?M#N&}}zUo!OCR%<8 zS(6EfwsmNFRZS#{IDsZIGV7EtQzf2Gk=~3l627&evil$>xcFj_PSBoK;$<{gvByed zW2UU^D5)%QV3F*?=51PRv5DgT&XuyNE87ntB4$MbjWt%o#njWm2DU-02{746Lkcw- z#Yov^EELW>*)r_E6@rwGP_yAccF@^gZwUV3ZDTZot>pdCSl5-@Hh3f(cJBA@%{C%w zg4y1t%{UI-3{`4BA*`FY6sb}z-T-X+sVY%)Qqr#%Gy~g&1V=(j^u~+;ktBOh7l?Y* zMZ`rd9q^E`rD#mLt}juDyRoef2aHnI89lRqZr(u$2TvuJ1!|?SX}VnM%WaHE=xNJV z(o3h}l%u&2PS%LB6#2!)r4Tc$SK;susVX8)UJfJR#>{buAyeE=2GdS^(pGSe0xCI6 z*c4L9VydOmqk2WK@>ttgkxD}&yeWe6)Qm`LM_7BL;g#G1^ikBYD_SBzknX#^AV7VU zKStDvHb)G|TOyW9#2T3cWRp&vdh&;{ep?}{Zd*#|mHY+~4PzJ}cAuuCf_mtMup@v& zUpa|3443$&z^ym)+R|cBg{?&CYO}dr$7((zXzwYGEd`&?+&q7-R;WsRbyRFmCOd}{|jVyetK5x-D$a$)k;MN~yDuG+qlKvIpt9n~0B9HI)j zxp=vpP>K|Ur@)pqi)_GQ?1P}eC~we<7Kdneg)tyG&_)PyvNq24Mg0-cCMrXLUMBvc zhO$sWJjXS0`&jtbwy>#NEi7to7W$ye)V$pcMK!>ZJu6GWmC4l*u@RibwOX(s+fs@z zAB)67xmMWkE+W^4@F-pEK5-Z-n@5s2au+Q?fywP8S_Dq7*}n#vOryHDBS03UAk{$5 zOkXPiN^3zb5Ro3HD{}wn2A`XG>p-ybl5ITvH%mxy&_CZfYvAyl-HY0pXXSa@f zvM9lPAod*kF4I2og<`%@0LZ-9Q$qxj7pZcGn$dOMhmI((P~{yN$#EXm#&NUwB+G zbrVrfCmbfV67*($8QI;mBH)L+S7J6Nl$9O2T}D^q2c;UBDIeTN<%6kJx>Z#qH<;$c zcbLAYgzlpxwd^0_3ZvK+iCi|-aS%4G&3XruYoW9h{z|Su-yvvU_S%kD zf}-nh#1CMev=WTSjay?!Na#F%!bS&gXTZ}^4irTfGT7MtCZub`?7cl0o(pxWID+c# zLHXSbR-at4E@1IM4FxElQH6}mw(!<0NjZ^DtDt~z__S*9rc#GaD=aaIR4L0D=t^z% ziJ-MBFuhr;-4cb{Z2Uw601i@Y`%IBErApO>gcPEc(s|evt%@7u^K89UnS`pl8-hSq z;4!O*y2lPa5t^47b$~C$&OA!kaAiv3(7DjZvS2M93;tuKpOZoo!96ID5Zw~MImMOM zWJNsRRTc+kReYa!^I5Uq()MkLSy$&}b)`fNR5Sb~?d-8LfYK5{@D$+QZOq{+AeLyP zKuX?IDN-_YI?BEhm=xL^WIAgoC##PmQckn!y+)9qmUUSYF}<0Sy3b_ImLRGsvgT($ z6>aFV3eOTOspJaFt_e+CFQh38htO^2Z#?(A#*g;|Ll%lNI3ejB^!m&fU5d_j8-;w0S))l0k1>FW?)7T* z^;Yb&-Ngy57u%O7^=n<~TKww^L6jXKd%`T9z=i(G`|+95_j*2j_&V?I>?W)IxO)`4 z@vu+a#V1tp3EO%C2fz93^ZV=FY@a?)%Xd!UD*)tQ=l$&qs!#yr^NVli?d4%N+)T^t z-jCbk{(~#;56jD>pRMQnYmp(J9_9rQygb&2NXuv1Twh8Hz|_X??FY2N-%aDrhUv@U zb~{WvyVvKN!CvHxKU$Y|?cY`GfGWvM!qv(4*ndOVxcX9C#v93-nJT^Om)LCwz`*(Y9T zN4V;byYT~(XdAf?iF((^ja|V0V>7*7IF%Dr_PgD@7W4Ch6MZwjJ*-|0+aEtf{n~S$ z5WioiFUI2hd2_1j=L?z1*gpu-Ex7bE+;=mqzMl{C_GaPt9amqx|9|dJ4`pNW?Yuoa z47;sna`nR2Yv2FlMn)n1YQOqoJpQ!$cDS7n4xoKO@%~`*d{})s?(XehY(zJrDW2?~ zlNuL%$?X@2@O1w$?+(I4-;d+jyg8QoVQ|J~eRntMU;Ne&chhFx&kwc$_JV$2PqyG^ zvGWeA!_$M4E}yRf+sn3x)E6*{6x<1=ht+m8Gjly3cXxuDpW(6{VV@A)w*GI%wb+bx z+ud{`=N@Ez9hAQoQu#1$7V|5z;@fd6UgqIo`$x|>{z^_?6|$+PJ4zjyEs7X;UIkYgADW(RAYef_jO#7%MErNmaYf>VxjnOV^p2%G9=) zA*2e24=EFmb!m$N$Vm=V1hjWhyia}XO_yWWmsF+Xa9|8e5eP7%w6*n1Lt;*K2cb;c z!&A8%T6u3^{)5iq6#{LDH8SfY^i+ebkcbGzN=g*ac50tV_YDbO`YMyfyezMJG0655 z--+|j!eO{cuY4$AK*aK30RSaj>kQlH@k*I7SoxL>!bV|3guO+Ju#G}xoea{RVlulD zRUTx>x4uMxoziFl zwU@-+r3M7b^mJ7&!sC`-aAG>v#fE}(MWtBKVUrYs+Bb|Pu2jhZRD(`#3&>%u1Qyg* z8M`aVl1l0?6Z%d?wMJ}-;0QRS19Z?h+8qmrb`~P8pe=#a($A1q6i%2gtD26Ly?U<= zz?j57RHURN;>?UxCYksu_8ET57k$T|5kaC$YJInZ${rY|UCqP+1&bRMgt+sg$ky+Y zm11=?nUlUJ75Bx~&A`%I}b7Q zV}sZcDpBRkmbfP60oi2H6x!rg8LNYAV;35P!IxsBMzEFmRk8e9N-P;mcToy=1RD#| z*?_}Gl*MJ1o7XSO%Fb6ftyC?@D$oA7!i_=Up>-JsFhA;#hf`?)66^N($sb?p8|e;tf~x7mMSQs z5&~PTSrQF75w!+^i(pF;?@=AZTL{<$HT89r-B+Ho$SMeFt*g8Mr)Q@ou8dCH-!Hay z6cZ?=sihl(X16V%-N%Mcz%<%7P9TD1lByNZn2wM-#EdFJ&!tW$5=dygmkK2c{IT#R zg*qfrvC>X3X7F&?4}>SQIo2TA-b!s`Xsyi=f@V^=PFOiBRU<(HC-t$S_O~tHDr2iJ z5_2FSM}>6lNVhQ6FED)}kC^xx+t6e3BIxxp5^G;{t?1||P>?G{UX}&QP3E<@P1X`F z$YU#dw_98RR8I!|MNOL}=?7P??gS&>Sg=6epUa^G*RI~Z!NRibk-&U1D*u7^S4cF5 zpfZXR1SO!pzGtgF+3S{Sv39b#Y7U%8^0DX-f=FgG?`aGz{v;4b6?L=&<3Xne z`^+wdpcDd~Y!+&?4VaA10}67`MLR&4G&^nuivV9OScr)|`9jiSV~ZL5uFN`dme^Nu z2>5G_RH!oVvb&0nHzr|i-W-so#dzK3N&X`3^tNpgJ<~b)JqF)>B)&T}$`wd6CTfL9 zFmZWP7R^%rrnPtqX;KBStB5wyW=pJPu!_V9K#@fgUAr4hrj`*VF*~`kl)Vy=x}s|} zqWPfavql?^I4;#v8Q?&SqWD5~16!@1;^I?dD*eRlU)iBZwe-PZFM5WEqA>fqMHaAx z#kr4E8#aM4XYFo;ZL#r1;`b25>`&*UOd!x6_u^=PE~KXKb5K#5K_~De%jguw@C)+` z&CjBRrEW4@*K&)oZA+05;yZ-P%^D6<#N0H3wm@{K{wRvL8Yy%k;6_a($l@FPk9@F~ zneub8*=iD})7Z|dimYHGMXfAiE>e-BCYlC|m0QWLk<(Zc?Mzn*&GzxZ-du@4#=-{G$ zw`Lu&C6tS|jm6onTA~?*Htk)hj~p}! zJV6mu;v$ipq`DFY7xEZNJ5!b~sMCw`t`Y~J-fat}W7eacoT|#JBe_$oqZPnsl^D^p zSZJ}OCHBVJ2_D;){K=-Y%?_mo3%+epAx5@*{DE)y;mTJF>Nc_ zZoqrm28}I>6N!>O>kKJUKTd;)Jc04CI2+WFm>r`jq?Rld@cM}Bw!lfLy}g|HK5QgQ zYT+?wp6r9Yg{pG!xOON;a`nVI7OR)y5e(bt)9O|riVAE`gKQP!3e zgUvt$vO~3P-I(>!xDgbMF|9eNAkcb9)udV|;QpL57P9==hTLJRP0S^|bF+L?|v0&4A=*3IH=2?Fr zGPWY;p-M|$*bZ2DL4f=Sn?d|Ywz?rfDTTHca2KUS2cS7Z_^Rf=kHz=@58$-#N0FTR zVdz9&49AIdO#E939RpB&5B;dfp%G!}WasFdn7^Qr4|*rnhhH+V;H5 ztKJRQQz9jOMp53_+b;OMoq&eB$;M_k-A-FaE57q_I5>an8G-0e2i<OVc&GbXs~8(USQc#FTB9);6h zpvbT9Yz7Y6F>i+TZk~R+S|(+gz12UDl-yC1&;PtTEX+$=* zg;#Ip(c@-1xd^C%zYhn15R+k)I4o(X(EwJ0{D`T(w0 z-`cp%k9y$+c)d>))cWn+*(9ELzE6{>RtOP6@7>1OwEWZR&1Zjh!tL`3ODzKDk7z@9 zA`8DCeiU-p{9;d@WR_v>Bckf#&!Jl`UtzU=Z)fty5 zD=coX_yy>|?8HsRIoDF@Yl!|_vglmUzmg1P?L-FkFthIcbto!WO76gCNTzZ z9z~EteuueG5yfc442#?tk~$6?{qz!@p|q;PTL2q~;|!r$X7h_OB1eOHp*3pR2}fI@ zh<95F_wvzC>MLR-GsHG{G$4i%1nwO1>{P@kQ$lSiV%isBaiN=InL$}5l?B| zWPVawr)rnSu3Jg}Ay`pG=`304zYtv%wyk@+H7IX2sx}7bMic!>ONnz~ez58;5^Tn8 zwDvcoAgU!0kFj!Pk$n&4zAA>Ew8*q~Ym$_cVg+(h3qxJd#*ze3i!|RB)>kFC$Exp? zh@Yw?9*GFF;6&pTMOv6a?T@9@kWn2}0c@r^R(Nr{Bp#khu~&scf~h$jmR#(`M2!TS zVyWhWMQl_q&+gx;nGnjOO_b`%|4ZJxEJ<=?XPU2K(jpuH(h}vvTsC}fxinY}2Ag?KsL@IrYO|qn<%6rY07A9{8+iYEt()Q7{osODj zqo?LES0>R{2JFs37apAY+o+iHK{m zu`MlLI}9W%Q6TyRu|@=NLnIM900=`GdCf6va4$wTiy2NgQF!A`h_N zV$@5LP;3-t2MN~4asky&S)2+0vZp$mh85d7&`)&JByxAg*Vx{Lf=eqR))B*2+fQmh z0C!xb7u*&z{F2FrUx>)50Ip_#yoWxO{EZjA$j*!U@)meH^?ifi|aTniw_8 zkQ6ceiruqZZsuk*Me#aUbb=T}ApBF0N<%;CZEQyrEU?(2D*~|E!=}1m?(#!t)ZmgE zrzS5{N35d?3si|P>tiF2-}Dv*kvG+gp#(!l`B~T!t9nO4keEXkEl1SEF$c52ulsXJ zy1W7Bvz1%~H5xQhy-!4*WxkvRb*L;ZsYUFQ3FczL$_~++oyhiDoJ+ZtD3lqoKAXR0 zt_v~tjCG0b&iRN|Mi$xf%=`s%^voR>-8eMtDzJcHWR#2#Je9HwP0#nRXua$n@))}p zqd5Tt7FDI*FId!2>)g>9 zd!%HxF&9`IY??h5U_UG@7Wj|bSq1a^c!PFVwnFv#7@3>bWFaMKCnVRmhF_#s6s}-7 z#I%OF)S-6lEcmNZ;f20^XA-5&qZcV~W_e{dkU;luqNH@-{2Rn*M*fC5_dNwYU8ib4 zGIlme<3fRornlHXT6to-9g)&pDjw84->cqFg0?~%Ac2L<83y1svjZAOQKL5q0}v>N zfCHT{?j-W1FjyEZ;sG%WQ8LEW-eA)-1-_=vIZJ9_7x~-3@_FWT5ZQT5>L3TBEjlyI zI-5f|#KK5u6opMgbec^p^YjniaZkbKAbmiBPKCY~yFaWtIg|yu-@_jBh>C1qUMWg6 zB6+TKL@3KTSU`fv;YNpV(H3)=Z{y4*F!mzT#Lih+bn-2Fl@CZ;rB?<3k}{oSS14*$ z97v2NqmwiuTdA$B61=LPD3T|Za(fN)UI{`J2PMPEbsPk$eS!K^lojQg_r?`++(m^> z>M;V+k<}FPn_>@(%}2r&HHz4=b5@s%J(0c+!mQE5ny1@C)v?o7FSel%hzbNrVSVmKObc~U*h~7&pnM}_rPK>hBy6&= zYuVd03)mIOz&aD_#v&TchfkBRTqxxZ3$P%cDr#4DxtbxRfc#PYm4#F+DQ_H0&AjFdFhqw3psj$$o$kB456s%@?*h!=9{3g2gahc2)JNp2A5~!OSw!J zJeD2QP01LTH-Q;TO7hyKX}dzJFP~#g2@uLYc2DYExt>D|`woam{j<;iayak*Phf1X zMC`iU=6?FaZeOAp&*ZH0dS4yxc5nB)-MwB`fI3X+2?r!(f!`(am=g$r!6spjk=ezal{jU#q zLR)^fwtubW>S4dTIj!b=o!9HVy~2YCMz?|)KRjMbh{VYWSPysh#}D({&B2kV`}OrnPhQ5&diOI(JXv^=oezQKlyLbk*{l(YmEBnJArwm1X zHJyY|uYR7cr~S=U_1)=y79RE4!#KU`>TW#?E1lt}4g$`L@cF2nVdT&8ykDJe-~V}Q z3-fq8A1|uyPGbO2K(D`|pNl{DW`0<&9M7uX9*(xnj@7~5P6+BBch~d&Sp9CdK5Xad z^zm

AZV=+Elv2X`%Xkn0F5gs{Vf3|MGaT>eCFV&FkRXxw@X#2U}Mn-Tmb5{eL?O zemhrh)|<_lFMB1kv9`%uug+U?zq3_s*24bou)xy&bw9h%8d%vizSzCd+kg0=+x6|; z(_1|=D-Q5IJi?OyZd(2EG#@VicHZvS)wePVwub#>@<)rExcp)-yRx-4Vyjrtb{i!( zcV}$;_8EJxzVsFTF}bquysd5fqgjU!d5ayE{gre z-UMIXz5jKu2|myF?;Q(oAAEkZ+t?zi{y~F#JlbwtD5}5MZMV)h{rh<*bL>3E^GTf5 ze!I)xPuFHVH`O=S^X4Ga?g;HS(|)^NXsd1UZ4YMcYy_vpdcB%&)|3A6TQI@Bg;s@H zRE`sryu=iDuO3-)sG7PIHpos&jZ8C1N=`((??h@?Y_hnnBP79K?k9571XW=mY!A{Z zB_#rpU5&)}If%T!M$!yvhoQuKFG+fEmRm(T;A*Ls9| zI*Np~B}p~AcfcZ>Im>-Vg#wFAm7OHGg!0=^!O#niSnN@fXrQ(+(J2(^;+#2aF<+p7 zITjIfj-{Krl%w}`qWmYMgXFQ6jnO{vT8kvU)HSida{Buz!;JdsD!LX(hP(~C#Ya&!{g?_gi|G7kaPc54GR21r+rV)6hOorU+I$3mM@ zgyn9?sb`@)urIt(Dv&A#W>or~U^>dh^<=bbt@?(WdPl)-==BxwGVrDm;R4Y+2PqU0 zGAv4o7y1c=+ZqeL9k)fQY!gwb=8HA4#GhK|y5%B9myj$_>5@bQ@2ZM+#KaI?xek%R zqO1d3uL5s`H#7?w+CuX&ua*=_^`2aIXltxT^hrA2xV2l=+7NSYryQB3azt*+gMUK%&ZY{7F2HTFfPQMqF#Qu*>?1kCmP70}Ax? zf+G@sGTGK?kww|IwJ5|8j4A9h)(llLfP9hbC1M_MOj3=4;Zi=C>EheIMn%0SWl~bR zA4h2#(~`Gs7^uG_*hr>GG3Z%GALv=4nW`;5&!wZIlcM@us@#8gu~So!Fd)_bc{l-kdd zSf3%p7)AA6kns504Qk4R*%0Lnshp}SqA*#0>`J@ zwsT63^=wE{6HpifH3C4upqc}6f{2Zu2%%!Y^tzZrM)ug1cSqd(C_EV@-p&83iT**V zk|jLLkDb^%ohBWiaqP)nikDN0kg$FfASV=1Pl|Zq$sF3=2bdFaW1>K^fXM-Q&=l3H z#|2L-k#f*RiR#TLfbzat#3%&6NVZK2N^|!`l6slpxso`L;G7;apD`JKJ4uSJ$2#MX z$6|43Cr}M`5Li3o7o8VMfCRG2L9sY43W;gd#Q~RN!sH+cSV;L|6nlyHi=pyrN`&ob zwk{S?WUe8P=A+@36auoL4s-JZb@(6~Wh(-6vUr0HGPRywrr`w+p!$fmaluB|@yOn4wTJsxx=dNOfn&_al}v zDEh$&x3O(H6u)tDzNnT$#HHHCAEDZ5v)dJ*U!M|-6p<}5Mn$+-8kQ#-Ao5gKJjOD; zU^Cd3*$VDQxbbGJC7Du=ygcnfuOLSU!smd`HwvIq<|XZbh_)s$Z6;A-a;;)^EXj6? z@g4Pp%nt<=Mo?Qq(Gst*8i4!P&5C(T*idLO`bgwR;+QHB>;40&tQHz!OH!jzy__~)Sf6PxsM0s@KFJ#KJ6NL)kwsj(L;eX&n`pF9DM2t7z+Uc_jpBmAP=z9Y0IfRGeD3uwYpG6b}Jt8D}d(HyipX0|to+{OMEX;^VAEU`7o;~kz z0yL_*=CX6pt!_@863EO}D{6UcX-PfOFqktC3rf4%iKgR#pr=OZ6j-Ec-0QKeSUzx5 z0Kt5>shC1RuboB#G&6&m+({`yOq5*@PNnN&A_sp^{~DEEhLmVR1{c@DhRNV6M6?SO zb~3PQlmsV=y2V7vsx+pt;NZ#OPh*EYDgej}rGkZ+9Xs-X<+cbDTUznXo4a6m(^ykl zMm?Sbw#fB$fw{~Hp4KIp;J}rz7%UQ7=n2&>2GIb+G0`Z3P0*d;4Kt*ZWu|8+SVDYg zl|rIqW2>Av{6c)Y5%yUlhdYvvUe`GVGE7pG6(!3M)e}iX3xbW}F(C>Odt_v-vG}y( zBDQ%t=(u!>Zc46>_DXJsok!T-G>F=^z*FbdIH;2yZe&qMPnw!i06;S!Y85*dX!F|5 zDQB(Fozx@8A;|M3<7?+CwZEvpsGy+naa5s`dg?l{{1ekyl(Z^&f9;@3D|$e}a8h!_ zT<(?_Gg2!uYfPaPz}q20<{^sEj68}_U9Y6X*ebDzKnVFq zWXYRWEKx96LM*ckkewotDlV;(t<-kfs9X&0c#sqZHE!EV?dZxoXm0uoEaD$b9tUotO=^7a6}Gs5P;%cXs&N5WrgwjZ`Gy zIFovv?XLeZnDBquW(1tTW1_uXZFYybI-IUg`|C`xdnPd5o?(#N`R(ChdT{>G`{{Q2 z$MtrB%|4v2H|vFs^BLapTL<6X2%b7uU$1Yc{m%ZmzJCAfe#%6=_W5_a)0=ryeSa-t z*y%W*_WtpglK^p*NJgjk|NHj+ulDIweJOHL|31S!=WX@rdYj2fFTX#{+t2jnp?d$% ztGnId41s<%9aneNr{B$|^=9+#@^IQ;I>^`l_{aCZ9_H!vr|N&Het!S!=KZg$-M#RZ z^XVu&;@A6WD|qeZllTAp$J6>@u0H)jsK49oeEEf4%)Z^7_Q%hB6u$lB``546tF;3t z57kc(^XfRQHuK?Xxy~nt>iBk-0k;q0=w08gKOWfgnB^-z4Vz{}C`5<5&JXK(m0_Nb zB*B8b`aB$*t#-Y=IUSDscURTtx3hS2^CMjTAi(4-RQTgF?KUE6&iLzRG^%HS`1xxA zdDq*MGav8lVKZy8p1qoMi?v?#^RIt!T<6+3a96v{W@qE(znQMD_wySk4c7Wu3zeMkHocdQ%Oy{wznrUuiNH;)fsZ?nzBvrjMi?rG$z->n}uyL%f3TOcpr z?bUiK-fBk#f4$q!>)TAz`_*Z0e`=HV`RmuydcWY`?ZbjaIL?K7u7$ci|V_d?VjKL_;*!k@&@+ZEP(ocGi&Va)#t~{b{Ewb zU%#q;{`))_zdvm@YhU?!8V<+B#GJXd*Lf&ygPa!d{=)HmRsA^Ktap#E{aoZR=WlnL_09AyudCNGN^k5ozZEqsqFr+lFdZWWYt*(ePj)DBrAtOc2n5o|Y?_3f zm#s^qBHLmkkr_ya3uy+PQ*WM5DyRr^+|il^f~1;xZ4EVP+2hs)dMH0E*6j!L=3by} zZR<#MnY}1l2B3s9zesFGSZ`Mg6Q&Yl6mplj79~iDEIi{_C@|$p@hHu2D%48WvT%Rf zEA=3EfmP22mHe5$i}4I1#Vjzz?oNrwgJd|R9Y;0a(WM_zx2|Qf5Y++&354MqgtbCG zzCpzXRQT$I+91HK#pbByH+0IZ!IVhNo=A9-EfsvK#_0|ytX1=4!nX9m;tTNagFS32 zqV$ql8la}%NFWbp zT@dS1T(fo%VMF4`}(_)gJgd7%W~ zV+u{&=)~>iYGq2w!~2W>o+xjFYwsfRNYIlOx*IwXD?q}rZBs-PdpEX;x^bNZ2%#r~ z&3izeme3_4YV1URhpfyJX^SK^FzrjW!?yW(AA^pV`*#IhNEWdK`EhxQrD7+D{`l` zoE!louy7%&s{sxd?Je}!g}p!(MLUSGNFCH_R@mO09w_O4 zv;R4$pNs$=A&oUs*<1M7mCVW6LlR^Ie3a!)P1A>p&D{6_THYG?Ih@0U&`brd}wk%Wn~ z8?2;vbx`hWuoFP?FYxN-n^Ni3o+yr3O4P)*R9_B4jF4J;P`(Q7U1*yECwM;KYs$YU zCsZ&I>nv?0G(>@&l$pSEv=%{i-~tM{uihU#Fk3wTTxps#d?f>hgr&m1b{*4C+n!EE zi0WUU*m5&E?ErSdisTJL{iLImoZL1tL49KmK`B3enL`3$w6J&AW(1?Ib`+U4)*@4) z)RPCmB=$=866>Lemd(^48CtNRr<{qsx4cg3KQTY5A-+~+!36~FZ#EsOQGp{M zAAypD0{z&{>A9o`*uuuOFmWh2dg=_JX`rb{jLq;!(adWL3%v<`s$!iENA6{5+>M3UM&8PyQ2NOxTy4|{!Z#z9iY~^yX9ut&gE*>@z1z%pKvEsw zw2|`ecH$*Fa!}H-dns?$7EY|66P;)%!B1CefY~+F^oJv&ED>#|*oIgD^6DjWEg~DY zR-K@m6mj(Sf|#Xm8fWpRluRw98ryzul{L-FU1k8Qu3IxbGX)qvJK*%jmWe0F0x0qH}GrZLOHkwOGl4PCw zpfE#>wNiT`vo{z}*J;FrUuZ*9f;sZ}7soQP@|7Y85e-SSb1oHGoVoO^XMUuP`u8Gm*-?)LWrhA#j8 z_!IHjjy5I!>Gk^7Nr%sPv*%#9i;U~MUvJjOY5z`8!RcnMj?*u??89bS&DH#7wr}3e z(_UPiarJ{=Q0%8u_2q86+U;-VtLmHM zf@2oKxq7{M_b`c%_{~DVn{QD)kq0ke&mX4aes}Wsy_p};htv5T#BY0ktwmUuJBy9P2r?uT!@YOr}uP@hjq5G+tPSt8NtuwFf zN!R*eZ?~T{ZJN~mNpJS`dHvmdGM$dQ`|X_P;C$ah^@nMHv)(SO$6rqS?c{$xpE1hM z0Mi*L{cI$kkjyvJQHx|+{c@%~ezm^2u~qWaF=U_NykD%ZH*1@u_kXt6 zenfFTTUphoUu?hr?2I^m1h;O^7kUn*zF%0e?-vO4w0-}-?u3nBACL2X?`U)Xo$bFL z*EW=!^?Z8z=-b`t%}y8k{z2&dr!Sm*c7K?KFI_%-hDiSK6~5l>xAPMT^DCP`5g!X4 zyx}Jvq=&X0Ir-w7_Q%8DEZg{l8)h=-b>6CGer#1gZnJCKjW@nw^rXU) zPnq)E!RhlV+pg^y6mGxLV4n8#-#ik6fA|u2YneE$#ARK+IdAv#Pz$7MulVix4KsA! zQQePd{qtp3Vy>PKwg>Iz>VR0U=DgOtr(wSQ{k)yutl$6H-v4pT7Y1~Pn4cMvuhpV) z%xwdD=i_;l@7Fh%79H2mA7iTa+U3`~-7gD4^!9XE?bi=Sn~e;nzp8!<{CBVu9qX8N zr_LoTH#^=D6TFi397rEx2&LvhcBGS-IU&w}0R^h0PDj0A!jq0g1!O7lh)%NXxpq<* z@`!~p>Pt7oUAguJ5#nSnDQT`WQAiXLH?&YpRF)9im`8!0k&qRnK`Oc+$vt4CGPI;B zDXq0Y@mtV*AUosO%Jqg?fQ3#PEto9xdMP|@&G#~A?&Et5iUchb{OS^LPDz4IK>ea3 zR^j!!!>D#S)GAc@oZnp~cT#*A#3k8NmfRwz)Y3u~cNKAGDRkZ9I=iM{|&s0~RZPS-XJB8-G+!vgluLKS%ImAIoEpg=|=VcYs7 z0&P?w9+d5eoohSVmIRguVgyBXJy;Sn0V1uT8!T*vcej|lAze7tw}yhsI46pdR|+ZI zX=)^`GE?L7S}=(ghI9kWqQrygbBU{rs8Q@dnI2LfB7d~t8+$2YQ>1MDREXn98N$|u zh|46M6y|D@C~Nz=B)3txY#~!yq9C)?@?aKzBNa^I6mWfXWM{XpnB2N`v@pE79ej;i zuPIs3B&yNc@=?{3aDDc1i#(-v8oLr3Oo>2o1mIJ{YG1S^`|&=+7)LamgNNXd!LoM` zMH5X<{M(uUHbiAAuz|g4hA|d&Vfj^hX_i9BEch>0nW8jop`s#QbION^Jt!c_jo8au z%5L;Q&JYWU*QE^RWj+*R!6W835sI)d+kPo?xkk#1o0BATH!XX~=E3o9ra6hv29r?@ z(sBstRp>G1r6LtJEQ|@yhK(LdG!iY2N->mt7i(3KNUs+r2`U3isJw*u z#1{dbSfWZTcxLqX!L|aDCVKH7ljxm!RrpXt8Vg1Yzqe@Uh=^885&61YC;?8(-pm|# zDCHg@s>6=48!7$hirhW!gg*6!P~ft0o1rMCx#yOo?^QN03EfEIWwzOD*1_MARB0$? z#KVg&bRU=H>fFXq`daRos@MaqPG%BY;yM|XmV@;vKyV>E>T4tk`%l|H4pOb8uAXs} zLr1=^8fb`4ca)Y06pe?3EJ(eXgjBE?`$HwKB$Y!RrFu%PedQ0{OP2%8A*PakU@ahpB~Wg^$4)*AEq4+GIw zYt$pPaS1Jt##>B0NK4nNWQ;dg@v;y>%u#&m%Sbw#g`up^W-F%R9+*lQy|!S*@-t1E zJqQ~aa4Bk;KrTccE*wV_+BFtgAvt2q46>6mZ24cGgne)=n<@S>P}U zS5B=@wKp;md|C+<3-ZawbG}wm0Jcz`3eBx7A2uurIv0wB$mMc|^w{-hOCn(gr&E_U z*_5`?u45AzMO_CtKnc&a4ES!$5Wz$?g6AWNka`Xi*VCmDfI}uaQSgn%I@JZyl^+*E z41o1e_bc%i$u<%qLS^HCc>XSFJ0hK)oq7}|=+2=4A7$^P5sm=7wU+f{YB@$I#nlR| z3iuqoL@f2{eFgPG14FlPJElf9HS*ek=Z&O(r&KuX0IJ7^-Bwm~V{-)-%A&ON*dx+Sw zSsY5~7~iCw{R9Wi2DwlI_odQkwGd3hg~%w~)`)?cA|W~V2)Phzb0LylXM|v1q|iO6 zk{CAqLXJ{F%8&yXi}8ydrR?5^n5fayY-Zv}q3as;X_UVh)LKlc)0##s@P{EHkkqW1 zG;_$3w`%DpAI1hgjD*k;uCXg{%}aL4;(U@vkLI%@p>-b|rU5VC;);-#GS9nc;)q$U zw=fsdnL!DBl2eoSB9)!ui{z#mNhAbA39>mV}}HRK?qGEpA4O(-ce@FO)%{_i$Tx$AY}IyK2<;*b~ZE>EE(@-7lqfIoxwngQSv++8&kW zC%dAK$R3V)B_RX0vZ=EMaev9#gdlLR$Wze<^i7SNZ5$g3u!7T|z7uL?R!};CsSCZD z^`M7i6nK(WN>CP=_%#C}k9(o614Jc3Ju2oXF`a)H3kg3zMMrZ;_%IO^58PwPO9e!) zH_=?=L;$iO%~c~2L)(rmFD@~4kY3f$sjx!Fwdk`7 z-X}Li_NW~ZA#_PVa(MG3K|)Hh>$NF~sVoGZij6|8j(imfvL*rjUDm#Eo70z`#qaYz+ajk7q3 zgLfY{&p!Ukk-z`1z@vQ#!+SOT&Zzuv5;$4i_J!r$py zcbj=H9OX9iZi=YRALQ`~pTL5LYCmtz2*rC{B47Kz2dS(+-L6-^Y^Qts)zAC)e?C;d z*Vi9G3qPygPP)_m!)897A0%~^V`lRh3u^ki-Tiv&_`@G`my-?c2ao!CciLCCo9SQ! zBgpSFhS>e==-z|i)6=G!R@2RV{{dn&)2XiR)^DD{M;AciZhyVoZXf6UJo1jFw4pz3 z?MM5=`uOho@GlqmW?M}+H)}^*F38Tu>p7dN(?K5}AMtwccwT>oS2jln`{r7R-7hyb z3=385rNgk-y2$auhFz%D`FQ!w_HbMuPqubG`Fww@=KPP<-TPm6o87J0dViYtHrKYA zUrp*>Ps124!12ud`fOBwxSRI( z(`r5)9rpQzt(|w`y53bcQ}y@u<8}3)pMLcCR~a_?)9P-w+h`fQa`x>HaJY`cp11FR zt$sHjR^o*{z@^R*@Q2<0=vd=3mu=>y-aR;y{GhKZdy$0=T-UZGwOg$--2e4%cf6i9 z8~>B@!#NH2@8`YPovW{}zPx&Tto`Zs+A;6;(|pmNj?>L~*Gvl$`H#~|`1RCWEGtWx z``6XmJDb`ke&#=%^zrFZxgZQqiFHa8+L_Sjt`^~mC7I%7*{GR7#h28!ADZ{_>oBwX!`dWAH>ysW; zcR7#L#Rr4n1G5((+KR@0fKdCNn>%xiP@5Q11L^=M``cKt& z3mfx!7as1`4-3rNFMKvLYoYy*)Am?gDsB@?fl8pxs!TbQ~lv9*qaUFOeqi~?aGA7^< z7WOR$P7MYf(gPU~|EBi_vc072!^cI59MamGV7<8USQiZJS27P(S{82X?H z8MWfXD+S{?hz=TZv-VVp7;c(cuy*nbm021J#@<{NX=3tMJ~c*LAc_DUi}E8Dh3nWS!Zld@+k@X7q}oeyLhIVZz``d&ZF-4LlCH>@)|fX; zJjsLP8VmC5oRP6jQjWP=CT*^Ws!;(Q34^tg<|TS0)%eOhiEiGDU<@lXFTp_Nz!QsN zCoB~)-gQZ-uyXdq&IxGB*7kqVbFIH0Z6P3FqwR%GLTQ#-$tDW=+i^?-#@8uIIg7Fp z7C3!OHC<<(YkPIVEsa5jfzY>6&ES+@wM}cQmjWl7*s+j<%X1iJId-eWgqXutI#KkE3Rg9WgK4c4WU77#k>!H*>xTgw$6ZpoP}j}1 z%9u(zUY99Csv+Bmk(O0oi=h+RB_wpkJ;JVgk4ZLHc%|KA$DrC#umlrFWFNP2ste9K zT}oMvqW9m;_#ogs;%mhXj+{-SZ609Okz9Jo@|2HNHN(K1OJ=M_W!PTBNaAPaMoKD^ z=A>lO&dEP@l7ZvUzJR|N8ZopXFu<`bbV_<#eEF=6c;*P0Dw0m@FY84PjpTdIf=m`D zIHd_;aXM8NNm@g-35MBA+O47E+o|)3;ii!!+Q3i-LHrS2IogIMkXKp~Jgw7Y~qk`;(R2=Gp5z)_lbznicSNoaR z_M>?lNZK*>jpQYe8`XY+&`LFn5ec&@y|z{m04A~v@>Lse;(LL43vu0zSc)2`LJ#IA zG&RVVwvu2TU_vaYuoDR>*V|YZy=1;h)oP0J;)4tzvBF3)frehLQ(LH~x;YzEr=)d4 z>eC91`a&)t-@zuN2pV(Usk11Ej%XFC8Y&rUM{z&I@IpLiBNB*D$~i*|Sc`i>BE}-e zrPXnP@KA+X4Mt)1Od3eS!nF2iFp|NXumNj+LPYfF=Z>vN<;ZpEI;ATpijX=00k$QE z1mUk4>=x@tK{^=(^6NxEf?SYhmr*{c8ym6f5*_-W)+n$+W@rM|ce>V%6V&@uZVcr9 znGNVVR38^Aj=G6#aK{0KSce!;lhGSTNo1fAmaM}qDgYLOe>*XEB3@cl-vf+rGm4!W znedW&7-?htK&m>Z9@x{i z?j=@%Dyf~2?L~`Q_LJ1*Oc@g?zmpSeOT~^dD7cz282W-4JuhN4Q-fQnZai2Jl3oC# zBHG)b=o5Fd)xN+8--=le93E)`jmXHB)PhS!#YV~wEYgn>Z!;^VOA`X29pV8fbYpWq z%BhyX1}keADAaGoa?P3z!%#FNc|_I^T|t_dlhv9mY8+KkS`jZrMU6zVJ3QrOS^z%U zig*}2+DuQ4p_eU{X5dESZeWfJVGk`n004q8(-?zD1|VfoS zU6g!s*l$a_uu{IUqU|{(;}>lN#R0WcR460Zwe}75od-KI%iSK;6GTz3^1Vf)xLz*^ z#7C{=&eO_MC`ublv6%&;x+UTIys zLr)E~E~1L8Ymt0q>>ge`b)?=9g3PJoz z*b;8{Kh+UqCb8sOpeXhWIuoS76e(Y)c&6n0ckfC$Nph~#c3d?3U3&MljTnYTqWZy6s1*lZ%r+(EaUf4+S-`JNmJuzNglE$|*%+r%NE4ttpAT zR79ZK$W@^uL$no?RtYK-524h;LW zF51(5;%$*qSM}&^L6CTKF&_$)ZJtEC1uI@1^K@qD9Nez1R3*{(Y zK!gt>%0g_tTwro^rF+#v+1=X%6KVC;>Ux54x5b{&azrP;!4D$c`Sn8h`>@}=o)3roT8{U%hnP?M4?9QTky}o%4)NC#) zA$Bg7--qV>!|L}osfX(NbgVW4vD)x_G3{6L#@_Lw`eI)H;gJFM+uinNw|z!Y`*gQw z&-!}36@l+EE4D2^#)wv*F67F22=a3*oUmW+Hai>g!)JEKL-lsH5uFY%flVI~vi4?U z^`p=C)0bcCIjT>uZRV;EZup32{Y+!FpXcg7e`S|_TzJ*D^K>i9-TWeF#Oi6k7i;s% z{?}=*1tMBqAL)7Cid*|##+}2B{`&lswa>zEp2X&=`e3U4IlemWcbcK$*+Bc4R)^zs z`^cE=h~wAO{qAIww_d$lZRTp9F~G}1ywy-#R6h&T{+v;J^LBc7&?VQ0yAP1+KdogL z%xCWNM%atpTN`2d+nbMA!oOM=yzN1Mp0`_}d$qQgt?~gF`h1gR%J#d{?cL6nTz;s( zd2Tkof0(xy)tx}SA5xY3BiTc&7JhGsZ0|O@;_s*Zes@@Gh>g_pIA7Xa?Ptd%KjWHq zkiRz(R6lLZ;idwtGn*tHJ zFq@cD5y3@qF&%#*x*^&+NfCkPVaIsr86a1{{}8V4RUi zmsR*g2w!v(W591VLZT9IRg@}3ERxKy6lo|QMGHS$RGth{gl-5{-m7rIvhQYw8Z1{z zBAX=S+oX&vacin67;0}_9HMXtkaaKzy%4zZbtj>A6bNZ-6Jd2k3SC%oZUn|0sg)su zB#}RsXi}&G)dejt!pBRe6R?T`IF~HDnyOBGS{N8AKEOtB5fVdG+}h5??(HgHv zXx;!CEXH)^*pdD%pj!kYOqBf7Yd6?Z-m0RC2^Z-}Ma(I=T3Abs3&ANDY3NloK~{ub z%;=C09^^lvEZU%&dX3YiK53LRweiB<{GlnVY`vhq2iT**ZeXq;N=ns2Qjl{o2AwH@W#%RCmb2{b@1Nl<53`SKzk1>!8xQ4~+*4`m}iP+7A}U&FLQ$l# z?IUINM{|L(Mb*N-SX2c4WN=O0@=EGBqM+A&W7|6jdNkLc&`kDWq=pU)2o!fx5!#8) z$vYj)V~2`k>+%nToE5&WMOHF9l7_laV_B@0hb1uZ!H-L1>WkQ;&{!N?N-w*m2_?1b zVw%jK26uW8$rzFM2O;0Ngl9Ic!==@m9U#gh^?atDSX5D27ywUOaG)%FKr-*y8+RX6_Q1PGtgW zZdbA}66vz-4r(KXY)7viT>v*W-YGyec=P_{Si*e8G4w_B<7?ZtC=z$s!O<7J{dMlC z7J!kKi!j?X#8Nov0T+>KCF^J}Mnz1-^w6c4C?zM540li14V1%#$1-22E{EPtWK=T| z%4^qZl!JJ&RkTV^oK(>RBVkT0HQ9^fxh^m1o91f{E%I-pP6>K9-)YsnSE(i&S^ z^oU z{HPBcQOkXj1PrzE3>j^^|{!|o?st`I@++v zAX@VpT^6#efus$~>{4D%0LmhKo`Mu?oGKT4I^LD>>E*5bEBo|YX*(ih`?vbM-r`+$sNLJYInB?$pjrc7i3 zqyZ}KeI>HW=Sl)-A``feFQu_YfsaWwM4D?wk-xVU!7N-3VbgCR>Z$RyOramY`BS zbw#%31@_1p+?(54)R40K6wQaA6kDT{tN;YK@#wJ)p;xCf6-0Vv-rA;r_W5sM`u(2< z11CtL1tgC=pEJ$ptl=Qnu%)E|Rb&4!QFH{bm2*9)NZo7G(X?QZqU{O%)w@^N=N`&_?WAMYH# zdb_pJc2x29UB1yX^ycFbfBnrD8S44PbogaHR_5ylAD0h4OR0M6w7z=0Cy1%P=b7hK zgvn=s>;K*GJ%f~9`#qk3-`~z}?BCA8JRkSFhuvm!CUbwQY~SwoBeRV0gJ> zr&Um`riaPFz!~`Z!+PHD7d-Fjdfp$`+pFsHWm6n?50@KTZ<~)7*mQqkt5?Lo_FDm# z^R4ZD&z~-j|IJq5+4iEEKD@-|wxHIlY5O6h{bCszUm7R9^z6WOA6Q?7kFZ^Tg+J`p zTcK$6fhOZ&wks}>U>kw2muo*h%j>tnT@kG6Y%Ils|q3Dzt2|cs~;CG?C-w(`svb# z)Aeb8Z6mCKv7gFDJ-^x`#yc;b7q9yL!*RXQn{5yF66@2w{^DW3exvuzciWjwuubW^ zFkIMq{q#=DWd0UxZ|ek@R3#q0+6CT5ttvc(J(3*}<4}1VSvV3JOmIlLjzX)^r9!|% zh{EJhc~zBg==QPKP(L_;27>x?s2xwN8{}pwhfNNJMWB(GER?e=hzV+^gQ|<1yFXDs zHi-LkaWbTqk`zZn$ptZ?nnO%LnO-?ziq}JMX+H3kC7=YU4U53cy(L!6)T9C=O_=#7TmWz zfP=&&Ab1@m3W9`WpcB9=2G}9qiOL(PGuql5DM~sBtN<7ai%gWKpeR}~KkQC5O$?N7 z7cp&Yq0vvt(7wps`c6B6m|8Q2f|=x8>5e2EC4nSD z8mTl)=_IvM^hnjvs!T@o3ffIJEQAKJsbw=G3WXviT4%z)rrNm_umoVy9(Y!%UbSFbCiqmefRlY3j@>0U<+M?dMBzOh8A;~8NNDb6rY_MWU zI-SdY(z*wCXD~a5DPci7Y`uV6BeA-ol9sD)L9aqE5|PUjfUwg+hVoIVCboF#%G%rm zH)f|^QRD+0M}f>Rtt9qVDs3v)2X)&{s$}FFn%z|Sg-nr}*APZ0-d@6>s*PksFur;w zTEIA{bfJ-?3Re)78}*@^`6x&Oa{68;O=6V`^XdZD4+f=8u>&QE^NE~CkuhQ0lLW?z z8cu;k*{|bctnv(m5m861@@2hfr+VA)Ix;4Gv_;Cv0Fe>O3c|J3B{A<0)c8= zsGXL??ZBM6t#}P-xaT7p{Ve`vSd@Fhnz|iyEF-%bIk=EZR|^P_Ca* z&4Ada-6~^AWO}23U36NQV~_9Gyd+s8norTN!9A1;qIUbi4+|*2>O~3x`+QWH$sz}+ zYut&ZLw8V@5qjgXL*mS+oHoLDgL;oq;B096u6X%8i&5K$aGzs`bPl3b^9Zf{iPY#3 zwj%22k#>-s(Progl4UCk`z?hNAlBoq9rVZ)Q`xd^2t6D}Fq3o`BOh^WfR z#eK*H&O1VVlZvxylmajt)MGwINpI>&5IK+b$rEE?Hr!h_H=fE^?I89!M46jURTngv zK8Y4bkHt9V11|K12byOwaS$SD~OA=I!9ANoRHL|?RCxC4{ZIsXsH zu{286b3;n2iX}L3<_;CQoq5Z31IsEl78I83tnPNHdW2mZ!ogE%QYUj$@XXepyBrgc z2?e=o1G$N6m@e_JbtZhNiw*}joa)KrzRupF5Fk$ry=`qgwsgwcYl7_@%y}LFJ#L>% zKcJiv(F`G*P_1|Z74Xza9iVqgY}z`}%JCHqH32=lq~Zx>zFVoKp!Ad!Vh}vu^dNPx zEFm*d*zP~bo*JX z^rvSxc!UyGpTC{<_Wj$n7(CZo=W;!>h_=peD>~7C|EGV+z{tM6YX8cgd%u2oqWH|1 z$9q9EcNrylQML77=|4V(0bW!gTs8jH!Y-O|%bj0%Tdk)3;i~##{fi)|TYYI)zMch~ zyxG~c=H=Dr&AhUQSm_6q_(iuzd)4QEzS_^zalScIq1ykup0Z9p& z+}#TaX_t7t-`!V2^BWh&xtr+>8n_$-4$tNZnKxA%4P>G`?~1X};K+HJQsvRX7d zfyw9j{2qtDc6jOK;bFFK{S6)CSbh5Q@BjY7A*~1JBmPF9T8F%T^{0pZd^lVPe!g0* zC;N-T^w~va4{$H!vR}ytajg+J|Lpy*ho@@@mTC{UJ*{mRz7r<&53~L2=dXVJ?BWr^ z`61TRp2sn(oAtc?EQ2_oJ~+d?SG)WBlK{@oXR12P`!`w*dBtRm>%|#TU!^k))gq9m znbSjr3-gTr?O%VnuRe`+cNPBaU+>qSX;JtCEvqN9;OgeK3s1MFecs&*`1$pEZwu0q zskW;!FY|fbWK8ViQ_6hY?gef?iX?a?2=y6dY=7-5`|bK>Hj8jN*Z^;KE8(CY_k6Y| z%jUKK%kHmTv|robPS@%jurQ;wb+Xy$0UxxUwzopQ-`MM9?76Vx_Wy_BaZH-}*=p|U z&i-rnaL9|!0=BCmk5A|u{quwIG28So??u3UICBA?!Oj2WKjc}k7nt*6IuFBt`41Oo zr0zxa_RfB4WB6u1KTp$-`3d|1%yrGTw;=e#o1Yi_GUqm;_d9J0+xp@HxBc|gzK|E0 zz4!DR>3UWCiRhmPi*f5hkF9nd-KOml)Or<|CgQ}Z-QA`dlN_=TNGZ*NFsc(u6|tvk zuc|d=*wSC6C^aPtOIvZ_B!VS{QUSw#S~cAxZ9{BdP=b?0SmeUH*b<4FPuWk5)$3#q zS<#J}{SdD-@z~wG%ihVT$dh78CUNdS+-<&7AQ)Bb zgz_d|P}xdA-*ingHb~@Z5yPO0C*+P)JEh!U7kl9&v2QsQ;zT(SqkNo0R$EDRp zWPDa%ZONUJOBYE@BISzjQY>Nv|3R!#Nmm?1B4(vTs`L*bE^1Yu4J4DSH3}u`q|`lF zfI&hmESzKUeoYf4kU3L{D>pZev=fBtKuB}+0s)j7dc`t;8JD)nVcAv>fJiU*3236ny$XS<^Co)?vk z6NQ^ylOp#=v<;zOI!D-1c~VSjMN@L#1Z~hdq_T+FYUa1=+c6QATuFej=%&rTFk4A2 z3SvQwi8JbE$qKe%Ungm72ZA=VE)D=DS-io8(6py`B&v+vNuGcg)vC{8|8QGpopk|WAQ&>QI$xM)HF<^v&>OSY!olw3vBb| zI{@~-6KET8bt0p}ikMduKF)~wW2K=ck?&r7Hihqy>$7cJU|RCX30rF4kWf=0@$FW{Ca^$O_+zQMR8Y8Hb}H1B&Zj?aFD*^X!w%vU#!dwZ zRKjRU7J+c0Djy+JN79>+*=!{cjMo~?`(x$et_=-m|A$5_CbYCiiPS>Qs})%Xl-8Qq zHi#x_gUa#%EgRKtQuL2$atHH|;IuLeIGR^OYq773#9qPni@-Fr%NAdM+f%5(*H8y%%aZ0{Te$rj(nE;?JWRs0=o<&NbT8;JTRG z5>r6|lUJ6VVjZy@MDzxn{F3S zO%!q33Xo2$8?l!XExcjlDR#y*t1AfkW5rZWr@qQ_FVLC3RFd%WZ4U|kta_oo?F{xvQs95UG+-7iHN6DabH=MR3dAoMFQ&wd2+W{J>y) z5gw}+{3*f->w*jowd6#WJDT0ZRS(O@7 zME#^rZH2V4qZs0`ORtmmcMqum^^KY%g^E`qJBB1$vMrhlKL3n)C>p=qsA*wSZ>+FE zO{7;5s-+`fN#p%6)aBHOHJbc4M?H`i-bcL#l{s~qi&hyfrj<|3$#$_~6vI_Bs2gV_ zuR6#Qn!Tq;qY=#!wujm}rVcvkA&HG1c3skzB&O4nX28!AMF#}f3by3Ump};`MGha& zZ@9AQ74a+Ovu!QbtjQpcg~W9&ByBMc5PW=8n<5NXPvDL@v*My+1RyWXpB3{fH$0}&d-265!GObiPU zwy3r-;FJVWLej}Jq$Cb#$o&oCd4|ZoAF$g6$cxv<_pP^o_)Qcsvd_{$SH8n&k4^FakP1g`xq3~-%A!_9-YiLw@0lTGa7OnSD zootNMwwa=InMJCF8b-EF!deo<*p|BFeIT1OI{h_9Ix%Pol|NC}59QvJ_-xfRLH>n% z=yWlO0%6-xQX~s=Vo{Uh@F@n0&OS@wuNv-2G_)J-EKfvlu7jUq)<#i$(JbWeZ)`XW=2liN{Ey|7KqeZ+ccW)iM_SJMLzyb_Yht4(W~nqji$X47C@E za{{SO>;#Q8Ln@yU3$dgidNdiGdxr=U?nlrZZLg9Vq1p~z(}1K^qrEn~o*C*vVqWE< zwWXn?GVHUYj39Pn+Zh>{u1gIelvfq)LL|Cw6{*x^49ncXm$VE;@Pp0%_F#e1%66SV zxhJVp#GtX&^DnVv)Z2tXQk*(KFw(`GYQpXcu`s_(^ly!@9x)EDz~dNZvL|Nc+^^0Q6w`(MxBo{^u)$Nw-d zw?E%Su<_mgCL?pjxvT|b0W&O+O7edV1S3$DK%aMeppiteYE+r2l3;JzmA%Y_+ldVqZE!d12RGEp*gx@7Ak3JX8ZScUGi$46X{){X+Lz?vku3Lcw|i^v^xqj|L1~ZYT|QUy&Fv$Ke`|kWgRx!E z$iD3sKIL6r*!H;3ruaO#j`kfpe~9aOH7#J?^HM4X!2doyV)I*v@4j7Vt~*$S&sU!o z+lOoG{21O&a6IpHVcO3e*!fRC(Qr+-3s87}p)pw)#;+ci+U17}?PQnN9(HAO;XKsq{bB;P0yMw9TnXj8Jj{6yzuMUvIs^273qUt?odYr8 zP&Ain98kkez|vTra5<(FA`H$Yi+O~rzzR}taUdo96uJ?~&L99zp^oT|RTD;3gl@%{ zgU*5&9Z4t1oO?=4NA=qi+-?-Ui)ullsDfs^B#Z<`-(of*K!>0-#gV_%u}6#cC}AP# zHsZhOg}S8R)Xbf^frLm(5>tog6+;uz<+g*Uyr@%5F$Z%NjA9=IuQ3d2%7XA-00qRG zHOZ!sr85Re7*lDkNx~4&UTAvhRA9My^lAYpUbJez1uw8^V=U{~KEf{KNVw$`Eiy+R zn9=o0)Y!XFAdv)thMJNzONdcs^Z_Gz*w~A|1*#OWjWwYWBveLh*1~NJ$Sv8A%6ZX# zR!Q{Zrwvz8G^k~dc+L>UE)YeO@H#*s*;{NANf0IpQ9(4bW=v{ibe7HrYmnM@O9;tU zgh&~hAkB~t{QKTPXh2zD=2e!IG7O?rCBe&90xx6&w60n~>dMXDH%S}nx#Vu~0r49s|9q=J)$#WbJJ1BppNg1L_p ziNxbGrGf`KXPcr*dO8Wh~4`9CG7Ye#nnqHzhlFE;N}x zQ@9U{E$9fBsA?8s;$s{WBK<@(*c0iosIy9qxNJ^^$RW-VH6vgSD@hp~3!-+s1*)Fd zn^PeqN*J|h2UzklU}mK;UP!O{OXzGXa3ntJz3|WAZ>SJj5Nf+|62^qM_Ei6G$=W1Q z7Sb2aph=qO>f9=D(b;gdu`8Kv+pE+Bb2eH=^$tSRQR;GNeT=~s5{{FA38ScrknU-e z@C22NLOYJB!R)pqhpDftR=&MU)XblBW(w9tyEWA?QOmoU#!?+h816M!OMJI9b@}Wh5lv>ZCDEA^^5K$eUEEJd)uzw^u$O zGGGlgmo?1*j7VE3MVe6Z(X?05Nq-_5A;DCzk_)Iw(iZU zmax~*3MB=TZ*$)#T9D=fj|3!%(!C^mE{6J+d*3^!(SftUJj52C${|M(qN_tWA-jxy z)uh6L_{eKx!%hjFStMG0v#RDCgSEE^4}rM7d2j`=G}qvytx2t+Ht03%Qn|o&$+w-8 zDUx}B<5Kz}&^;n>MW9ZUP%BwQ>;*wEH$d&C7y88dD8GtG?9J68`zK8)Y|&8aRhM*AgHd0WYeS z_GC6Re*+Rc2T2z|z?s_8{??g8+BPAaRfELc5V0j@UaaZcqzLCl6l#LHqUc2{k#;08 zlWaA1ZpTqe5|22GkWqbH;BThT7gaS8zS_hWUHR_H%gVFcoElE_2x2@SKe^=gP%^C+ zTQeoUx<0y60J%!$C-y{HsT72G-*BF{RpO zpcXngPaWiEC67>=0^cU)P@`hekc3?&e;`KnI-*p2uR6vHT$U}df!ir<2L`qMN?)CB zZ%+7tIZyEmv<0(V4vWSpq7m{GLL7=XAY;iC=&%P9x$D`sVnnvk&LQy-w?y?Qf>D)Y z6(FoOSJ3tfX*64uJhFFO`~2cJLUKc6K~y^cdhITNK|R5p@LM$RqlyodQ;@0aA%QBS z`hd=m-6hBsD<9bWmzGlP?kbe6fc}k{M{`5LNT?T^S)NM_YY;Jr{D$%bW>KlcTT8J$Qax&L zHAGm&t=h||0Z4R?<``fq!~*Ps<+ApSsDe!JgS~YW2Fn%NATp~}_C_Xli)W;oS30{c z#DOs9G=#bUlV|Ip;DAh##!G2Y3fj~q2+gKyDkS!#)&{=mu^?Z|*XLS862o{oEVg~j z_d^O-J7>cHIemqmW6^hz^O6!RB}37V*e%wHtpfX5L-RrkP6@E0yDb|d)h`L>9Jtlf z7K#|4%Zc}5HqqZ$)K^dlouLw7I3LfRMq6}53;BzcH~WZ zMEb2*F_D;+c|A=-H!)Yt;&J4Qa8_CH20Nx@h!U*BM4JVpbH!3OoNzaJ75a>tL#9Dh=dXv&dov{ebP;X+?)DKi7i-t<49QBq#@Nv za}X?QFB*BW`!7{y0+QRuBn#CyDkRVeYKLKmR0gBeDX{k333i04C=QNhnVzvg{VeG! zTMW>mCW;PJP%yJ#rzgX5&|ep&MMX}{l{wl~jFo4ft>`Z#T>*Y+R6p8d7o;fMFX?$-}@ z`rm&%J;D-S?>3v=+x!gE!^3{}r}e!Mi4Rjo!-aIbyA%rUw0Fqjas7I|I`fSBozIX$ zUE%XMJU!}y@>_4WQnSwx#3yj7znT-EZs+1RJ10V2esjjBiez-!RyKa~_8?^PemC7b zVr}0(;)k1yx~>;$&x_`a@3VIiSFFavpW%GVGg9v#?5!V>dITJPoTX_cUf=4ie((%> zD;Tq$|BOIF2z+D=bkZ86OH z06H!6?3Y_$tJ~G)bdv#~U+Omtf$yuymcsVpgU^09Z`aReIDh4o!QY?ut3NIS{pGvW z#_qQa^TOI{j(eUli*+C8%ek0+CFUvHYgs%|Jo``_=Ep~TSRZWR>PKRfzNoIB7@H6C z!?gFYbSCAUV6l%lQyYzkyLSg6eXHvQzM0?R+Wyz`D^6#EXuqtz+IDw5W62*h1oj_! z#nvrE^<^fCw&y#3xI6QhAMWfETTu(vTnK7=xxat^>t=oZ zNq%_m2{_trsG3^a^-RkBNWh(muCGtN{*P6-i25TNCcp59K>4p0;_dbE-ADhaEoYv&Pz|i!k#>-r@&02gTrs_)i0{oc{=Y;o8$Vy=J~L*m8&uPP2{?crvZU$`gVfY((qRJ46*S&%2Vy5Zceag=wV&X&Vhmt;D+luCbgsar`p3|TKb^NuO`>rezt zzbfUAR!u$x_OpxW?Zs+dkJ@Zv`X>a6$TdiBSo-V;SXWn0AkwW?Yy^~8*HQsqqG2c$ zxHPxo*++n`IM&L66^u^UV5ow}u`X~K*=1`~`J>{ZXpFj#bN$U#BSdXoVkP~2F%PUp zn%6Ganui+GDDNA^ep*VshlC6W#zQj=QrW(OnPUdZ>p2Y6*89+jf* z=x4I?$-ziQ`I*E}W`GpQkGa<+nF?DqsIOzBmv0oybaeW5%61f_G&xR|t52dJ zW{Y|%y7p2frREutRRttC6rf;0AkU!gX7WUPac7bgp(|a$8;T_vJ6yZb(+codg9Psj zV4I&nqX^I`U#6DMaD-uqBi&hAUO_-;Y)$W*zEJsazg8F~I3e03 z860@7Mw;0jcu}oVX(Z~XMq1l-a1tb)wd6h~PcA}9wwG8d9N7w(u%}UI0uD>mPFlW% zPCp7IbTaisv^fYXQSgsuOD9Y(VBkb+fDDzbs@g=rE#Q_tSn<3fQi~m;(4vT$NC-;F zL(=+H33n_SmhvtfQcGC*c8HM#Tsy_7$lgDU8Y{5nYEqymv2*Dd3YMgN5eUj)9*m+W z0&%Jtg2KB_Q5=iPB(-j1Ds+rJ`cT5y^{6Z>T`&?97721f)+V(mD6fF@12&X}gW=QF zitCzkY@$6Iuz}C{o=GE1TIopc+u2g$%0Mp^DfP=nCk=qfx1cnFu8V}1B0x4oeoEN> zR>@oJ=+qvG1!ssqt#X)<^>akExp2hbD?x4|AZa>RcA>6sJ2FO;F%gcmMWVRIf^}*@ zQX?+{Vh4?TBpQXmTqwfth9DlW>9Y}#Z-6DFKDII!>bDo|ZP7K9GppiPLqj+^a2}R~1!qlxjjvrsWgJ_CaRv!W zAlrbVE$VMJ4{^jwmFO-p**n4v>_6bT?rh{YxF_vSO(s{vL|A}KvmDGcJ^ zD9}qZG8jbp!Pps=KF0;{48&b>OuLwOi2I zUS`{18x69+(y~Wjnv8PE#hw#|LsxW4jJ`f1Lmz~PYswXLc$d)xBm!(&<>(9oXC;Jq7Iigd?P}6@>f;bC*Z`` zw^Zp1=HQXeSEC6Alyi^|Z>IU#$A39+_x}-iS*Aa2F3wobzWysA1m|b>$}FbB!!01$ z_y2V>U*61@8E{%TPp2dOR=eeIg`}I?8hWc+pSOUseD9Pydp!d>`B{-{(x@x(v<}C#pUA-8>zu=}^6$ zHygY9{VW*ZT;1A7pZfjo-rek{(`LOAS*|`f>^9T>{7g1`fAv}BN3FK=?Ph(uzP6E^ z?rq#0Z~Jy_-}xV3Ri9UXJY}}dGZ5Dv;*&GFaCP*D5uz|N6|Z*ZA)NAy=y@}?_50Jj zy*!-umkT=f`n2Ebhm%d!f9mVY`}Ln3vbqz`tUbWn-OirEUQrOqX9#2e@-S`J8N&FC z;CEZCkB4fWM4FqqXYKz^3t8;(3H0j^km+>S@BjQ}pE072M!u@P*2Al?b%7Avr`4Ko z>hqU@%>LS&`Ea;=*ljkByIgL#aN0US>g7B=M>RhPXa0Cs^D&n}ezV@#8oRXzwJU#h zx_$p=d*m+rg~h3_C|;Kjql6-EY9QI`jN5K-^Kn? zK-c^_y5r-Yok6V&FYcKv_F=F6@n9FsdrD7wW1l%(b(tFnZqN4u=$=P#x4Bs^cu_r^ z?jL+}+tY1)t#9lZH;;dN(EXks*Y@SSUsmh#xS68%Op?3W%#$zCXPmZ=nBAX62!7^* z*5_*t(?T#Uj_&8b^5?h3|M0MRY(TI}?8L_X1pa*n>AmQl0Qg-BkAP>N_uD)fhxzRj zO|F=G^RjcW{q~H9Kfj%Qxz4-$&Dv(e{;!^Ky*h2wt8hOZHSwSR@G8`wT|5#7uW#=} zI{XO9&#S$S|Msu{=l^m0pYGe=2I}q1{|~6|+?w-*LUp2b8(O6EGq0|VaAL(Flx!{e zFYKoBi-n?e${4r;}Zz%AR2>U>~LegoiSpm51+ph3+>&zCJkXciLpxBWw zZnI>D3x&PMLMO+wL?N)Uj(yQ|Sgy6_YA6;Ct|AQ;@or42KVo_(rJ<;qdEg`G33Zi3 zAWWfqQfn3T5!%!?1#{jKeW+TBEr!bEK$p4mZ=v2@+pGJWg5j=j1A-(Z_FL%G))}yf zU9dL1qMIOQHwBO|>%CZ8Dg6>_p`<9RRf{*GCG|{HbnRT`7@pQ*l0@~XPRTP&Zx7Po zBW^qgW|K6OjjatZ+@uu~C595>A+`VnT;slPo5DBF31g|rlLTSvLZRH6i!`k=L*+2q zf+O-J^VM4#6AO_4ukOxf$&Dnt(tAAxc3ox0Obh>sa1ZT@EOuA7)!#IyM{6(>WI}}m zQUJ1895b0|tIcfm5>wJvZ@@RvlgvHWfy^ZRV^8-iW=tc=R8{v_}< zWLK&{mqfN>FPu=jW_Th&E~6(VDRhd$Dj=JutpbWP)!A4pF?U(S95U)bi8nc-SrG7H zL{76HR2s5whny2-am?@zh*J>&t z681ti+~g05z4k?+D{yp;&REeuy@jr`4!C}Iy66sK3k7)u*CD4GJ88aj8RsBbSk9-0 zQBpmG8jhj_$39%uiOlx<-}H!dE}F|sB?hS7|mNAiPTcUzJvxp;L# zrC`CFWE&9Sq0NleF5R!Pjl?t&hG|YW5fc|la}FKqs|rgj83xK#%U(`YoQHm};5Z5-#WOx*!-QCdv3#!&sLE_XvZ!^y4-X@< z>eVD9DREnl!3Dhf6(aT;Sn06}R)(kz;mx_zCzw};uF0tGAk;Ayc^(cc3*tlOJE-%5 zq!?JB6sL?PPfJ*ioA z#CAJ{khC#_E0&{PMiJ2}HW`E{Q4Wgti*zP27`DJAlFw}iN2nbo=AhKYgtbnel!6_$ zQsEBWTl5c7oru9YNnsfeXVoDo7${!-Miyh6$3kHKlA||tqTgvw1Dq^EMUwTiMUtpu zY_>r<$&gO5>79@#Oij+Bm`uhV=rLWYm#L{byQBt?2SvhBQ+2sw52*{lQbGbkU} zi5j#s3Jd#!(32ogmKYKB$R$x}BNRdTnu)-~QfC%EtDRWKn}e0I`kw@?wC{yz*zi2eW?h&8&xkw_4Rq`Ug9Ympn!rLbS)u7J!KIO zV{cBbQdtLIap?QA+lny5PaeIwmLlJ#Tcp6@y%wj4W7f@a875eI)ih$8|F zgu3)>)l@#Zqa-w&6h9$D?a!L))*dph%rc z)1U|eKm`_5XPQE>y9V_B#8)z+_IQ*4IC>&}dUxY(xt16PLd09tf=-1R@g{83vM|zY zJguSFN)m90sSyGL<>=iIaIEJEJDbD_YBmjv7z(L?nm=aZ_>|lesioU0oOo+B1JDfPO2#p0lp45q}4S~t`6A!o|GSE-8u0?GS~fVwK(=>FT z(7}~eGLdi3K@lw>t8BN^oOqf7Re44sze%?yPO2CX6@${vkespLLehTVg-OHa?gXTKW~6Ti+(CJ?DemIw!5OoaNu;<=HF|Qi?T?j2YwgXanW77oI4Ti1 zD8ffleRX0-y0;YJL`c)W#<|q4?(1(c%mVsR!XS(084JJmL4;Wp3sOKEQPhA>LFFvL zQ$j9Cfrk1lLRP~W%&s~$k-J?{=!BxOVoWxtgN~(CGjK4OGZzvbmG*s(nv-qDOa%i; zDNtvT$`x|bASMY;3JYd#6!@I_M`P&A>g~Xg6gTxVr_2yt2@odq8jHHdVOdKKuU04| zIMGJ>j9#VyE0i^qMS+lAiGh$a07C=|sZ{V6`#-he7L_frTSh$SMB-)UM3EIGOfA|> zSZPh7#6tGv7@*lI1PPTQ-?m1JgcG7hM%FLEZ4-Fvz z;bb31_v{If7c1dbK?~!b?9!=;as|xA)t1 zgDl>C`m<+?E5R`Z9(?>wa-Pnc)zxviKdIhd?Y5!>{rG!k+x(k?8VYInZub;RxZ7Oo z0U86Z{mUBxcMsA&8uHx2UnJO9K+v|pXU&+1lqfH^*X^T*|8dDW1y`_;8( z?e+V`SKI6ThhG=pt`^^a_;(x9;`QqIZe>$02g<)2q-O9yTKnz&FRRJ9|Dc!v1FQ-Rj!r)yFBiofD7l(b1AO_N=SJiDh?d&(lz$f3Q2* z_ibhx?06;2W&7Hj_1AaZFa9Rn^&rey$)Sj}I;o0Kl?cHj> z`xtur@aQLObh~5Ye%rh_54bs zUIZb39s$MLUFV0kh2@ap)5U$2Z|(iPKEb$8&$ZYrZ3cu7H{0?>m}~j+G!LgG`$ERy z!@q6y;T+E&Y9?3vtM~rb^5x@9z1Y|?dw=nKd0fBUni;X@`9h6FU;dTcwJ}>LO$axk zGds1!7I8?12N4JoxUIy;n$(hrakd~3kam`f6}sbgXpDy)EY}aun3}m`uMQErbHK z16#o6t72+0`R$PlOx=$Zz9i*D;Ng>mZmDzLMm+Z5Lyv?;>5)KbZ^~oG?ctuJ;9SIo z8_G2Hv^c)J)sPPOEXgYl*pF@-A`lZx61P&q?mIGwe(Wv zPITT+aph8>DclSJLaB?;X~cdnFedUYrAZyC!~iy_>M!NyZ(h5AE zkk?jHeTj|*H9(ZEs@t*^pJ6Z2ww}A>g8%?R6C;3g5V$^LWpj)pd9*d4XsG$D;_oJi zlLVEiaAAfcVFt$*V^u80KvE1D+(3OQ|ZgXiZ4tBzC05L7 z6q_;lxfTy1ilnHHp7%6F!Jni_E+_?bV4z9#64o z1*}taTnnE7_UIi((ML8&jRw3Vb39PKT($qMA)>`^LG*9fohFM;R#V9F+99xprtmt_ zw8#b!cXkjf9TCO{3&JC@DI`IzJCaF1N~+kw5%sKQ5Nh2u0omeO^R`iiV}?xW(l z^4~(~!2>SKcl`eVa_cQPI1! zunb_FF231>zX zC;_lk$$){W>I521>K&fBZ9%}34v6ZphS-u}wv^?ko+6P8_P7^c*|v29pFS7!eX}B zHVT4Wo!XZY0V0oxfXD&>oMEC4!^z;MaW62>2Q|XDH+Ho;-B0=6Z zCRq-Y8l7XLYEYTgNJ2hG1CA21QCpDMtt~zlSen}M=6OO{yC4l0lvs>aeHP%Lj!8!Y zybs$3*vdbtDhf{+<#JHtfa}YWQ)v#Rosy8smVVhtpiZ$cWLiD8naq%*kB~H~ZsMDt zAEQk@iEW5E3}DHVr05D|VM12PhuW9ScTqPIQXPq^A0V|B{8*Sr$e1erEJ%RscpYiP zqFD<_&Jk3qK^J)HYiPU0V;iD^<9AHi>2)vh2l9{Z6&WXE8?Q(qPG}cO8#i zbsknE>&tR9v6gH;M-GUw4gxr#OvCJi9*a(j5Q+V9)_my7*syZhN+ z;$ii`ar;QoD&+F*+HtWBHGJo|=C4;4_J@PM>Ql1*Xh6Q+KVp>L ztZ&y&Y<*{M^39359kJF@6ZH7jM!rA&yFJC$2C-_m z@Qw2d|7d@B|JQ+O$5Pbm38%a{CtW-H;3+MOy(y(-htL=y@D&&$OcX9oLV*&t!?|Q3 z1dxL@1KvbbX_$WU-Xq}750ck}{Di7#bC*_dPt@+f3TI-d!vdR&f+5H%%84UEIEnIh z3E7r!m{dtZY;(>XiHW<2o@c=@gaN^ofuY%2a3V-M3O-Hu zR{NCsf&|F2Xd*)*esp(;{A1BsaKx!DX{1uD>I{d%L9qB-U9t2=5`_CLnn@6rsRk^O zb?ls0ZVG`{A{j)^-%)~HRHd`{2i^86a2Y%{KW+{iv>KQj1s0rY~v^t(*}FJ|2V~Cn6^c$^>hHXRppF4P-(s2kw{vi##BzZ*qMh7AmWe=np3)$N{AyIH$<)kS0M5yuf}W?X804_Z$Y+REf4q zGl3@OaYE(e(ITcm2sYbQ19IL)Z*nKt{3J{d7R=3-QN2c7(qjj(ZRHZ%2&yQknQ^l)@vN`9Tb6g*dYnJr@P*}8?3J8rF zl|950F_%1y)KNW3UVEL%fzSa) ze7vc4GPoCQt}IMPK?#XfBdAOO7s~tw^*VH%{r;P(^+>Lingrg$$RtT!p zU^yajqp(1%87;tXfSz3M6&9~7*r?e_j?qL;y;`nMrHV1gaVNf+flv^6tiAS2n|xa- zT}bOkkR7qmi1@+;XP89R%4iIWrx&19oP0RH<#&6(1pqvuzWhJZQLLiDi)xX?|i2EJZt= z;9yp}YA+k$mcZg^hECa)CSsNmReJ`bW`1Cqx<_dt#8f-w=w|FkZnG54xdkv}>}*hx zlpoB}Gfl3#m27E~BOx3DLJmagYYsBi-9`2fhEQ%-MozO7(j1YpshX{=2;&idf5KB} z7V#_@$>eAVn*B60FRDx3B1^FIQZdEgJk|+W3oI6#=nS@BSlB`>$wi%VSgMZ&w~@|s z@fIL_P5f7M1gpEIQ+W4N)9h_PENvBVa)jMds}e8-JXZ5OmFrbrsEoLq%r%*?h15PjF=lW|4U@&H2Kb}jeSh25RSGW(S#i>nBCU|vSVtWZ&4hh0Qj9UzqHtja1< zf+>xuc224PMDePmWgAv}fo&IJ6qFGtk=-$I{#zj{YFy%S#XkqT5dT{&#coo4rN@6nRc~!wOntH7eCyK3-!-@?Z@TzxVEn>H;3Kg zr|s_DR(zX(_RAN~cdyrPx9|VVb$|2wpM~K(qijEOx#!Eb+tvLqmy6#vI@!C`?ryVk zG-sO+f)cK`SDSnL@ehZq<$m#Z>&>;z_;2?ge!c$i>)yezyE{G5AJ(__TK0AUp%y>< z^R;7F*UL9Ps#iiAFMelpbZ~Ci6ZG=s_5GE@8XqZmPmIBKbq7fbr}+rQJu~~>?RR&( z{TUCbzuG%o`Nix>+H+kle!sQ%I?T_`Jg0fEkM?@Y4^UEj*u&yxwGjq$v$l_}Ru@hi zytrQ9ZFaZI#jc^BSNmt58^Hsq@#P|T9c8yTN<=)}4*%m-VepflWqAjbFIU#j`~y z?Z558-`&^-^wqpt3MAWZU_XAdIvjR;!CCczpvoU#GhWV%+x2FnyB?0VCcADbj0f!{;fUur|tS@8=nBI=NBfT zVF79%_HT%fJJ-P#Uzpznz&dM#k`e*Jd6U2Yb)tLydZ zXZx2c;pyKBdu@wHauasRqa(^+?{`1hb?@)&#>bt%$)mvT%lqSQyYu~Ye;%%5g5TJs z96#-YIL()B!&`gCcVlBcoEGeF z4i8(xUbo)E{s~TbwRmgaQoF&2Uu|mbpa1bs|FRgz#o=yswO($_uwVPj+c%rv7Rklq zA~^$@Z5P@aIYXN7?q3Us|Ky2nM(?j|39SA~=<#UoAQ35;yE}=^1<-6F*a@iKolBAS zGB~!=z@&vbOD&|J22*p1I@Uy;{pN}0PWo7>AhmO&v`4WUU1us$BZUTW2kC<32Af!+)4-zH{Fq5%sULAC?5TZ;Hz|>ATi7cYyt7%cJ zIq4PBaKR!96nL<3Q4ML(XlZfsX8?G%Cg!ezuVp8Tn0&C@CyWsRmXsOBnzdLN2;EuJ zFb530m7TQ+4XW(3TtA9&Rq35bx>Wgqk=mEV`oy}|g0W*dL@ITi##zhXsDlw+qpcx} z1qiGehEg%z=<0n#CrIa!zvF{orxf8I0d*$SX-BT)E_#r9IZRizr~)j3{g0CSFz-2a zQT#=?tda;R9G+Cs_;+M(niE@%T@S|K*v^UZtID1b(r9pXZPWu6Z4BXBB>$+gIDDcV zhx&3_K{FA`Op_ZBFlv%CrAwJT%LH(bVroDV&!omp2%3~qLM=66uL=ZHqND7Ef_zVj z+mcctg9~!gTnUBKDcrUNpHv-%w>Va5y3poPqbFq_!{ltjKx5!q=yU>zsfmcCMO7ra z1`1=9p~8XL-pQyZlLdN368Id&EGdyXhwTatuJ?Xt_Q;1N}`g!L=zZl{{9#}-;CAeZl~{uiohi~^DtEO4sz7$w7# z(sqOzD(V9v%wH8B6Rgvzsy!d9K%RrTvB6PMg^%(uK~x4*R2ehl-jglsUJ1|*M^fiJ zr1{KZ6S;kqi-JdJ6fm@tFLgGv(EEp{EWG%45ksGRlXSPA=OC_euyZQJ!+3?xCfJ4& zqI9s9*y-Cglc8QWQVJF?)!e~#2G`y;Q%Vo#q{AdgrV1&p!eG0g!{GU22mwLGq8ybj zLoV|{Oa++V3$uFKAYPc`q(ovr?L~NU0OPA4GIa8mzSBmb;$gK{@r47JGtQHF{!qbF z!$>A1#i(X8QeUjaY%knGtzM{Dp_8y*(Gsm1>=sZSRnO(rK}hnYUWFM%1{Q||ns&aZ zXK`E+MeA%2k3{WeMClEdtmZs$<#NtS!=h597)y}SBB6Y-Q2)qUTct{v$R?U=Vy1~W z+QiY>A@YaTW{2GfQKjZW&T10Z4wWS&{*45wSuK$rM5rAURX$4pBx3MFNygl>LivTJ zsZhaq615Ccz)rdI<1|y|i;rTLA%iS|5+c>H>5!T$ob%HFEarm~(AAj9n&~D;S{EdQ zQYb=FwVJ9iqY&=QKNM4ieK?LuU9(K~lZjiV+WFB3dM%Bi-X(%O*lO#kHQSyFG%7FB zAR~!IW@eQcP@FonRIB;ht+{VYy9bM}w%1|@QS}e9#M@DPyVntF=RxgNJz8o><`3Hl z#Va#)TyJ`mB-vqbN0Y_0u%175|&!Qz=+hP-Vr}yA#9Qa0!1V0?8*?y+iuSUhYRYuC_qN6 zKI$=Hr-YfgL^V@M8&Lvjlf-1!v@JGnhYu;9Y{2#OHpsZ=7)6Q#mDB9&_G)nR?b z7YXnnq(keP*wbuTj}%iTo!02>*|9WG?`ae}X`r6}?5+Z>5<8Qrt5Nwps%xl!7XUL; zR#!GKu|&1x&>1SBgN}x|#WtVe*|upy&Ny{OWK?e-y@`jC2m&e*&w{Nq>==;}dd5!N zEND@|PefaTgs{SQ0_e#cw662}l)0aoRb>)U2%uiQGY@fPMHZ($^_56O6T)2!1ADcw=HdxK+67AGQgjZ}Trl$@0K9seVM`27 z(@xGt77m*_Yi#FDAgp)i}d%Xs>ZqiQXHK%AQV8y-O-gF%iQrG`EoR=89j!5Cg%&~~vvLXFbkf5*?f zM)Lu?ANYdME`}3ZbvEVEXy4mm0QC#A+|CmNYfBYI2nZ*SU-9iGi>x6*q4u}fH3K*I zMmkp5Z{+mJD9@Y6fu2xJV5Liiw*4#RD~w9i6wry(4qii!64>gerHCnukn3ff)q2~1 z5Ncs%LCHyWxzOfq>QJYWY*cote{@IL6LZLv<+=hG+5Fk=oNd;A3Izov6`+riocXL| zF9k7r480fnB!y0|!Y9jodF196n;rJKx4u(AizrJ;@=D@r2$i5kG53*RGrtxxRqVrOe$&Ipu)sM+YEg8XAN)h{ceA|w12+3zZ1}SbuZlakFUOc3JMi_@O~xs-$wPj zzZbRepEkGWL64u~e06AW%+D4dfsSu>w~h^c`Vu~x{k34LYoUZ2R&sT9v)yfWZ{IJz z_;I&+@BeLY?Dr=)>*@(?wcW^`_rtGG9^v%&i&x95&GP+6A3VTTAOHB}v&G$RV?((T zA?;}(U%YDna#;Mh`_k!wmv0+v`ipN~zBu3h<*QfrAJ6X(qRc%kUOD(ye}3^}|JUDt z|LWyeUw62<>w>7lAdhe)Qf6~?d=k?VC2==2j)8NClsn6?Hmw6!8ZJf%B zuMUUx;pm`g-&6l!>qOi6n-Bkfw>~cKe`#QQUG9soe!TeRN1MmxdHb2o(3fl&?+?e- zt=9JO0d(!Wy}sJ#kMY(GHZSz=-F|sxtHEr-O4#bd{hfg0LVq9hdN&Ooy}Z6%Z`TeC zb_ljTYt9e%79RO{UZv;Ggii-%KY4k--`%bJ%cnUIy8U5(ACcY9Y~LJK%l*~OGp+WY zZMitx@T2|u-@b}5E?&!!-ZTT~->_xWHu1y2?LJp4(G)+<)OcARVC2i~)@<40&hDud z`}6uBn&P%ho{*V8p;mu-?bC$VYdo$t7hju^_y`_8jp5nZ3Vk$T9kTxPg?GFC=K7s% zwe4S9_coV%rzTgw#_?u%|F*RuT;1>Y8m@@a{T-hMVsmcu=LT2&)mM)6zJDOEmM`*G z;*&)LMU^6@)>Pg`vL1_-QMDmS>Y>k<2~d%{5_2mus3JpDqw2>W!J9T`Gj${wgKMnD zo-^#^>)Ck%l!WSiB0!Ghs8a^aoCI`3#Rn;pV8c9@AaDZ(%cEN5P$gVAU@AvV&Ow4Q zr$v42q^3} zP?n%*$w0wzmVhNCaHJ9=u|yj1H03VeHe5D=rB0n%0~T{?g?F5tyMhw>s+3_$U5pwq zRfE{iXsKAhQn3Oc5L4}L(&C&TiWABz717&}M%SdiPAK>(XhJFgDVkD=ZW6O0rop0c z=}dPjD~OO5!8<8I64IOs!9mSz3*t^`VTpG!B4f6PBsn?Bn^3QKQuUCkjbV4eJ08<%LP$!@7!-KiwY)<|TOC^MDQKsPN&ZLHF^DzVdR@O>+tbVyQ)y#&>8 zqpJ5EX|+H2IK@b62KM$uBSl)gk^`sCt8Duqqi-R7WTxs8sE-nYn)L#kXSVNSHZW^n z0Y#rsx&h%oLKsELg^mKjKoZ&u+_g)V4bDv%Y$B;ePcT4wKw-H@Vl9w<-+-7MiH#+a z6&52@pf@5qKhzr$*IPtb^Ek<`Lp^76Y9nP&RS*Q9Mu6^#+zLC`1Pba`mV`$K=&MRm zjtCea@17z=VNyMYs+6{RA0P_$|7Ja6gL>~nj%F*_~ zr(r_8G>2izy)=B=%&Ciz{74hSq-})uhcnU50TE#0ZW69+C~BP z*`hbhGMI*N7f#jY8d@>dZmY@`=dCJ59J*%6AI=99+bbkFsTfovY z2UyvpzD^=KRNqfWjOBCXG<9&^3T8p!y^<{%(F{^y?Rs$iBpz+J1iVrNL=DSQ*~Nx` z^QMY=0zfJ8v}Mu z*jktob)wFFB|v)~E884qm*_OkL!B^OP1TH9r_|L_W3|{~#Js4IG}ulz2^vhOI`I%7 zU{M+!GwK7F`5yr{XwP0G&qOn6zexb#7vuZfOLes5y4uCD8^6RvDGULFMHdywl-h9v`x;7Wc%>nmk?%#?+~KOC{f1LqWE}|dtl}`I401rW z(>beQhI&3Nbn5!)+}WA14K1npgbt+1R|B>U%)<0yRz+Igq#z`lMSZ0qdp`kSA}4hp zlk;tkOH53NoD~cq6N_|;$lKClsGV?>n`ZMR@%HAFMl>7Q0yKb8svorixICJdH9*uT z|1WePuffw?L)WD3B&)lu1=ALOR7TD;K!Aas`(GL$S6NxzH!g? z?I)IM7yU0ZCj&!(T_*vY`B5hg*;=_fW0zU?M;XV`C54-llywL}Or-86%vvb!qGMl9 zU?*(gGp`;vihR_|LmSvEq)g5yAH+)~opb2(TWw8LB{V5voTX<*tuSeHRS_%^OF4%6 zHIo3?#KU6~AA6E&|3EMgh{UA_P{|yryVp(+aCT=2I?`#QUVCW18dRmlehi8A#zYHG zlNAQY+p00D({&iUM_ZvPDwL%-cWMZ34HTDAGk=)OO@nB{oc~y;NIQyFd4`BIi(tKD z0&ANzOC1~Qh)Kv0D2CXcHXBVfNnd)zgk5HDWb3b0J!GTGoJ|&GE(S00Q8=Dm{+vPp zVdBXdwe>(!(2{U3T~AN5_2Z1hGf|U85-&k&6|^J`t^yuW*@YA*%^@m?CUM)9wAO7{YZ~VmHoWAz#%**-|;rM`Pe0GLvZWb?g z0x&*XJYQekERX;3Pyg~_v((j}ulC#J{`$!UAAaK3`0(p$yZGH|rz>ClxVybuU+?by zG8)X&fjnO=@BiK&=E#_-YT2%kr=m7Wezz&w@Yh7iLuIK)t&-t>Y7H^=lD0@9r7~`iGYb9)Gv8jdQ~3+c>^k?tfa@{l2=lt+jtju>0_f=gaL+ z+FuV}Idks56WDq8Zrl0}1Yfp!y7Fu5^H)BBU)}B38_nMI1jxSn#kSjF{j(UGbwAst zYumS1-ZF5s>k2F{DE{A`V79(pA6CoF`tI<^#;sSme7j$@egaL@8`}okYkkeHe)V+n zpIr6TYUS@y|Ni~%*hc-w)tlAc0pj|v7q*5UR^Y4M{r>d#-`?*}Lt4uAI8XR5KK+}; zX1)E%zx3CjXce^19ru2#mPuLyRsIy9b>eXTNff9K@^%)9^a2v@F3uL%bc{uQrl}K{ zWDLiMk`vOgX4!Vh)q=YsghH1x99j#KxkPj{swz%1Qq32w*hgkATovL$1eyz-z?i8H8CS~$bxvqh{3 z6f&kd^fV65vl0joR+S6ClG|f*K6iShr(nuf4{LLd(|`m+j`sRYJVh<(gZGzNEJv03 zF!nWQkHDngOKr~L$creACGac-6j=gL*fDLpcIto?9(+!!c2Wp3_)6&?WL9brHi=Ei zR7nt4Dw`cR?mY3K>jg}giX(IVjFgCIwQzN%YThVGsi^V0V{;i!jE#`;$s&eDvcFn8 z6?mU@0qiM9GKs*4iriH(HzmO&2U9WnQHwrU*D_)LO+}~DiQ~0kX&x{pQrI(gJ<`4n zP;g!Sux9*_DA&yNOyP=JEhex^7FtP)2G@w~bEJ3?!Rpkd7~Ag5fyn5x60!*Gw`jZc zB!K6*dP2fIv%cBxPmD3lNG?(N5j#fHWNW26B;#d?&f&=;2H7aX%;B~ZIqJa(~vZ=7?0>c6snGSx$M^JoB=qI?UdM&ODC<5h@$N{U^kJH1=l&-v(J7Hb-Mw(wK_XXy zdG!i4!=*1fF9=VQMOQ=|yR`uq`&*36VK9lCb)s&Sb?Ej#|r6O=7xfh8W&&01Jec!2LC2O}2LDB*JfW>t} zp;ZaWD}!+~U#{2Q^EsAl6G>?xs7MRvg?$?sUYt3)*SK?6)@HEcc_B)oZhBk7b*zZe z80QKK4}&nO&S76iz4ghLG1P8V>445-Nyn*%&NBgw;MLbqP}dVfM|O5mK@#&Vdb^aI#}> zqy|Hk?7?PDDtdI(%FU;c^+oo|gvX2eYciYuB&xrzU(i>BT9FIYd{n>bbcftM3(|rD z7?lzkp+ji<9#JBJ1}1ILfGH=7XTuyRJ~dyOsxeuT1*q<0PhmI5=vWQ61OpeOd_sMK!-Z|NY1 zG20qJ(X2_0k8s*2_0l1%OuWL>W^D6qdl8K9WWGtphDhOp5@NMPt7+#Oh4MmqqenH0 zKaH%cYJ;gm<@+*e&I6{@14;NV#5^{Ql13A`nG&KU-!`u-i)v z3D~V2lcW2oDmlpx0scr1O6^j5z8=D8?}|)~8Xl(HvE&a9&aoAbDID{>vG=jaoKrX? zeKW*in0ID#iM=4Ud4#BhyVw*5N}0%$Cz&fH87gIE#6d^yVe=cYFR@Cz9U)fY4;Wyh zm;)&7dQ{|AGX$(VQ7vNUjQRI-*AvxVW2n+|fs~DAizc|ntkf`aRP=HV9;(!wk;Psz zHVNCp)%Y&`K3LyVa5=)>fw5EE4qq_Yb|eju>C}enyJUUSNl=-N81d$f9o4q)U^S1x z0cUZmZBx`?8bzr`bnR0%50MhXS^jsY(&zg`k_wc}wU7i=phhLckZW4XACyKP2ez~l ztF9<`q$UxnKdI-x1Dg9A6r_=0_9H|JZDHF79n8I@iyhOA9N%UhX}_BjF9#KVF+1@m zSyb}^p(jh8BjnCv(R%NxXe47v-*@*D=1;l)KEco&?`V%k^PwK{&H?q6GbhwwQOJP; zQ07O5fmOwYn?jCZ^YX&fIYDg(1NRe~6V%mg+D(eoVE`6svD=KrOS7;Uz05M&M#rqM zq&=wV-MoX($P%hO5F3z%(4t5sd%O<8f=;Oc4{(uH%w#U3>&a8+@SsW(d0Rp91B?M5 zMg4)kMNw9bNP%r+U3%ZwW0l+tEFuLyT?wH*WSB}vEAI!TCn5VsRB{{PY>hqsQ=?qH z!K&{7@|P7=45cjKRK>>5^%mi$F_~(sh@xOo#3)oEXFy&GI0M2D2ciQAcII~+pnjI0D zm~yXo>SvOukcqxPXAi4r>id~Ir!2T8Sm&fI1OPxWH`|jCAJ@QgSJ9KjkV+}@Jays{ zEupC1Vu;yo!=;MgfWc`M2I?-6*49Y60@5fz?pa${>O_Zux$_Ne2|<>AhQeT(EU=|b z9Ru`->jxGog->y!{`Yx8?0YCRGo-f!uYLBXk7`_qvX0`#p!)b&C;3REWu}S_l`17E zLcfp~8xfprzOqUbpMFv=gAmA!GFL__baH&Y7j*m)=0!FzB9r)}MCbt3n>gEak%qAy r{Rwrz#1=@TFKV8td7Pg7{Rw{dZ=U`||Mic5^N;@vP + { + "service": "test_service", + "service_settings": { + "model": "my_model", + "api_key": "abc64" + }, + "task_settings": { + } + } + + - do: + inference.put: + task_type: text_embedding + inference_id: dense-inference-id + body: > + { + "service": "text_embedding_test_service", + "service_settings": { + "model": "my_model", + "dimensions": 10, + "api_key": "abc64", + "similarity": "COSINE" + }, + "task_settings": { + } + } + + - do: + indices.create: + index: test-sparse-index + body: + mappings: + properties: + body: + type: semantic_text + inference_id: sparse-inference-id + + - do: + indices.create: + index: test-dense-index + body: + mappings: + properties: + body: + type: semantic_text + inference_id: dense-inference-id + +--- +"Highlighting using a sparse embedding model": + - do: + index: + index: test-sparse-index + id: doc_1 + body: + body: ["ElasticSearch is an open source, distributed, RESTful, search engine which is built on top of Lucene internally and enjoys all the features it provides.", "You Know, for Search!"] + refresh: true + + - match: { result: created } + + - do: + search: + index: test-sparse-index + body: + query: + semantic: + field: "body" + query: "What is Elasticsearch?" + highlight: + fields: + body: + type: "semantic" + number_of_fragments: 1 + + - match: { hits.total.value: 1 } + - match: { hits.hits.0._id: "doc_1" } + - length: { hits.hits.0.highlight.body: 1 } + - match: { hits.hits.0.highlight.body.0: "ElasticSearch is an open source, distributed, RESTful, search engine which is built on top of Lucene internally and enjoys all the features it provides." } + + - do: + search: + index: test-sparse-index + body: + query: + semantic: + field: "body" + query: "What is Elasticsearch?" + highlight: + fields: + body: + type: "semantic" + number_of_fragments: 2 + + - match: { hits.total.value: 1 } + - match: { hits.hits.0._id: "doc_1" } + - length: { hits.hits.0.highlight.body: 2 } + - match: { hits.hits.0.highlight.body.0: "ElasticSearch is an open source, distributed, RESTful, search engine which is built on top of Lucene internally and enjoys all the features it provides." } + - match: { hits.hits.0.highlight.body.1: "You Know, for Search!" } + + - do: + search: + index: test-sparse-index + body: + query: + semantic: + field: "body" + query: "What is Elasticsearch?" + highlight: + fields: + body: + type: "semantic" + order: "score" + number_of_fragments: 1 + + - match: { hits.total.value: 1 } + - match: { hits.hits.0._id: "doc_1" } + - length: { hits.hits.0.highlight.body: 1 } + - match: { hits.hits.0.highlight.body.0: "ElasticSearch is an open source, distributed, RESTful, search engine which is built on top of Lucene internally and enjoys all the features it provides." } + + - do: + search: + index: test-sparse-index + body: + query: + semantic: + field: "body" + query: "What is Elasticsearch?" + highlight: + fields: + body: + type: "semantic" + order: "score" + number_of_fragments: 2 + + - match: { hits.total.value: 1 } + - match: { hits.hits.0._id: "doc_1" } + - length: { hits.hits.0.highlight.body: 2 } + - match: { hits.hits.0.highlight.body.0: "ElasticSearch is an open source, distributed, RESTful, search engine which is built on top of Lucene internally and enjoys all the features it provides." } + - match: { hits.hits.0.highlight.body.1: "You Know, for Search!" } + +--- +"Highlighting using a dense embedding model": + - do: + index: + index: test-dense-index + id: doc_1 + body: + body: ["ElasticSearch is an open source, distributed, RESTful, search engine which is built on top of Lucene internally and enjoys all the features it provides.", "You Know, for Search!"] + refresh: true + + - match: { result: created } + + - do: + search: + index: test-dense-index + body: + query: + semantic: + field: "body" + query: "What is Elasticsearch?" + highlight: + fields: + body: + type: "semantic" + number_of_fragments: 1 + + - match: { hits.total.value: 1 } + - match: { hits.hits.0._id: "doc_1" } + - length: { hits.hits.0.highlight.body: 1 } + - match: { hits.hits.0.highlight.body.0: "You Know, for Search!" } + + - do: + search: + index: test-dense-index + body: + query: + semantic: + field: "body" + query: "What is Elasticsearch?" + highlight: + fields: + body: + type: "semantic" + number_of_fragments: 2 + + - match: { hits.total.value: 1 } + - match: { hits.hits.0._id: "doc_1" } + - length: { hits.hits.0.highlight.body: 2 } + - match: { hits.hits.0.highlight.body.0: "ElasticSearch is an open source, distributed, RESTful, search engine which is built on top of Lucene internally and enjoys all the features it provides." } + - match: { hits.hits.0.highlight.body.1: "You Know, for Search!" } + + - do: + search: + index: test-dense-index + body: + query: + semantic: + field: "body" + query: "What is Elasticsearch?" + highlight: + fields: + body: + type: "semantic" + order: "score" + number_of_fragments: 1 + + - match: { hits.total.value: 1 } + - match: { hits.hits.0._id: "doc_1" } + - length: { hits.hits.0.highlight.body: 1 } + - match: { hits.hits.0.highlight.body.0: "You Know, for Search!" } + + - do: + search: + index: test-dense-index + body: + query: + semantic: + field: "body" + query: "What is Elasticsearch?" + highlight: + fields: + body: + type: "semantic" + order: "score" + number_of_fragments: 2 + + - match: { hits.total.value: 1 } + - match: { hits.hits.0._id: "doc_1" } + - length: { hits.hits.0.highlight.body: 2 } + - match: { hits.hits.0.highlight.body.0: "You Know, for Search!" } + - match: { hits.hits.0.highlight.body.1: "ElasticSearch is an open source, distributed, RESTful, search engine which is built on top of Lucene internally and enjoys all the features it provides." } + + From a04d67180c552a549fbac644bacd209b4e0370a4 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 6 Dec 2024 12:01:36 -0800 Subject: [PATCH 08/27] Allow early termination of exchange source (#118129) This change introduces the ability to gracefully terminate the exchange source early by instructing all remote exchange sinks to stop their computations. 1. When sufficient data has been accumulated (e.g., reaching the LIMIT), the exchange source signals remote sinks to stop generating new pages, allowing the query to finish sooner. 2. When users request immediate results, even if they are partial, incomplete, or potentially inaccurate. --- .../operator/exchange/ExchangeService.java | 31 ++++++++--- .../exchange/ExchangeSourceHandler.java | 32 ++++++++++- .../compute/operator/exchange/RemoteSink.java | 9 +-- .../exchange/ExchangeServiceTests.java | 55 ++++++++++++++++++- 4 files changed, 111 insertions(+), 16 deletions(-) diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeService.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeService.java index 00c68c4f48e8..62cc4daf5fde 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeService.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeService.java @@ -14,6 +14,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.action.support.ChannelActionListener; +import org.elasticsearch.action.support.SubscribableListener; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -23,6 +24,7 @@ import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.compute.data.BlockFactory; import org.elasticsearch.compute.data.BlockStreamInput; +import org.elasticsearch.compute.data.Page; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.TimeValue; import org.elasticsearch.tasks.CancellableTask; @@ -40,10 +42,11 @@ import java.io.IOException; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.concurrent.Executor; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; /** * {@link ExchangeService} is responsible for exchanging pages between exchange sinks and sources on the same or different nodes. @@ -293,7 +296,7 @@ static final class TransportRemoteSink implements RemoteSink { final Executor responseExecutor; final AtomicLong estimatedPageSizeInBytes = new AtomicLong(0L); - final AtomicBoolean finished = new AtomicBoolean(false); + final AtomicReference> completionListenerRef = new AtomicReference<>(null); TransportRemoteSink( TransportService transportService, @@ -318,13 +321,14 @@ public void fetchPageAsync(boolean allSourcesFinished, ActionListener completionListener = completionListenerRef.get(); + if (completionListener != null) { + completionListener.addListener(listener.map(unused -> new ExchangeResponse(blockFactory, null, true))); return; } doFetchPageAsync(false, ActionListener.wrap(r -> { if (r.finished()) { - finished.set(true); + completionListenerRef.compareAndSet(null, SubscribableListener.newSucceeded(null)); } listener.onResponse(r); }, e -> close(ActionListener.running(() -> listener.onFailure(e))))); @@ -356,10 +360,19 @@ private void doFetchPageAsync(boolean allSourcesFinished, ActionListener listener) { - if (finished.compareAndSet(false, true)) { - doFetchPageAsync(true, listener.delegateFailure((l, unused) -> l.onResponse(null))); - } else { - listener.onResponse(null); + final SubscribableListener candidate = new SubscribableListener<>(); + final SubscribableListener actual = completionListenerRef.updateAndGet( + curr -> Objects.requireNonNullElse(curr, candidate) + ); + actual.addListener(listener); + if (candidate == actual) { + doFetchPageAsync(true, ActionListener.wrap(r -> { + final Page page = r.takePage(); + if (page != null) { + page.releaseBlocks(); + } + candidate.onResponse(null); + }, e -> candidate.onResponse(null))); } } } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceHandler.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceHandler.java index b53ddea3da58..aa722695b841 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceHandler.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/ExchangeSourceHandler.java @@ -12,6 +12,7 @@ import org.elasticsearch.action.support.RefCountingRunnable; import org.elasticsearch.action.support.SubscribableListener; import org.elasticsearch.common.util.concurrent.AbstractRunnable; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.compute.EsqlRefCountingListener; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.FailureCollector; @@ -19,6 +20,7 @@ import org.elasticsearch.core.Releasable; import java.util.List; +import java.util.Map; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicInteger; @@ -41,6 +43,9 @@ public final class ExchangeSourceHandler { // The final failure collected will be notified to callers via the {@code completionListener}. private final FailureCollector failure = new FailureCollector(); + private final AtomicInteger nextSinkId = new AtomicInteger(); + private final Map remoteSinks = ConcurrentCollections.newConcurrentMap(); + /** * Creates a new ExchangeSourceHandler. * @@ -53,7 +58,9 @@ public ExchangeSourceHandler(int maxBufferSize, Executor fetchExecutor, ActionLi this.buffer = new ExchangeBuffer(maxBufferSize); this.fetchExecutor = fetchExecutor; this.outstandingSinks = new PendingInstances(() -> buffer.finish(false)); - this.outstandingSources = new PendingInstances(() -> buffer.finish(true)); + final PendingInstances closingSinks = new PendingInstances(() -> {}); + closingSinks.trackNewInstance(); + this.outstandingSources = new PendingInstances(() -> finishEarly(true, ActionListener.running(closingSinks::finishInstance))); buffer.addCompletionListener(ActionListener.running(() -> { final ActionListener listener = ActionListener.assertAtLeastOnce(completionListener); try (RefCountingRunnable refs = new RefCountingRunnable(() -> { @@ -64,6 +71,7 @@ public ExchangeSourceHandler(int maxBufferSize, Executor fetchExecutor, ActionLi listener.onResponse(null); } })) { + closingSinks.completion.addListener(refs.acquireListener()); for (PendingInstances pending : List.of(outstandingSinks, outstandingSources)) { // Create an outstanding instance and then finish to complete the completionListener // if we haven't registered any instances of exchange sinks or exchange sources before. @@ -257,7 +265,11 @@ void onSinkComplete() { * @see ExchangeSinkHandler#fetchPageAsync(boolean, ActionListener) */ public void addRemoteSink(RemoteSink remoteSink, boolean failFast, int instances, ActionListener listener) { - final ActionListener sinkListener = ActionListener.assertAtLeastOnce(ActionListener.notifyOnce(listener)); + final int sinkId = nextSinkId.incrementAndGet(); + remoteSinks.put(sinkId, remoteSink); + final ActionListener sinkListener = ActionListener.assertAtLeastOnce( + ActionListener.notifyOnce(ActionListener.runBefore(listener, () -> remoteSinks.remove(sinkId))) + ); fetchExecutor.execute(new AbstractRunnable() { @Override public void onFailure(Exception e) { @@ -291,6 +303,22 @@ public Releasable addEmptySink() { return outstandingSinks::finishInstance; } + /** + * Gracefully terminates the exchange source early by instructing all remote exchange sinks to stop their computations. + * This can happen when the exchange source has accumulated enough data (e.g., reaching the LIMIT) or when users want to + * see the current result immediately. + * + * @param drainingPages whether to discard pages already fetched in the exchange + */ + public void finishEarly(boolean drainingPages, ActionListener listener) { + buffer.finish(drainingPages); + try (EsqlRefCountingListener refs = new EsqlRefCountingListener(listener)) { + for (RemoteSink remoteSink : remoteSinks.values()) { + remoteSink.close(refs.acquire()); + } + } + } + private static class PendingInstances { private final AtomicInteger instances = new AtomicInteger(); private final SubscribableListener completion = new SubscribableListener<>(); diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/RemoteSink.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/RemoteSink.java index aaa937ef17c0..63b5d324ce85 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/RemoteSink.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/exchange/RemoteSink.java @@ -8,6 +8,7 @@ package org.elasticsearch.compute.operator.exchange; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.compute.data.Page; public interface RemoteSink { @@ -15,11 +16,11 @@ public interface RemoteSink { default void close(ActionListener listener) { fetchPageAsync(true, listener.delegateFailure((l, r) -> { - try { - r.close(); - } finally { - l.onResponse(null); + final Page page = r.takePage(); + if (page != null) { + page.releaseBlocks(); } + l.onResponse(null); })); } } diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/exchange/ExchangeServiceTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/exchange/ExchangeServiceTests.java index fc6c850ba187..8f7532b582bc 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/exchange/ExchangeServiceTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/exchange/ExchangeServiceTests.java @@ -55,7 +55,9 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Queue; import java.util.Set; +import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -421,7 +423,7 @@ public void testExchangeSourceContinueOnFailure() { } } - public void testEarlyTerminate() { + public void testClosingSinks() { BlockFactory blockFactory = blockFactory(); IntBlock block1 = blockFactory.newConstantIntBlockWith(1, 2); IntBlock block2 = blockFactory.newConstantIntBlockWith(1, 2); @@ -441,6 +443,57 @@ public void testEarlyTerminate() { assertTrue(sink.isFinished()); } + public void testFinishEarly() throws Exception { + ExchangeSourceHandler sourceHandler = new ExchangeSourceHandler(20, threadPool.generic(), ActionListener.noop()); + Semaphore permits = new Semaphore(between(1, 5)); + BlockFactory blockFactory = blockFactory(); + Queue pages = ConcurrentCollections.newQueue(); + ExchangeSource exchangeSource = sourceHandler.createExchangeSource(); + AtomicBoolean sinkClosed = new AtomicBoolean(); + PlainActionFuture sinkCompleted = new PlainActionFuture<>(); + sourceHandler.addRemoteSink((allSourcesFinished, listener) -> { + if (allSourcesFinished) { + sinkClosed.set(true); + permits.release(10); + listener.onResponse(new ExchangeResponse(blockFactory, null, sinkClosed.get())); + } else { + try { + if (permits.tryAcquire(between(0, 100), TimeUnit.MICROSECONDS)) { + boolean closed = sinkClosed.get(); + final Page page; + if (closed) { + page = new Page(blockFactory.newConstantIntBlockWith(1, 1)); + pages.add(page); + } else { + page = null; + } + listener.onResponse(new ExchangeResponse(blockFactory, page, closed)); + } else { + listener.onResponse(new ExchangeResponse(blockFactory, null, sinkClosed.get())); + } + } catch (Exception e) { + throw new AssertionError(e); + } + } + }, false, between(1, 3), sinkCompleted); + threadPool.schedule( + () -> sourceHandler.finishEarly(randomBoolean(), ActionListener.noop()), + TimeValue.timeValueMillis(between(0, 10)), + threadPool.generic() + ); + sinkCompleted.actionGet(); + Page p; + while ((p = exchangeSource.pollPage()) != null) { + assertSame(p, pages.poll()); + p.releaseBlocks(); + } + while ((p = pages.poll()) != null) { + p.releaseBlocks(); + } + assertTrue(exchangeSource.isFinished()); + exchangeSource.finish(); + } + public void testConcurrentWithTransportActions() { MockTransportService node0 = newTransportService(); ExchangeService exchange0 = new ExchangeService(Settings.EMPTY, threadPool, ESQL_TEST_EXECUTOR, blockFactory()); From 11061713c51552fa04c16bdc74785454350159a4 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 6 Dec 2024 21:17:50 +0100 Subject: [PATCH 09/27] Unmute #108628 (#118152) This should be fixed for a while now. Closes #116249 --- muted-tests.yml | 3 --- 1 file changed, 3 deletions(-) diff --git a/muted-tests.yml b/muted-tests.yml index 887b462fa122..07072e9743c9 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -117,9 +117,6 @@ tests: - class: org.elasticsearch.xpack.deprecation.DeprecationHttpIT method: testDeprecatedSettingsReturnWarnings issue: https://github.com/elastic/elasticsearch/issues/108628 -- class: org.elasticsearch.action.search.SearchQueryThenFetchAsyncActionTests - method: testBottomFieldSort - issue: https://github.com/elastic/elasticsearch/issues/116249 - class: org.elasticsearch.xpack.shutdown.NodeShutdownIT method: testAllocationPreventedForRemoval issue: https://github.com/elastic/elasticsearch/issues/116363 From 2619149bc677fec18376fb36c5a54aa3d260431d Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Fri, 6 Dec 2024 21:41:27 +0100 Subject: [PATCH 10/27] [TEST] Work around race condition when starting clusters in parallel (#118145) With #117820 we are starting clusters in parallel in CCS related tests. There is a race condition when calling RandomizedTest#isNightly, which is called as part of `InternalTestCluster#beforeTest`. See https://github.com/randomizedtesting/randomizedtesting/issues/311. Until this is fixed upstream, we can simply call isNightly before forking, which is going to trigger the lazy initialization of the inner map that causes the race. From then on, all threads will have a consistent view of it. Closes #118124 --- muted-tests.yml | 3 --- .../elasticsearch/test/AbstractMultiClustersTestCase.java | 7 +++++++ 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/muted-tests.yml b/muted-tests.yml index 07072e9743c9..a39265756599 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -239,9 +239,6 @@ tests: - class: org.elasticsearch.packaging.test.ConfigurationTests method: test30SymlinkedDataPath issue: https://github.com/elastic/elasticsearch/issues/118111 -- class: org.elasticsearch.datastreams.ResolveClusterDataStreamIT - method: testClusterResolveWithDataStreamsUsingAlias - issue: https://github.com/elastic/elasticsearch/issues/118124 - class: org.elasticsearch.packaging.test.KeystoreManagementTests method: test30KeystorePasswordFromFile issue: https://github.com/elastic/elasticsearch/issues/118123 diff --git a/test/framework/src/main/java/org/elasticsearch/test/AbstractMultiClustersTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/AbstractMultiClustersTestCase.java index b4f91f68b8bb..7cd7bce4db18 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/AbstractMultiClustersTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/AbstractMultiClustersTestCase.java @@ -9,6 +9,8 @@ package org.elasticsearch.test; +import com.carrotsearch.randomizedtesting.RandomizedTest; + import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.admin.cluster.remote.RemoteInfoRequest; @@ -108,6 +110,11 @@ public final void startClusters() throws Exception { MockTransportService.TestPlugin.class, getTestTransportPlugin() ); + // We are going to initialize multiple clusters concurrently, but there is a race condition around the lazy initialization of test + // groups in GroupEvaluator across multiple threads. See https://github.com/randomizedtesting/randomizedtesting/issues/311. + // Calling isNightly before parallelizing is enough to work around that issue. + @SuppressWarnings("unused") + boolean nightly = RandomizedTest.isNightly(); runInParallel(clusterAliases.size(), i -> { String clusterAlias = clusterAliases.get(i); final String clusterName = clusterAlias.equals(LOCAL_CLUSTER) ? "main-cluster" : clusterAlias; From 467fdb879c6ed1ea084d02087a4afaa12babe7c6 Mon Sep 17 00:00:00 2001 From: Max Hniebergall <137079448+maxhniebergall@users.noreply.github.com> Date: Fri, 6 Dec 2024 15:52:31 -0500 Subject: [PATCH 11/27] [Inference API] Add unified api for chat completions (#117589) * Adding some shell classes * modeling the request objects * Writeable changes to schema * Working parsing tests * Creating a new action * Add outbound request writing (WIP) * Improvements to request serialization * Adding separate transport classes * separate out unified request and combine inputs * Reworking unified inputs * Adding unsupported operation calls * Fixing parsing logic * get the build working * Update docs/changelog/117589.yaml * Fixing injection issue * Allowing model to be overridden but not working yet * Fixing issues * Switch field name for tool * Add suport for toolCalls and refusal in streaming completion * Working tool call response * Separate unified and legacy code paths * Updated the parser, but there are some class cast exceptions to fix * Refactoring tests and request entities * Parse response from OpenAI * Removing unused request classes * precommit * Adding tests for UnifiedCompletionAction Request * Refactoring stop to be a list of strings * Testing for OpenAI response parsing * Refactoring transport action tests to test unified validation code * Fixing various tests * Fixing license header * Reformat streaming results * Finalize response format * remove debug logs * remove changes for debugging * Task type and base inference action tests * Adding openai service tests * Adding model tests * tests for StreamingUnifiedChatCompletionResultsTests toXContentChunked * Fixing change log and removing commented out code * Switch usage to accept null * Adding test for TestStreamingCompletionServiceExtension * Avoid serializing empty lists + request entity tests * Register named writeables from UnifiedCompletionRequest * Removing commented code * Clean up and add more of an explination * remove duplicate test * remove old todos * Refactoring some duplication * Adding javadoc * Addressing feedback --------- Co-authored-by: Jonathan Buttner Co-authored-by: Jonathan Buttner <56361221+jonathan-buttner@users.noreply.github.com> --- docs/changelog/117589.yaml | 5 + .../xcontent/ChunkedToXContentHelper.java | 9 + .../inference/InferenceService.java | 17 + .../org/elasticsearch/inference/TaskType.java | 4 + .../inference/UnifiedCompletionRequest.java | 425 +++++++++ .../org/elasticsearch/test/ESTestCase.java | 22 +- .../action/BaseInferenceActionRequest.java | 31 + .../inference/action/InferenceAction.java | 3 +- .../action/UnifiedCompletionAction.java | 129 +++ ...StreamingUnifiedChatCompletionResults.java | 329 +++++++ .../action/InferenceActionRequestTests.java | 3 +- .../UnifiedCompletionActionRequestTests.java | 97 ++ .../action/UnifiedCompletionRequestTests.java | 293 ++++++ ...mingUnifiedChatCompletionResultsTests.java | 198 ++++ .../authz/store/ReservedRolesStoreTests.java | 1 + .../inference/InferenceBaseRestTest.java | 32 +- .../xpack/inference/InferenceCrudIT.java | 55 ++ .../TestDenseInferenceServiceExtension.java | 11 + .../mock/TestRerankingServiceExtension.java | 11 + .../TestSparseInferenceServiceExtension.java | 11 + ...stStreamingCompletionServiceExtension.java | 73 ++ .../InferenceNamedWriteablesProvider.java | 8 + .../xpack/inference/InferencePlugin.java | 20 +- .../inference/UnifiedCompletionFeature.java | 20 + .../action/BaseTransportInferenceAction.java | 250 +++++ .../action/TransportInferenceAction.java | 219 +---- ...sportUnifiedCompletionInferenceAction.java | 77 ++ .../inference/common/DelegatingProcessor.java | 45 +- .../SingleInputSenderExecutableAction.java | 9 +- .../action/openai/OpenAiActionCreator.java | 2 +- ...baCloudSearchCompletionRequestManager.java | 2 +- ...onBedrockChatCompletionRequestManager.java | 8 +- .../AnthropicCompletionRequestManager.java | 8 +- ...eAiStudioChatCompletionRequestManager.java | 8 +- .../AzureOpenAiCompletionRequestManager.java | 8 +- .../http/sender/ChatCompletionInput.java | 39 + .../CohereCompletionRequestManager.java | 8 +- .../http/sender/DocumentsOnlyInput.java | 10 +- ...oogleAiStudioCompletionRequestManager.java | 5 +- .../external/http/sender/InferenceInputs.java | 26 +- .../OpenAiCompletionRequestManager.java | 12 +- ...OpenAiUnifiedCompletionRequestManager.java | 61 ++ .../http/sender/QueryAndDocsInputs.java | 11 +- .../http/sender/UnifiedChatInput.java | 62 ++ .../openai/OpenAiStreamingProcessor.java | 18 +- ...iUnifiedChatCompletionResponseHandler.java | 34 + .../OpenAiUnifiedStreamingProcessor.java | 287 ++++++ .../GoogleAiStudioCompletionRequest.java | 6 +- .../OpenAiChatCompletionRequestEntity.java | 79 -- ...> OpenAiUnifiedChatCompletionRequest.java} | 20 +- ...nAiUnifiedChatCompletionRequestEntity.java | 185 ++++ .../inference/rest/BaseInferenceAction.java | 32 +- .../xpack/inference/rest/Paths.java | 6 + .../RestUnifiedCompletionInferenceAction.java | 49 + .../inference/services/SenderService.java | 43 +- .../inference/services/ServiceUtils.java | 4 + .../AlibabaCloudSearchService.java | 14 +- .../amazonbedrock/AmazonBedrockService.java | 12 + .../services/anthropic/AnthropicService.java | 12 + .../azureaistudio/AzureAiStudioService.java | 12 + .../azureopenai/AzureOpenAiService.java | 12 + .../services/cohere/CohereService.java | 12 + .../elastic/ElasticInferenceService.java | 12 + .../ElasticsearchInternalService.java | 12 + .../googleaistudio/GoogleAiStudioService.java | 15 +- .../googlevertexai/GoogleVertexAiService.java | 12 + .../huggingface/HuggingFaceService.java | 13 + .../elser/HuggingFaceElserService.java | 12 + .../ibmwatsonx/IbmWatsonxService.java | 12 + .../services/mistral/MistralService.java | 12 + .../services/openai/OpenAiService.java | 27 + .../completion/OpenAiChatCompletionModel.java | 22 + ...enAiChatCompletionRequestTaskSettings.java | 1 - .../xpack/inference/TaskTypeTests.java | 27 + .../elasticsearch/xpack/inference/Utils.java | 3 + .../BaseTransportInferenceActionTestCase.java | 364 ++++++++ .../action/TransportInferenceActionTests.java | 337 +------ ...TransportUnifiedCompletionActionTests.java | 124 +++ ...ingleInputSenderExecutableActionTests.java | 20 +- .../AmazonBedrockActionCreatorTests.java | 5 +- .../AnthropicActionCreatorTests.java | 7 +- .../AnthropicChatCompletionActionTests.java | 12 +- .../AzureAiStudioActionAndCreatorTests.java | 3 +- .../AzureOpenAiActionCreatorTests.java | 7 +- .../AzureOpenAiCompletionActionTests.java | 10 +- .../cohere/CohereActionCreatorTests.java | 5 +- .../cohere/CohereCompletionActionTests.java | 18 +- .../GoogleAiStudioCompletionActionTests.java | 12 +- .../openai/OpenAiActionCreatorTests.java | 21 +- .../OpenAiChatCompletionActionTests.java | 19 +- .../AmazonBedrockMockRequestSender.java | 12 +- .../AmazonBedrockRequestSenderTests.java | 3 +- .../http/sender/InferenceInputsTests.java | 40 + .../http/sender/UnifiedChatInputTests.java | 46 + .../OpenAiUnifiedStreamingProcessorTests.java | 383 ++++++++ .../GoogleAiStudioCompletionRequestTests.java | 6 +- ...penAiChatCompletionRequestEntityTests.java | 53 -- ...ifiedChatCompletionRequestEntityTests.java | 856 ++++++++++++++++++ ...nAiUnifiedChatCompletionRequestTests.java} | 59 +- .../rest/BaseInferenceActionTests.java | 43 + ...UnifiedCompletionInferenceActionTests.java | 81 ++ .../services/SenderServiceTests.java | 9 + .../services/openai/OpenAiServiceTests.java | 63 ++ .../OpenAiChatCompletionModelTests.java | 42 +- .../xpack/security/operator/Constants.java | 1 + 105 files changed, 5488 insertions(+), 867 deletions(-) create mode 100644 docs/changelog/117589.yaml create mode 100644 server/src/main/java/org/elasticsearch/inference/UnifiedCompletionRequest.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/action/BaseInferenceActionRequest.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/action/UnifiedCompletionAction.java create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/results/StreamingUnifiedChatCompletionResults.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/action/UnifiedCompletionActionRequestTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/action/UnifiedCompletionRequestTests.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/results/StreamingUnifiedChatCompletionResultsTests.java create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/UnifiedCompletionFeature.java create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/BaseTransportInferenceAction.java create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/TransportUnifiedCompletionInferenceAction.java create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/ChatCompletionInput.java create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/OpenAiUnifiedCompletionRequestManager.java create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/UnifiedChatInput.java create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedChatCompletionResponseHandler.java create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedStreamingProcessor.java delete mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiChatCompletionRequestEntity.java rename x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/openai/{OpenAiChatCompletionRequest.java => OpenAiUnifiedChatCompletionRequest.java} (80%) create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiUnifiedChatCompletionRequestEntity.java create mode 100644 x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/RestUnifiedCompletionInferenceAction.java create mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/TaskTypeTests.java create mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/BaseTransportInferenceActionTestCase.java create mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/TransportUnifiedCompletionActionTests.java create mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/http/sender/InferenceInputsTests.java create mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/http/sender/UnifiedChatInputTests.java create mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedStreamingProcessorTests.java delete mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiChatCompletionRequestEntityTests.java create mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiUnifiedChatCompletionRequestEntityTests.java rename x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/openai/{OpenAiChatCompletionRequestTests.java => OpenAiUnifiedChatCompletionRequestTests.java} (75%) create mode 100644 x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/rest/RestUnifiedCompletionInferenceActionTests.java diff --git a/docs/changelog/117589.yaml b/docs/changelog/117589.yaml new file mode 100644 index 000000000000..e6880fd9477b --- /dev/null +++ b/docs/changelog/117589.yaml @@ -0,0 +1,5 @@ +pr: 117589 +summary: "Add Inference Unified API for chat completions for OpenAI" +area: Machine Learning +type: enhancement +issues: [] diff --git a/server/src/main/java/org/elasticsearch/common/xcontent/ChunkedToXContentHelper.java b/server/src/main/java/org/elasticsearch/common/xcontent/ChunkedToXContentHelper.java index 2e78cc6f516b..6a5aa2943de9 100644 --- a/server/src/main/java/org/elasticsearch/common/xcontent/ChunkedToXContentHelper.java +++ b/server/src/main/java/org/elasticsearch/common/xcontent/ChunkedToXContentHelper.java @@ -12,6 +12,7 @@ import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.xcontent.ToXContent; +import java.util.Collections; import java.util.Iterator; public enum ChunkedToXContentHelper { @@ -53,6 +54,14 @@ public static Iterator field(String name, String value) { return Iterators.single(((builder, params) -> builder.field(name, value))); } + public static Iterator optionalField(String name, String value) { + if (value == null) { + return Collections.emptyIterator(); + } else { + return field(name, value); + } + } + /** * Creates an Iterator of a single ToXContent object that serializes the given object as a single chunk. Just wraps {@link * Iterators#single}, but still useful because it avoids any type ambiguity. diff --git a/server/src/main/java/org/elasticsearch/inference/InferenceService.java b/server/src/main/java/org/elasticsearch/inference/InferenceService.java index 4497254aad1f..c2d690d8160a 100644 --- a/server/src/main/java/org/elasticsearch/inference/InferenceService.java +++ b/server/src/main/java/org/elasticsearch/inference/InferenceService.java @@ -112,6 +112,23 @@ void infer( ); /** + * Perform completion inference on the model using the unified schema. + * + * @param model The model + * @param request Parameters for the request + * @param timeout The timeout for the request + * @param listener Inference result listener + */ + void unifiedCompletionInfer( + Model model, + UnifiedCompletionRequest request, + TimeValue timeout, + ActionListener listener + ); + + /** + * Chunk long text. + * * @param model The model * @param query Inference query, mainly for re-ranking * @param input Inference input diff --git a/server/src/main/java/org/elasticsearch/inference/TaskType.java b/server/src/main/java/org/elasticsearch/inference/TaskType.java index b0e5bababbbc..fcb8ea721379 100644 --- a/server/src/main/java/org/elasticsearch/inference/TaskType.java +++ b/server/src/main/java/org/elasticsearch/inference/TaskType.java @@ -38,6 +38,10 @@ public static TaskType fromString(String name) { } public static TaskType fromStringOrStatusException(String name) { + if (name == null) { + throw new ElasticsearchStatusException("Task type must not be null", RestStatus.BAD_REQUEST); + } + try { TaskType taskType = TaskType.fromString(name); return Objects.requireNonNull(taskType); diff --git a/server/src/main/java/org/elasticsearch/inference/UnifiedCompletionRequest.java b/server/src/main/java/org/elasticsearch/inference/UnifiedCompletionRequest.java new file mode 100644 index 000000000000..e596be626b51 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/inference/UnifiedCompletionRequest.java @@ -0,0 +1,425 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.inference; + +import org.elasticsearch.common.io.stream.NamedWriteable; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.xcontent.XContentParserUtils; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.xcontent.ConstructingObjectParser; +import org.elasticsearch.xcontent.ObjectParser; +import org.elasticsearch.xcontent.ParseField; +import org.elasticsearch.xcontent.XContentParseException; +import org.elasticsearch.xcontent.XContentParser; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.xcontent.ConstructingObjectParser.optionalConstructorArg; + +public record UnifiedCompletionRequest( + List messages, + @Nullable String model, + @Nullable Long maxCompletionTokens, + @Nullable List stop, + @Nullable Float temperature, + @Nullable ToolChoice toolChoice, + @Nullable List tools, + @Nullable Float topP +) implements Writeable { + + public sealed interface Content extends NamedWriteable permits ContentObjects, ContentString {} + + @SuppressWarnings("unchecked") + public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + UnifiedCompletionRequest.class.getSimpleName(), + args -> new UnifiedCompletionRequest( + (List) args[0], + (String) args[1], + (Long) args[2], + (List) args[3], + (Float) args[4], + (ToolChoice) args[5], + (List) args[6], + (Float) args[7] + ) + ); + + static { + PARSER.declareObjectArray(constructorArg(), Message.PARSER::apply, new ParseField("messages")); + PARSER.declareString(optionalConstructorArg(), new ParseField("model")); + PARSER.declareLong(optionalConstructorArg(), new ParseField("max_completion_tokens")); + PARSER.declareStringArray(optionalConstructorArg(), new ParseField("stop")); + PARSER.declareFloat(optionalConstructorArg(), new ParseField("temperature")); + PARSER.declareField( + optionalConstructorArg(), + (p, c) -> parseToolChoice(p), + new ParseField("tool_choice"), + ObjectParser.ValueType.OBJECT_OR_STRING + ); + PARSER.declareObjectArray(optionalConstructorArg(), Tool.PARSER::apply, new ParseField("tools")); + PARSER.declareFloat(optionalConstructorArg(), new ParseField("top_p")); + } + + public static List getNamedWriteables() { + return List.of( + new NamedWriteableRegistry.Entry(Content.class, ContentObjects.NAME, ContentObjects::new), + new NamedWriteableRegistry.Entry(Content.class, ContentString.NAME, ContentString::new), + new NamedWriteableRegistry.Entry(ToolChoice.class, ToolChoiceObject.NAME, ToolChoiceObject::new), + new NamedWriteableRegistry.Entry(ToolChoice.class, ToolChoiceString.NAME, ToolChoiceString::new) + ); + } + + public static UnifiedCompletionRequest of(List messages) { + return new UnifiedCompletionRequest(messages, null, null, null, null, null, null, null); + } + + public UnifiedCompletionRequest(StreamInput in) throws IOException { + this( + in.readCollectionAsImmutableList(Message::new), + in.readOptionalString(), + in.readOptionalVLong(), + in.readOptionalStringCollectionAsList(), + in.readOptionalFloat(), + in.readOptionalNamedWriteable(ToolChoice.class), + in.readOptionalCollectionAsList(Tool::new), + in.readOptionalFloat() + ); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeCollection(messages); + out.writeOptionalString(model); + out.writeOptionalVLong(maxCompletionTokens); + out.writeOptionalStringCollection(stop); + out.writeOptionalFloat(temperature); + out.writeOptionalNamedWriteable(toolChoice); + out.writeOptionalCollection(tools); + out.writeOptionalFloat(topP); + } + + public record Message(Content content, String role, @Nullable String name, @Nullable String toolCallId, List toolCalls) + implements + Writeable { + + @SuppressWarnings("unchecked") + static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + Message.class.getSimpleName(), + args -> new Message((Content) args[0], (String) args[1], (String) args[2], (String) args[3], (List) args[4]) + ); + + static { + PARSER.declareField(constructorArg(), (p, c) -> parseContent(p), new ParseField("content"), ObjectParser.ValueType.VALUE_ARRAY); + PARSER.declareString(constructorArg(), new ParseField("role")); + PARSER.declareString(optionalConstructorArg(), new ParseField("name")); + PARSER.declareString(optionalConstructorArg(), new ParseField("tool_call_id")); + PARSER.declareObjectArray(optionalConstructorArg(), ToolCall.PARSER::apply, new ParseField("tool_calls")); + } + + private static Content parseContent(XContentParser parser) throws IOException { + var token = parser.currentToken(); + if (token == XContentParser.Token.START_ARRAY) { + var parsedContentObjects = XContentParserUtils.parseList(parser, (p) -> ContentObject.PARSER.apply(p, null)); + return new ContentObjects(parsedContentObjects); + } else if (token == XContentParser.Token.VALUE_STRING) { + return ContentString.of(parser); + } + + throw new XContentParseException("Expected an array start token or a value string token but found token [" + token + "]"); + } + + public Message(StreamInput in) throws IOException { + this( + in.readNamedWriteable(Content.class), + in.readString(), + in.readOptionalString(), + in.readOptionalString(), + in.readOptionalCollectionAsList(ToolCall::new) + ); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeNamedWriteable(content); + out.writeString(role); + out.writeOptionalString(name); + out.writeOptionalString(toolCallId); + out.writeOptionalCollection(toolCalls); + } + } + + public record ContentObjects(List contentObjects) implements Content, NamedWriteable { + + public static final String NAME = "content_objects"; + + public ContentObjects(StreamInput in) throws IOException { + this(in.readCollectionAsImmutableList(ContentObject::new)); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeCollection(contentObjects); + } + + @Override + public String getWriteableName() { + return NAME; + } + } + + public record ContentObject(String text, String type) implements Writeable { + static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + ContentObject.class.getSimpleName(), + args -> new ContentObject((String) args[0], (String) args[1]) + ); + + static { + PARSER.declareString(constructorArg(), new ParseField("text")); + PARSER.declareString(constructorArg(), new ParseField("type")); + } + + public ContentObject(StreamInput in) throws IOException { + this(in.readString(), in.readString()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(text); + out.writeString(type); + } + + public String toString() { + return text + ":" + type; + } + + } + + public record ContentString(String content) implements Content, NamedWriteable { + public static final String NAME = "content_string"; + + public static ContentString of(XContentParser parser) throws IOException { + var content = parser.text(); + return new ContentString(content); + } + + public ContentString(StreamInput in) throws IOException { + this(in.readString()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(content); + } + + @Override + public String getWriteableName() { + return NAME; + } + + public String toString() { + return content; + } + } + + public record ToolCall(String id, FunctionField function, String type) implements Writeable { + + static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + ToolCall.class.getSimpleName(), + args -> new ToolCall((String) args[0], (FunctionField) args[1], (String) args[2]) + ); + + static { + PARSER.declareString(constructorArg(), new ParseField("id")); + PARSER.declareObject(constructorArg(), FunctionField.PARSER::apply, new ParseField("function")); + PARSER.declareString(constructorArg(), new ParseField("type")); + } + + public ToolCall(StreamInput in) throws IOException { + this(in.readString(), new FunctionField(in), in.readString()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(id); + function.writeTo(out); + out.writeString(type); + } + + public record FunctionField(String arguments, String name) implements Writeable { + static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + "tool_call_function_field", + args -> new FunctionField((String) args[0], (String) args[1]) + ); + + static { + PARSER.declareString(constructorArg(), new ParseField("arguments")); + PARSER.declareString(constructorArg(), new ParseField("name")); + } + + public FunctionField(StreamInput in) throws IOException { + this(in.readString(), in.readString()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(arguments); + out.writeString(name); + } + } + } + + private static ToolChoice parseToolChoice(XContentParser parser) throws IOException { + var token = parser.currentToken(); + if (token == XContentParser.Token.START_OBJECT) { + return ToolChoiceObject.PARSER.apply(parser, null); + } else if (token == XContentParser.Token.VALUE_STRING) { + return ToolChoiceString.of(parser); + } + + throw new XContentParseException("Unsupported token [" + token + "]"); + } + + public sealed interface ToolChoice extends NamedWriteable permits ToolChoiceObject, ToolChoiceString {} + + public record ToolChoiceObject(String type, FunctionField function) implements ToolChoice, NamedWriteable { + + public static final String NAME = "tool_choice_object"; + + static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + ToolChoiceObject.class.getSimpleName(), + args -> new ToolChoiceObject((String) args[0], (FunctionField) args[1]) + ); + + static { + PARSER.declareString(constructorArg(), new ParseField("type")); + PARSER.declareObject(constructorArg(), FunctionField.PARSER::apply, new ParseField("function")); + } + + public ToolChoiceObject(StreamInput in) throws IOException { + this(in.readString(), new FunctionField(in)); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(type); + function.writeTo(out); + } + + @Override + public String getWriteableName() { + return NAME; + } + + public record FunctionField(String name) implements Writeable { + static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + "tool_choice_function_field", + args -> new FunctionField((String) args[0]) + ); + + static { + PARSER.declareString(constructorArg(), new ParseField("name")); + } + + public FunctionField(StreamInput in) throws IOException { + this(in.readString()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(name); + } + } + } + + public record ToolChoiceString(String value) implements ToolChoice, NamedWriteable { + public static final String NAME = "tool_choice_string"; + + public static ToolChoiceString of(XContentParser parser) throws IOException { + var content = parser.text(); + return new ToolChoiceString(content); + } + + public ToolChoiceString(StreamInput in) throws IOException { + this(in.readString()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(value); + } + + @Override + public String getWriteableName() { + return NAME; + } + } + + public record Tool(String type, FunctionField function) implements Writeable { + + static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + Tool.class.getSimpleName(), + args -> new Tool((String) args[0], (FunctionField) args[1]) + ); + + static { + PARSER.declareString(constructorArg(), new ParseField("type")); + PARSER.declareObject(constructorArg(), FunctionField.PARSER::apply, new ParseField("function")); + } + + public Tool(StreamInput in) throws IOException { + this(in.readString(), new FunctionField(in)); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(type); + function.writeTo(out); + } + + public record FunctionField( + @Nullable String description, + String name, + @Nullable Map parameters, + @Nullable Boolean strict + ) implements Writeable { + + @SuppressWarnings("unchecked") + static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + "tool_function_field", + args -> new FunctionField((String) args[0], (String) args[1], (Map) args[2], (Boolean) args[3]) + ); + + static { + PARSER.declareString(optionalConstructorArg(), new ParseField("description")); + PARSER.declareString(constructorArg(), new ParseField("name")); + PARSER.declareObject(optionalConstructorArg(), (p, c) -> p.mapOrdered(), new ParseField("parameters")); + PARSER.declareBoolean(optionalConstructorArg(), new ParseField("strict")); + } + + public FunctionField(StreamInput in) throws IOException { + this(in.readOptionalString(), in.readString(), in.readGenericMap(), in.readOptionalBoolean()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeOptionalString(description); + out.writeString(name); + out.writeGenericMap(parameters); + out.writeOptionalBoolean(strict); + } + } + } +} diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java index d983fc854bdf..a71f61740e17 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java @@ -1205,10 +1205,30 @@ public static SecureString randomSecureStringOfLength(int codeUnits) { return new SecureString(randomAlpha.toCharArray()); } - public static String randomNullOrAlphaOfLength(int codeUnits) { + public static String randomAlphaOfLengthOrNull(int codeUnits) { return randomBoolean() ? null : randomAlphaOfLength(codeUnits); } + public static Long randomLongOrNull() { + return randomBoolean() ? null : randomLong(); + } + + public static Long randomPositiveLongOrNull() { + return randomBoolean() ? null : randomNonNegativeLong(); + } + + public static Integer randomIntOrNull() { + return randomBoolean() ? null : randomInt(); + } + + public static Integer randomPositiveIntOrNull() { + return randomBoolean() ? null : randomNonNegativeInt(); + } + + public static Float randomFloatOrNull() { + return randomBoolean() ? null : randomFloat(); + } + /** * Creates a valid random identifier such as node id or index name */ diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/action/BaseInferenceActionRequest.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/action/BaseInferenceActionRequest.java new file mode 100644 index 000000000000..e426574c52ce --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/action/BaseInferenceActionRequest.java @@ -0,0 +1,31 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.core.inference.action; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.inference.TaskType; + +import java.io.IOException; + +public abstract class BaseInferenceActionRequest extends ActionRequest { + + public BaseInferenceActionRequest() { + super(); + } + + public BaseInferenceActionRequest(StreamInput in) throws IOException { + super(in); + } + + public abstract boolean isStreaming(); + + public abstract TaskType getTaskType(); + + public abstract String getInferenceEntityId(); +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/action/InferenceAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/action/InferenceAction.java index a19edd5a0816..f88909ba4208 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/action/InferenceAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/action/InferenceAction.java @@ -10,7 +10,6 @@ import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.TransportVersion; import org.elasticsearch.TransportVersions; -import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; @@ -54,7 +53,7 @@ public InferenceAction() { super(NAME); } - public static class Request extends ActionRequest { + public static class Request extends BaseInferenceActionRequest { public static final TimeValue DEFAULT_TIMEOUT = TimeValue.timeValueSeconds(30); public static final ParseField INPUT = new ParseField("input"); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/action/UnifiedCompletionAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/action/UnifiedCompletionAction.java new file mode 100644 index 000000000000..8d121463fb46 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/action/UnifiedCompletionAction.java @@ -0,0 +1,129 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.core.inference.action; + +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.inference.TaskType; +import org.elasticsearch.inference.UnifiedCompletionRequest; +import org.elasticsearch.xcontent.XContentParser; + +import java.io.IOException; +import java.util.Objects; + +public class UnifiedCompletionAction extends ActionType { + public static final UnifiedCompletionAction INSTANCE = new UnifiedCompletionAction(); + public static final String NAME = "cluster:monitor/xpack/inference/unified"; + + public UnifiedCompletionAction() { + super(NAME); + } + + public static class Request extends BaseInferenceActionRequest { + public static Request parseRequest(String inferenceEntityId, TaskType taskType, TimeValue timeout, XContentParser parser) + throws IOException { + var unifiedRequest = UnifiedCompletionRequest.PARSER.apply(parser, null); + return new Request(inferenceEntityId, taskType, unifiedRequest, timeout); + } + + private final String inferenceEntityId; + private final TaskType taskType; + private final UnifiedCompletionRequest unifiedCompletionRequest; + private final TimeValue timeout; + + public Request(String inferenceEntityId, TaskType taskType, UnifiedCompletionRequest unifiedCompletionRequest, TimeValue timeout) { + this.inferenceEntityId = Objects.requireNonNull(inferenceEntityId); + this.taskType = Objects.requireNonNull(taskType); + this.unifiedCompletionRequest = Objects.requireNonNull(unifiedCompletionRequest); + this.timeout = Objects.requireNonNull(timeout); + } + + public Request(StreamInput in) throws IOException { + super(in); + this.inferenceEntityId = in.readString(); + this.taskType = TaskType.fromStream(in); + this.unifiedCompletionRequest = new UnifiedCompletionRequest(in); + this.timeout = in.readTimeValue(); + } + + public TaskType getTaskType() { + return taskType; + } + + public String getInferenceEntityId() { + return inferenceEntityId; + } + + public UnifiedCompletionRequest getUnifiedCompletionRequest() { + return unifiedCompletionRequest; + } + + /** + * The Unified API only supports streaming so we always return true here. + * @return true + */ + public boolean isStreaming() { + return true; + } + + public TimeValue getTimeout() { + return timeout; + } + + @Override + public ActionRequestValidationException validate() { + if (unifiedCompletionRequest == null || unifiedCompletionRequest.messages() == null) { + var e = new ActionRequestValidationException(); + e.addValidationError("Field [messages] cannot be null"); + return e; + } + + if (unifiedCompletionRequest.messages().isEmpty()) { + var e = new ActionRequestValidationException(); + e.addValidationError("Field [messages] cannot be an empty array"); + return e; + } + + if (taskType.isAnyOrSame(TaskType.COMPLETION) == false) { + var e = new ActionRequestValidationException(); + e.addValidationError("Field [taskType] must be [completion]"); + return e; + } + + return null; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(inferenceEntityId); + taskType.writeTo(out); + unifiedCompletionRequest.writeTo(out); + out.writeTimeValue(timeout); + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) return false; + Request request = (Request) o; + return Objects.equals(inferenceEntityId, request.inferenceEntityId) + && taskType == request.taskType + && Objects.equals(unifiedCompletionRequest, request.unifiedCompletionRequest) + && Objects.equals(timeout, request.timeout); + } + + @Override + public int hashCode() { + return Objects.hash(inferenceEntityId, taskType, unifiedCompletionRequest, timeout); + } + } + +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/results/StreamingUnifiedChatCompletionResults.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/results/StreamingUnifiedChatCompletionResults.java new file mode 100644 index 000000000000..90038c67036c --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/inference/results/StreamingUnifiedChatCompletionResults.java @@ -0,0 +1,329 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.core.inference.results; + +import org.elasticsearch.common.collect.Iterators; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ChunkedToXContent; +import org.elasticsearch.common.xcontent.ChunkedToXContentHelper; +import org.elasticsearch.inference.InferenceResults; +import org.elasticsearch.inference.InferenceServiceResults; +import org.elasticsearch.xcontent.ToXContent; + +import java.io.IOException; +import java.util.Collections; +import java.util.Deque; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Flow; + +/** + * Chat Completion results that only contain a Flow.Publisher. + */ +public record StreamingUnifiedChatCompletionResults(Flow.Publisher publisher) + implements + InferenceServiceResults { + + public static final String NAME = "chat_completion_chunk"; + public static final String MODEL_FIELD = "model"; + public static final String OBJECT_FIELD = "object"; + public static final String USAGE_FIELD = "usage"; + public static final String INDEX_FIELD = "index"; + public static final String ID_FIELD = "id"; + public static final String FUNCTION_NAME_FIELD = "name"; + public static final String FUNCTION_ARGUMENTS_FIELD = "arguments"; + public static final String FUNCTION_FIELD = "function"; + public static final String CHOICES_FIELD = "choices"; + public static final String DELTA_FIELD = "delta"; + public static final String CONTENT_FIELD = "content"; + public static final String REFUSAL_FIELD = "refusal"; + public static final String ROLE_FIELD = "role"; + private static final String TOOL_CALLS_FIELD = "tool_calls"; + public static final String FINISH_REASON_FIELD = "finish_reason"; + public static final String COMPLETION_TOKENS_FIELD = "completion_tokens"; + public static final String TOTAL_TOKENS_FIELD = "total_tokens"; + public static final String PROMPT_TOKENS_FIELD = "prompt_tokens"; + public static final String TYPE_FIELD = "type"; + + @Override + public boolean isStreaming() { + return true; + } + + @Override + public List transformToCoordinationFormat() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public List transformToLegacyFormat() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public Map asMap() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public String getWriteableName() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public Iterator toXContentChunked(ToXContent.Params params) { + throw new UnsupportedOperationException("Not implemented"); + } + + public record Results(Deque chunks) implements ChunkedToXContent { + @Override + public Iterator toXContentChunked(ToXContent.Params params) { + return Iterators.concat(Iterators.flatMap(chunks.iterator(), c -> c.toXContentChunked(params))); + } + } + + public static class ChatCompletionChunk implements ChunkedToXContent { + private final String id; + + public String getId() { + return id; + } + + public List getChoices() { + return choices; + } + + public String getModel() { + return model; + } + + public String getObject() { + return object; + } + + public Usage getUsage() { + return usage; + } + + private final List choices; + private final String model; + private final String object; + private final ChatCompletionChunk.Usage usage; + + public ChatCompletionChunk(String id, List choices, String model, String object, ChatCompletionChunk.Usage usage) { + this.id = id; + this.choices = choices; + this.model = model; + this.object = object; + this.usage = usage; + } + + @Override + public Iterator toXContentChunked(ToXContent.Params params) { + + Iterator choicesIterator = Collections.emptyIterator(); + if (choices != null) { + choicesIterator = Iterators.concat( + ChunkedToXContentHelper.startArray(CHOICES_FIELD), + Iterators.flatMap(choices.iterator(), c -> c.toXContentChunked(params)), + ChunkedToXContentHelper.endArray() + ); + } + + Iterator usageIterator = Collections.emptyIterator(); + if (usage != null) { + usageIterator = Iterators.concat( + ChunkedToXContentHelper.startObject(USAGE_FIELD), + ChunkedToXContentHelper.field(COMPLETION_TOKENS_FIELD, usage.completionTokens()), + ChunkedToXContentHelper.field(PROMPT_TOKENS_FIELD, usage.promptTokens()), + ChunkedToXContentHelper.field(TOTAL_TOKENS_FIELD, usage.totalTokens()), + ChunkedToXContentHelper.endObject() + ); + } + + return Iterators.concat( + ChunkedToXContentHelper.startObject(), + ChunkedToXContentHelper.field(ID_FIELD, id), + choicesIterator, + ChunkedToXContentHelper.field(MODEL_FIELD, model), + ChunkedToXContentHelper.field(OBJECT_FIELD, object), + usageIterator, + ChunkedToXContentHelper.endObject() + ); + } + + public record Choice(ChatCompletionChunk.Choice.Delta delta, String finishReason, int index) { + + /* + choices: Array<{ + delta: { ... }; + finish_reason: string | null; + index: number; + }>; + */ + public Iterator toXContentChunked(ToXContent.Params params) { + return Iterators.concat( + ChunkedToXContentHelper.startObject(), + delta.toXContentChunked(params), + ChunkedToXContentHelper.optionalField(FINISH_REASON_FIELD, finishReason), + ChunkedToXContentHelper.field(INDEX_FIELD, index), + ChunkedToXContentHelper.endObject() + ); + } + + public static class Delta { + private final String content; + private final String refusal; + private final String role; + private List toolCalls; + + public Delta(String content, String refusal, String role, List toolCalls) { + this.content = content; + this.refusal = refusal; + this.role = role; + this.toolCalls = toolCalls; + } + + /* + delta: { + content?: string | null; + refusal?: string | null; + role?: 'system' | 'user' | 'assistant' | 'tool'; + tool_calls?: Array<{ ... }>; + }; + */ + public Iterator toXContentChunked(ToXContent.Params params) { + var xContent = Iterators.concat( + ChunkedToXContentHelper.startObject(DELTA_FIELD), + ChunkedToXContentHelper.optionalField(CONTENT_FIELD, content), + ChunkedToXContentHelper.optionalField(REFUSAL_FIELD, refusal), + ChunkedToXContentHelper.optionalField(ROLE_FIELD, role) + ); + + if (toolCalls != null && toolCalls.isEmpty() == false) { + xContent = Iterators.concat( + xContent, + ChunkedToXContentHelper.startArray(TOOL_CALLS_FIELD), + Iterators.flatMap(toolCalls.iterator(), t -> t.toXContentChunked(params)), + ChunkedToXContentHelper.endArray() + ); + } + xContent = Iterators.concat(xContent, ChunkedToXContentHelper.endObject()); + return xContent; + + } + + public String getContent() { + return content; + } + + public String getRefusal() { + return refusal; + } + + public String getRole() { + return role; + } + + public List getToolCalls() { + return toolCalls; + } + + public static class ToolCall { + private final int index; + private final String id; + public ChatCompletionChunk.Choice.Delta.ToolCall.Function function; + private final String type; + + public ToolCall(int index, String id, ChatCompletionChunk.Choice.Delta.ToolCall.Function function, String type) { + this.index = index; + this.id = id; + this.function = function; + this.type = type; + } + + public int getIndex() { + return index; + } + + public String getId() { + return id; + } + + public ChatCompletionChunk.Choice.Delta.ToolCall.Function getFunction() { + return function; + } + + public String getType() { + return type; + } + + /* + index: number; + id?: string; + function?: { + arguments?: string; + name?: string; + }; + type?: 'function'; + */ + public Iterator toXContentChunked(ToXContent.Params params) { + var content = Iterators.concat( + ChunkedToXContentHelper.startObject(), + ChunkedToXContentHelper.field(INDEX_FIELD, index), + ChunkedToXContentHelper.optionalField(ID_FIELD, id) + ); + + if (function != null) { + content = Iterators.concat( + content, + ChunkedToXContentHelper.startObject(FUNCTION_FIELD), + ChunkedToXContentHelper.optionalField(FUNCTION_ARGUMENTS_FIELD, function.getArguments()), + ChunkedToXContentHelper.optionalField(FUNCTION_NAME_FIELD, function.getName()), + ChunkedToXContentHelper.endObject() + ); + } + + content = Iterators.concat( + content, + ChunkedToXContentHelper.field(TYPE_FIELD, type), + ChunkedToXContentHelper.endObject() + ); + return content; + } + + public static class Function { + private final String arguments; + private final String name; + + public Function(String arguments, String name) { + this.arguments = arguments; + this.name = name; + } + + public String getArguments() { + return arguments; + } + + public String getName() { + return name; + } + } + } + } + } + + public record Usage(int completionTokens, int promptTokens, int totalTokens) {} + } +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/action/InferenceActionRequestTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/action/InferenceActionRequestTests.java index a9ca5e6da872..01c0ff88be22 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/action/InferenceActionRequestTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/action/InferenceActionRequestTests.java @@ -41,8 +41,7 @@ protected InferenceAction.Request createTestInstance() { return new InferenceAction.Request( randomFrom(TaskType.values()), randomAlphaOfLength(6), - // null, - randomNullOrAlphaOfLength(10), + randomAlphaOfLengthOrNull(10), randomList(1, 5, () -> randomAlphaOfLength(8)), randomMap(0, 3, () -> new Tuple<>(randomAlphaOfLength(4), randomAlphaOfLength(4))), randomFrom(InputType.values()), diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/action/UnifiedCompletionActionRequestTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/action/UnifiedCompletionActionRequestTests.java new file mode 100644 index 000000000000..1872ac3caa23 --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/action/UnifiedCompletionActionRequestTests.java @@ -0,0 +1,97 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.core.inference.action; + +import org.elasticsearch.TransportVersion; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.inference.TaskType; +import org.elasticsearch.inference.UnifiedCompletionRequest; +import org.elasticsearch.xpack.core.ml.AbstractBWCWireSerializationTestCase; + +import java.io.IOException; +import java.util.List; + +import static org.hamcrest.Matchers.is; + +public class UnifiedCompletionActionRequestTests extends AbstractBWCWireSerializationTestCase { + + public void testValidation_ReturnsException_When_UnifiedCompletionRequestMessage_Is_Null() { + var request = new UnifiedCompletionAction.Request( + "inference_id", + TaskType.COMPLETION, + UnifiedCompletionRequest.of(null), + TimeValue.timeValueSeconds(10) + ); + var exception = request.validate(); + assertThat(exception.getMessage(), is("Validation Failed: 1: Field [messages] cannot be null;")); + } + + public void testValidation_ReturnsException_When_UnifiedCompletionRequest_Is_EmptyArray() { + var request = new UnifiedCompletionAction.Request( + "inference_id", + TaskType.COMPLETION, + UnifiedCompletionRequest.of(List.of()), + TimeValue.timeValueSeconds(10) + ); + var exception = request.validate(); + assertThat(exception.getMessage(), is("Validation Failed: 1: Field [messages] cannot be an empty array;")); + } + + public void testValidation_ReturnsException_When_TaskType_IsNot_Completion() { + var request = new UnifiedCompletionAction.Request( + "inference_id", + TaskType.SPARSE_EMBEDDING, + UnifiedCompletionRequest.of(List.of(UnifiedCompletionRequestTests.randomMessage())), + TimeValue.timeValueSeconds(10) + ); + var exception = request.validate(); + assertThat(exception.getMessage(), is("Validation Failed: 1: Field [taskType] must be [completion];")); + } + + public void testValidation_ReturnsNull_When_TaskType_IsAny() { + var request = new UnifiedCompletionAction.Request( + "inference_id", + TaskType.ANY, + UnifiedCompletionRequest.of(List.of(UnifiedCompletionRequestTests.randomMessage())), + TimeValue.timeValueSeconds(10) + ); + assertNull(request.validate()); + } + + @Override + protected UnifiedCompletionAction.Request mutateInstanceForVersion(UnifiedCompletionAction.Request instance, TransportVersion version) { + return instance; + } + + @Override + protected Writeable.Reader instanceReader() { + return UnifiedCompletionAction.Request::new; + } + + @Override + protected UnifiedCompletionAction.Request createTestInstance() { + return new UnifiedCompletionAction.Request( + randomAlphaOfLength(10), + randomFrom(TaskType.values()), + UnifiedCompletionRequestTests.randomUnifiedCompletionRequest(), + TimeValue.timeValueMillis(randomLongBetween(1, 2048)) + ); + } + + @Override + protected UnifiedCompletionAction.Request mutateInstance(UnifiedCompletionAction.Request instance) throws IOException { + return randomValueOtherThan(instance, this::createTestInstance); + } + + @Override + protected NamedWriteableRegistry getNamedWriteableRegistry() { + return new NamedWriteableRegistry(UnifiedCompletionRequest.getNamedWriteables()); + } +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/action/UnifiedCompletionRequestTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/action/UnifiedCompletionRequestTests.java new file mode 100644 index 000000000000..47a0814a584b --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/action/UnifiedCompletionRequestTests.java @@ -0,0 +1,293 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.core.inference.action; + +import org.elasticsearch.TransportVersion; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.inference.UnifiedCompletionRequest; +import org.elasticsearch.xcontent.json.JsonXContent; +import org.elasticsearch.xpack.core.ml.AbstractBWCWireSerializationTestCase; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.Matchers.is; + +public class UnifiedCompletionRequestTests extends AbstractBWCWireSerializationTestCase { + + public void testParseAllFields() throws IOException { + String requestJson = """ + { + "model": "gpt-4o", + "messages": [ + { + "role": "user", + "content": [ + { + "text": "some text", + "type": "string" + } + ], + "name": "a name", + "tool_call_id": "100", + "tool_calls": [ + { + "id": "call_62136354", + "type": "function", + "function": { + "arguments": "{'order_id': 'order_12345'}", + "name": "get_delivery_date" + } + } + ] + } + ], + "max_completion_tokens": 100, + "stop": ["stop"], + "temperature": 0.1, + "tools": [ + { + "type": "function", + "function": { + "name": "get_current_weather", + "description": "Get the current weather in a given location", + "parameters": { + "type": "object" + } + } + } + ], + "tool_choice": { + "type": "function", + "function": { + "name": "some function" + } + }, + "top_p": 0.2 + } + """; + + try (var parser = createParser(JsonXContent.jsonXContent, requestJson)) { + var request = UnifiedCompletionRequest.PARSER.apply(parser, null); + var expected = new UnifiedCompletionRequest( + List.of( + new UnifiedCompletionRequest.Message( + new UnifiedCompletionRequest.ContentObjects( + List.of(new UnifiedCompletionRequest.ContentObject("some text", "string")) + ), + "user", + "a name", + "100", + List.of( + new UnifiedCompletionRequest.ToolCall( + "call_62136354", + new UnifiedCompletionRequest.ToolCall.FunctionField("{'order_id': 'order_12345'}", "get_delivery_date"), + "function" + ) + ) + ) + ), + "gpt-4o", + 100L, + List.of("stop"), + 0.1F, + new UnifiedCompletionRequest.ToolChoiceObject( + "function", + new UnifiedCompletionRequest.ToolChoiceObject.FunctionField("some function") + ), + List.of( + new UnifiedCompletionRequest.Tool( + "function", + new UnifiedCompletionRequest.Tool.FunctionField( + "Get the current weather in a given location", + "get_current_weather", + Map.of("type", "object"), + null + ) + ) + ), + 0.2F + ); + + assertThat(request, is(expected)); + } + } + + public void testParsing() throws IOException { + String requestJson = """ + { + "model": "gpt-4o", + "messages": [ + { + "role": "user", + "content": "What is the weather like in Boston today?" + } + ], + "stop": "none", + "tools": [ + { + "type": "function", + "function": { + "name": "get_current_weather", + "description": "Get the current weather in a given location", + "parameters": { + "type": "object" + } + } + } + ], + "tool_choice": "auto" + } + """; + + try (var parser = createParser(JsonXContent.jsonXContent, requestJson)) { + var request = UnifiedCompletionRequest.PARSER.apply(parser, null); + var expected = new UnifiedCompletionRequest( + List.of( + new UnifiedCompletionRequest.Message( + new UnifiedCompletionRequest.ContentString("What is the weather like in Boston today?"), + "user", + null, + null, + null + ) + ), + "gpt-4o", + null, + List.of("none"), + null, + new UnifiedCompletionRequest.ToolChoiceString("auto"), + List.of( + new UnifiedCompletionRequest.Tool( + "function", + new UnifiedCompletionRequest.Tool.FunctionField( + "Get the current weather in a given location", + "get_current_weather", + Map.of("type", "object"), + null + ) + ) + ), + null + ); + + assertThat(request, is(expected)); + } + } + + public static UnifiedCompletionRequest randomUnifiedCompletionRequest() { + return new UnifiedCompletionRequest( + randomList(5, UnifiedCompletionRequestTests::randomMessage), + randomAlphaOfLengthOrNull(10), + randomPositiveLongOrNull(), + randomStopOrNull(), + randomFloatOrNull(), + randomToolChoiceOrNull(), + randomToolListOrNull(), + randomFloatOrNull() + ); + } + + public static UnifiedCompletionRequest.Message randomMessage() { + return new UnifiedCompletionRequest.Message( + randomContent(), + randomAlphaOfLength(10), + randomAlphaOfLengthOrNull(10), + randomAlphaOfLengthOrNull(10), + randomToolCallListOrNull() + ); + } + + public static UnifiedCompletionRequest.Content randomContent() { + return randomBoolean() + ? new UnifiedCompletionRequest.ContentString(randomAlphaOfLength(10)) + : new UnifiedCompletionRequest.ContentObjects(randomList(10, UnifiedCompletionRequestTests::randomContentObject)); + } + + public static UnifiedCompletionRequest.ContentObject randomContentObject() { + return new UnifiedCompletionRequest.ContentObject(randomAlphaOfLength(10), randomAlphaOfLength(10)); + } + + public static List randomToolCallListOrNull() { + return randomBoolean() ? randomList(10, UnifiedCompletionRequestTests::randomToolCall) : null; + } + + public static UnifiedCompletionRequest.ToolCall randomToolCall() { + return new UnifiedCompletionRequest.ToolCall(randomAlphaOfLength(10), randomToolCallFunctionField(), randomAlphaOfLength(10)); + } + + public static UnifiedCompletionRequest.ToolCall.FunctionField randomToolCallFunctionField() { + return new UnifiedCompletionRequest.ToolCall.FunctionField(randomAlphaOfLength(10), randomAlphaOfLength(10)); + } + + public static List randomStopOrNull() { + return randomBoolean() ? randomStop() : null; + } + + public static List randomStop() { + return randomList(5, () -> randomAlphaOfLength(10)); + } + + public static UnifiedCompletionRequest.ToolChoice randomToolChoiceOrNull() { + return randomBoolean() ? randomToolChoice() : null; + } + + public static UnifiedCompletionRequest.ToolChoice randomToolChoice() { + return randomBoolean() + ? new UnifiedCompletionRequest.ToolChoiceString(randomAlphaOfLength(10)) + : new UnifiedCompletionRequest.ToolChoiceObject(randomAlphaOfLength(10), randomToolChoiceObjectFunctionField()); + } + + public static UnifiedCompletionRequest.ToolChoiceObject.FunctionField randomToolChoiceObjectFunctionField() { + return new UnifiedCompletionRequest.ToolChoiceObject.FunctionField(randomAlphaOfLength(10)); + } + + public static List randomToolListOrNull() { + return randomBoolean() ? randomList(10, UnifiedCompletionRequestTests::randomTool) : null; + } + + public static UnifiedCompletionRequest.Tool randomTool() { + return new UnifiedCompletionRequest.Tool(randomAlphaOfLength(10), randomToolFunctionField()); + } + + public static UnifiedCompletionRequest.Tool.FunctionField randomToolFunctionField() { + return new UnifiedCompletionRequest.Tool.FunctionField( + randomAlphaOfLengthOrNull(10), + randomAlphaOfLength(10), + null, + randomOptionalBoolean() + ); + } + + @Override + protected UnifiedCompletionRequest mutateInstanceForVersion(UnifiedCompletionRequest instance, TransportVersion version) { + return instance; + } + + @Override + protected Writeable.Reader instanceReader() { + return UnifiedCompletionRequest::new; + } + + @Override + protected UnifiedCompletionRequest createTestInstance() { + return randomUnifiedCompletionRequest(); + } + + @Override + protected UnifiedCompletionRequest mutateInstance(UnifiedCompletionRequest instance) throws IOException { + return randomValueOtherThan(instance, this::createTestInstance); + } + + @Override + protected NamedWriteableRegistry getNamedWriteableRegistry() { + return new NamedWriteableRegistry(UnifiedCompletionRequest.getNamedWriteables()); + } +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/results/StreamingUnifiedChatCompletionResultsTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/results/StreamingUnifiedChatCompletionResultsTests.java new file mode 100644 index 000000000000..a8f569dbef9d --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/inference/results/StreamingUnifiedChatCompletionResultsTests.java @@ -0,0 +1,198 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + * + * this file was contributed to by a generative AI + */ + +package org.elasticsearch.xpack.core.inference.results; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.json.JsonXContent; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.Deque; +import java.util.List; + +public class StreamingUnifiedChatCompletionResultsTests extends ESTestCase { + + public void testResults_toXContentChunked() throws IOException { + String expected = """ + { + "id": "chunk1", + "choices": [ + { + "delta": { + "content": "example_content", + "refusal": "example_refusal", + "role": "assistant", + "tool_calls": [ + { + "index": 1, + "id": "tool1", + "function": { + "arguments": "example_arguments", + "name": "example_function" + }, + "type": "function" + } + ] + }, + "finish_reason": "example_reason", + "index": 0 + } + ], + "model": "example_model", + "object": "example_object", + "usage": { + "completion_tokens": 10, + "prompt_tokens": 5, + "total_tokens": 15 + } + } + """; + + StreamingUnifiedChatCompletionResults.ChatCompletionChunk chunk = new StreamingUnifiedChatCompletionResults.ChatCompletionChunk( + "chunk1", + List.of( + new StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice( + new StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta( + "example_content", + "example_refusal", + "assistant", + List.of( + new StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta.ToolCall( + 1, + "tool1", + new StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta.ToolCall.Function( + "example_arguments", + "example_function" + ), + "function" + ) + ) + ), + "example_reason", + 0 + ) + ), + "example_model", + "example_object", + new StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Usage(10, 5, 15) + ); + + Deque deque = new ArrayDeque<>(); + deque.add(chunk); + StreamingUnifiedChatCompletionResults.Results results = new StreamingUnifiedChatCompletionResults.Results(deque); + XContentBuilder builder = JsonXContent.contentBuilder(); + results.toXContentChunked(null).forEachRemaining(xContent -> { + try { + xContent.toXContent(builder, null); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + assertEquals(expected.replaceAll("\\s+", ""), Strings.toString(builder.prettyPrint()).trim()); + } + + public void testChoiceToXContentChunked() throws IOException { + String expected = """ + { + "delta": { + "content": "example_content", + "refusal": "example_refusal", + "role": "assistant", + "tool_calls": [ + { + "index": 1, + "id": "tool1", + "function": { + "arguments": "example_arguments", + "name": "example_function" + }, + "type": "function" + } + ] + }, + "finish_reason": "example_reason", + "index": 0 + } + """; + + StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice choice = + new StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice( + new StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta( + "example_content", + "example_refusal", + "assistant", + List.of( + new StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta.ToolCall( + 1, + "tool1", + new StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta.ToolCall.Function( + "example_arguments", + "example_function" + ), + "function" + ) + ) + ), + "example_reason", + 0 + ); + + XContentBuilder builder = JsonXContent.contentBuilder(); + choice.toXContentChunked(null).forEachRemaining(xContent -> { + try { + xContent.toXContent(builder, null); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + assertEquals(expected.replaceAll("\\s+", ""), Strings.toString(builder.prettyPrint()).trim()); + } + + public void testToolCallToXContentChunked() throws IOException { + String expected = """ + { + "index": 1, + "id": "tool1", + "function": { + "arguments": "example_arguments", + "name": "example_function" + }, + "type": "function" + } + """; + + StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta.ToolCall toolCall = + new StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta.ToolCall( + 1, + "tool1", + new StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta.ToolCall.Function( + "example_arguments", + "example_function" + ), + "function" + ); + + XContentBuilder builder = JsonXContent.contentBuilder(); + toolCall.toXContentChunked(null).forEachRemaining(xContent -> { + try { + xContent.toXContent(builder, null); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + assertEquals(expected.replaceAll("\\s+", ""), Strings.toString(builder.prettyPrint()).trim()); + } + +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java index 17579fd6368c..eeffa1db5485 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java @@ -4175,6 +4175,7 @@ public void testInferenceUserRole() { assertTrue(role.cluster().check("cluster:monitor/xpack/inference", request, authentication)); assertTrue(role.cluster().check("cluster:monitor/xpack/inference/get", request, authentication)); assertFalse(role.cluster().check("cluster:admin/xpack/inference/put", request, authentication)); + assertTrue(role.cluster().check("cluster:monitor/xpack/inference/unified", request, authentication)); assertFalse(role.cluster().check("cluster:admin/xpack/inference/delete", request, authentication)); assertTrue(role.cluster().check("cluster:monitor/xpack/ml/trained_models/deployment/infer", request, authentication)); assertFalse(role.cluster().check("cluster:admin/xpack/ml/trained_models/deployment/start", request, authentication)); diff --git a/x-pack/plugin/inference/qa/inference-service-tests/src/javaRestTest/java/org/elasticsearch/xpack/inference/InferenceBaseRestTest.java b/x-pack/plugin/inference/qa/inference-service-tests/src/javaRestTest/java/org/elasticsearch/xpack/inference/InferenceBaseRestTest.java index 86c0128a3e53..1716057cdfe4 100644 --- a/x-pack/plugin/inference/qa/inference-service-tests/src/javaRestTest/java/org/elasticsearch/xpack/inference/InferenceBaseRestTest.java +++ b/x-pack/plugin/inference/qa/inference-service-tests/src/javaRestTest/java/org/elasticsearch/xpack/inference/InferenceBaseRestTest.java @@ -21,6 +21,9 @@ import org.elasticsearch.test.cluster.ElasticsearchCluster; import org.elasticsearch.test.cluster.local.distribution.DistributionType; import org.elasticsearch.test.rest.ESRestTestCase; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentFactory; +import org.elasticsearch.xcontent.XContentType; import org.elasticsearch.xpack.inference.external.response.streaming.ServerSentEvent; import org.junit.ClassRule; @@ -341,10 +344,21 @@ protected Deque streamInferOnMockService(String modelId, TaskTy return callAsync(endpoint, input); } + protected Deque unifiedCompletionInferOnMockService(String modelId, TaskType taskType, List input) + throws Exception { + var endpoint = Strings.format("_inference/%s/%s/_unified", taskType, modelId); + return callAsyncUnified(endpoint, input, "user"); + } + private Deque callAsync(String endpoint, List input) throws Exception { - var responseConsumer = new AsyncInferenceResponseConsumer(); var request = new Request("POST", endpoint); request.setJsonEntity(jsonBody(input, null)); + + return execAsyncCall(request); + } + + private Deque execAsyncCall(Request request) throws Exception { + var responseConsumer = new AsyncInferenceResponseConsumer(); request.setOptions(RequestOptions.DEFAULT.toBuilder().setHttpAsyncResponseConsumerFactory(() -> responseConsumer).build()); var latch = new CountDownLatch(1); client().performRequestAsync(request, new ResponseListener() { @@ -362,6 +376,22 @@ public void onFailure(Exception exception) { return responseConsumer.events(); } + private Deque callAsyncUnified(String endpoint, List input, String role) throws Exception { + var request = new Request("POST", endpoint); + + request.setJsonEntity(createUnifiedJsonBody(input, role)); + return execAsyncCall(request); + } + + private String createUnifiedJsonBody(List input, String role) throws IOException { + var messages = input.stream().map(i -> Map.of("content", i, "role", role)).toList(); + XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON); + builder.startObject(); + builder.field("messages", messages); + builder.endObject(); + return org.elasticsearch.common.Strings.toString(builder); + } + protected Map infer(String modelId, TaskType taskType, List input) throws IOException { var endpoint = Strings.format("_inference/%s/%s", taskType, modelId); return inferInternal(endpoint, input, null, Map.of()); diff --git a/x-pack/plugin/inference/qa/inference-service-tests/src/javaRestTest/java/org/elasticsearch/xpack/inference/InferenceCrudIT.java b/x-pack/plugin/inference/qa/inference-service-tests/src/javaRestTest/java/org/elasticsearch/xpack/inference/InferenceCrudIT.java index 604e1d4f553b..2099ec8287a7 100644 --- a/x-pack/plugin/inference/qa/inference-service-tests/src/javaRestTest/java/org/elasticsearch/xpack/inference/InferenceCrudIT.java +++ b/x-pack/plugin/inference/qa/inference-service-tests/src/javaRestTest/java/org/elasticsearch/xpack/inference/InferenceCrudIT.java @@ -11,13 +11,18 @@ import org.apache.http.util.EntityUtils; import org.elasticsearch.client.ResponseException; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.inference.TaskType; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentFactory; +import org.elasticsearch.xcontent.XContentType; import org.elasticsearch.xpack.inference.services.elastic.ElasticInferenceServiceFeature; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; @@ -481,6 +486,56 @@ public void testSupportedStream() throws Exception { } } + public void testUnifiedCompletionInference() throws Exception { + String modelId = "streaming"; + putModel(modelId, mockCompletionServiceModelConfig(TaskType.COMPLETION)); + var singleModel = getModel(modelId); + assertEquals(modelId, singleModel.get("inference_id")); + assertEquals(TaskType.COMPLETION.toString(), singleModel.get("task_type")); + + var input = IntStream.range(1, 2 + randomInt(8)).mapToObj(i -> randomUUID()).toList(); + try { + var events = unifiedCompletionInferOnMockService(modelId, TaskType.COMPLETION, input); + var expectedResponses = expectedResultsIterator(input); + assertThat(events.size(), equalTo((input.size() + 1) * 2)); + events.forEach(event -> { + switch (event.name()) { + case EVENT -> assertThat(event.value(), equalToIgnoringCase("message")); + case DATA -> assertThat(event.value(), equalTo(expectedResponses.next())); + } + }); + } finally { + deleteModel(modelId); + } + } + + private static Iterator expectedResultsIterator(List input) { + return Stream.concat(input.stream().map(String::toUpperCase).map(InferenceCrudIT::expectedResult), Stream.of("[DONE]")).iterator(); + } + + private static String expectedResult(String input) { + try { + XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON); + builder.startObject(); + builder.field("id", "id"); + builder.startArray("choices"); + builder.startObject(); + builder.startObject("delta"); + builder.field("content", input); + builder.endObject(); + builder.field("index", 0); + builder.endObject(); + builder.endArray(); + builder.field("model", "gpt-4o-2024-08-06"); + builder.field("object", "chat.completion.chunk"); + builder.endObject(); + + return Strings.toString(builder); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + public void testGetZeroModels() throws IOException { var models = getModels("_all", TaskType.COMPLETION); assertThat(models, empty()); diff --git a/x-pack/plugin/inference/qa/test-service-plugin/src/main/java/org/elasticsearch/xpack/inference/mock/TestDenseInferenceServiceExtension.java b/x-pack/plugin/inference/qa/test-service-plugin/src/main/java/org/elasticsearch/xpack/inference/mock/TestDenseInferenceServiceExtension.java index ae11a02d312e..f5f682b143a7 100644 --- a/x-pack/plugin/inference/qa/test-service-plugin/src/main/java/org/elasticsearch/xpack/inference/mock/TestDenseInferenceServiceExtension.java +++ b/x-pack/plugin/inference/qa/test-service-plugin/src/main/java/org/elasticsearch/xpack/inference/mock/TestDenseInferenceServiceExtension.java @@ -31,6 +31,7 @@ import org.elasticsearch.inference.SimilarityMeasure; import org.elasticsearch.inference.TaskSettingsConfiguration; import org.elasticsearch.inference.TaskType; +import org.elasticsearch.inference.UnifiedCompletionRequest; import org.elasticsearch.inference.configuration.SettingsConfigurationDisplayType; import org.elasticsearch.inference.configuration.SettingsConfigurationFieldType; import org.elasticsearch.rest.RestStatus; @@ -132,6 +133,16 @@ public void infer( } } + @Override + public void unifiedCompletionInfer( + Model model, + UnifiedCompletionRequest request, + TimeValue timeout, + ActionListener listener + ) { + listener.onFailure(new UnsupportedOperationException("unifiedCompletionInfer not supported")); + } + @Override public void chunkedInfer( Model model, diff --git a/x-pack/plugin/inference/qa/test-service-plugin/src/main/java/org/elasticsearch/xpack/inference/mock/TestRerankingServiceExtension.java b/x-pack/plugin/inference/qa/test-service-plugin/src/main/java/org/elasticsearch/xpack/inference/mock/TestRerankingServiceExtension.java index 9320571572f0..fa1e27005c28 100644 --- a/x-pack/plugin/inference/qa/test-service-plugin/src/main/java/org/elasticsearch/xpack/inference/mock/TestRerankingServiceExtension.java +++ b/x-pack/plugin/inference/qa/test-service-plugin/src/main/java/org/elasticsearch/xpack/inference/mock/TestRerankingServiceExtension.java @@ -29,6 +29,7 @@ import org.elasticsearch.inference.SettingsConfiguration; import org.elasticsearch.inference.TaskSettingsConfiguration; import org.elasticsearch.inference.TaskType; +import org.elasticsearch.inference.UnifiedCompletionRequest; import org.elasticsearch.inference.configuration.SettingsConfigurationDisplayType; import org.elasticsearch.inference.configuration.SettingsConfigurationFieldType; import org.elasticsearch.rest.RestStatus; @@ -120,6 +121,16 @@ public void infer( } } + @Override + public void unifiedCompletionInfer( + Model model, + UnifiedCompletionRequest request, + TimeValue timeout, + ActionListener listener + ) { + listener.onFailure(new UnsupportedOperationException("unifiedCompletionInfer not supported")); + } + @Override public void chunkedInfer( Model model, diff --git a/x-pack/plugin/inference/qa/test-service-plugin/src/main/java/org/elasticsearch/xpack/inference/mock/TestSparseInferenceServiceExtension.java b/x-pack/plugin/inference/qa/test-service-plugin/src/main/java/org/elasticsearch/xpack/inference/mock/TestSparseInferenceServiceExtension.java index fe0223cce032..64569fd8c5c6 100644 --- a/x-pack/plugin/inference/qa/test-service-plugin/src/main/java/org/elasticsearch/xpack/inference/mock/TestSparseInferenceServiceExtension.java +++ b/x-pack/plugin/inference/qa/test-service-plugin/src/main/java/org/elasticsearch/xpack/inference/mock/TestSparseInferenceServiceExtension.java @@ -29,6 +29,7 @@ import org.elasticsearch.inference.SettingsConfiguration; import org.elasticsearch.inference.TaskSettingsConfiguration; import org.elasticsearch.inference.TaskType; +import org.elasticsearch.inference.UnifiedCompletionRequest; import org.elasticsearch.inference.configuration.SettingsConfigurationDisplayType; import org.elasticsearch.inference.configuration.SettingsConfigurationFieldType; import org.elasticsearch.rest.RestStatus; @@ -123,6 +124,16 @@ public void infer( } } + @Override + public void unifiedCompletionInfer( + Model model, + UnifiedCompletionRequest request, + TimeValue timeout, + ActionListener listener + ) { + throw new UnsupportedOperationException("unifiedCompletionInfer not supported"); + } + @Override public void chunkedInfer( Model model, diff --git a/x-pack/plugin/inference/qa/test-service-plugin/src/main/java/org/elasticsearch/xpack/inference/mock/TestStreamingCompletionServiceExtension.java b/x-pack/plugin/inference/qa/test-service-plugin/src/main/java/org/elasticsearch/xpack/inference/mock/TestStreamingCompletionServiceExtension.java index 6d7983bc8cb5..f7a05a27354e 100644 --- a/x-pack/plugin/inference/qa/test-service-plugin/src/main/java/org/elasticsearch/xpack/inference/mock/TestStreamingCompletionServiceExtension.java +++ b/x-pack/plugin/inference/qa/test-service-plugin/src/main/java/org/elasticsearch/xpack/inference/mock/TestStreamingCompletionServiceExtension.java @@ -30,12 +30,14 @@ import org.elasticsearch.inference.SettingsConfiguration; import org.elasticsearch.inference.TaskSettingsConfiguration; import org.elasticsearch.inference.TaskType; +import org.elasticsearch.inference.UnifiedCompletionRequest; import org.elasticsearch.inference.configuration.SettingsConfigurationDisplayType; import org.elasticsearch.inference.configuration.SettingsConfigurationFieldType; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.xcontent.ToXContentObject; import org.elasticsearch.xcontent.XContentBuilder; import org.elasticsearch.xpack.core.inference.results.StreamingChatCompletionResults; +import org.elasticsearch.xpack.core.inference.results.StreamingUnifiedChatCompletionResults; import java.io.IOException; import java.util.EnumSet; @@ -121,6 +123,24 @@ public void infer( } } + @Override + public void unifiedCompletionInfer( + Model model, + UnifiedCompletionRequest request, + TimeValue timeout, + ActionListener listener + ) { + switch (model.getConfigurations().getTaskType()) { + case COMPLETION -> listener.onResponse(makeUnifiedResults(request)); + default -> listener.onFailure( + new ElasticsearchStatusException( + TaskType.unsupportedTaskTypeErrorMsg(model.getConfigurations().getTaskType(), name()), + RestStatus.BAD_REQUEST + ) + ); + } + } + private StreamingChatCompletionResults makeResults(List input) { var responseIter = input.stream().map(String::toUpperCase).iterator(); return new StreamingChatCompletionResults(subscriber -> { @@ -152,6 +172,59 @@ private ChunkedToXContent completionChunk(String delta) { ); } + private StreamingUnifiedChatCompletionResults makeUnifiedResults(UnifiedCompletionRequest request) { + var responseIter = request.messages().stream().map(message -> message.content().toString().toUpperCase()).iterator(); + return new StreamingUnifiedChatCompletionResults(subscriber -> { + subscriber.onSubscribe(new Flow.Subscription() { + @Override + public void request(long n) { + if (responseIter.hasNext()) { + subscriber.onNext(unifiedCompletionChunk(responseIter.next())); + } else { + subscriber.onComplete(); + } + } + + @Override + public void cancel() {} + }); + }); + } + + /* + The response format looks like this + { + "id": "chatcmpl-AarrzyuRflye7yzDF4lmVnenGmQCF", + "choices": [ + { + "delta": { + "content": " information" + }, + "index": 0 + } + ], + "model": "gpt-4o-2024-08-06", + "object": "chat.completion.chunk" + } + */ + private ChunkedToXContent unifiedCompletionChunk(String delta) { + return params -> Iterators.concat( + ChunkedToXContentHelper.startObject(), + ChunkedToXContentHelper.field("id", "id"), + ChunkedToXContentHelper.startArray("choices"), + ChunkedToXContentHelper.startObject(), + ChunkedToXContentHelper.startObject("delta"), + ChunkedToXContentHelper.field("content", delta), + ChunkedToXContentHelper.endObject(), + ChunkedToXContentHelper.field("index", 0), + ChunkedToXContentHelper.endObject(), + ChunkedToXContentHelper.endArray(), + ChunkedToXContentHelper.field("model", "gpt-4o-2024-08-06"), + ChunkedToXContentHelper.field("object", "chat.completion.chunk"), + ChunkedToXContentHelper.endObject() + ); + } + @Override public void chunkedInfer( Model model, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceNamedWriteablesProvider.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceNamedWriteablesProvider.java index 673b841317a3..a4187f4c4fa9 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceNamedWriteablesProvider.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferenceNamedWriteablesProvider.java @@ -16,6 +16,7 @@ import org.elasticsearch.inference.SecretSettings; import org.elasticsearch.inference.ServiceSettings; import org.elasticsearch.inference.TaskSettings; +import org.elasticsearch.inference.UnifiedCompletionRequest; import org.elasticsearch.xpack.core.inference.results.ChatCompletionResults; import org.elasticsearch.xpack.core.inference.results.ErrorChunkedInferenceResults; import org.elasticsearch.xpack.core.inference.results.InferenceChunkedSparseEmbeddingResults; @@ -137,11 +138,18 @@ public static List getNamedWriteables() { addEisNamedWriteables(namedWriteables); addAlibabaCloudSearchNamedWriteables(namedWriteables); + addUnifiedNamedWriteables(namedWriteables); + namedWriteables.addAll(StreamingTaskManager.namedWriteables()); return namedWriteables; } + private static void addUnifiedNamedWriteables(List namedWriteables) { + var writeables = UnifiedCompletionRequest.getNamedWriteables(); + namedWriteables.addAll(writeables); + } + private static void addAmazonBedrockNamedWriteables(List namedWriteables) { namedWriteables.add( new NamedWriteableRegistry.Entry( diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferencePlugin.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferencePlugin.java index d7d623ab2014..148a78445636 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferencePlugin.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/InferencePlugin.java @@ -51,6 +51,7 @@ import org.elasticsearch.xpack.core.inference.action.GetInferenceServicesAction; import org.elasticsearch.xpack.core.inference.action.InferenceAction; import org.elasticsearch.xpack.core.inference.action.PutInferenceModelAction; +import org.elasticsearch.xpack.core.inference.action.UnifiedCompletionAction; import org.elasticsearch.xpack.core.inference.action.UpdateInferenceModelAction; import org.elasticsearch.xpack.inference.action.TransportDeleteInferenceEndpointAction; import org.elasticsearch.xpack.inference.action.TransportGetInferenceDiagnosticsAction; @@ -59,6 +60,7 @@ import org.elasticsearch.xpack.inference.action.TransportInferenceAction; import org.elasticsearch.xpack.inference.action.TransportInferenceUsageAction; import org.elasticsearch.xpack.inference.action.TransportPutInferenceModelAction; +import org.elasticsearch.xpack.inference.action.TransportUnifiedCompletionInferenceAction; import org.elasticsearch.xpack.inference.action.TransportUpdateInferenceModelAction; import org.elasticsearch.xpack.inference.action.filter.ShardBulkInferenceActionFilter; import org.elasticsearch.xpack.inference.common.Truncator; @@ -86,6 +88,7 @@ import org.elasticsearch.xpack.inference.rest.RestInferenceAction; import org.elasticsearch.xpack.inference.rest.RestPutInferenceModelAction; import org.elasticsearch.xpack.inference.rest.RestStreamInferenceAction; +import org.elasticsearch.xpack.inference.rest.RestUnifiedCompletionInferenceAction; import org.elasticsearch.xpack.inference.rest.RestUpdateInferenceModelAction; import org.elasticsearch.xpack.inference.services.ServiceComponents; import org.elasticsearch.xpack.inference.services.alibabacloudsearch.AlibabaCloudSearchService; @@ -159,8 +162,9 @@ public InferencePlugin(Settings settings) { @Override public List> getActions() { - return List.of( + var availableActions = List.of( new ActionHandler<>(InferenceAction.INSTANCE, TransportInferenceAction.class), + new ActionHandler<>(GetInferenceModelAction.INSTANCE, TransportGetInferenceModelAction.class), new ActionHandler<>(PutInferenceModelAction.INSTANCE, TransportPutInferenceModelAction.class), new ActionHandler<>(UpdateInferenceModelAction.INSTANCE, TransportUpdateInferenceModelAction.class), @@ -169,6 +173,13 @@ public InferencePlugin(Settings settings) { new ActionHandler<>(GetInferenceDiagnosticsAction.INSTANCE, TransportGetInferenceDiagnosticsAction.class), new ActionHandler<>(GetInferenceServicesAction.INSTANCE, TransportGetInferenceServicesAction.class) ); + + List> conditionalActions = + UnifiedCompletionFeature.UNIFIED_COMPLETION_FEATURE_FLAG.isEnabled() + ? List.of(new ActionHandler<>(UnifiedCompletionAction.INSTANCE, TransportUnifiedCompletionInferenceAction.class)) + : List.of(); + + return Stream.concat(availableActions.stream(), conditionalActions.stream()).toList(); } @Override @@ -183,7 +194,7 @@ public List getRestHandlers( Supplier nodesInCluster, Predicate clusterSupportsFeature ) { - return List.of( + var availableRestActions = List.of( new RestInferenceAction(), new RestStreamInferenceAction(), new RestGetInferenceModelAction(), @@ -193,6 +204,11 @@ public List getRestHandlers( new RestGetInferenceDiagnosticsAction(), new RestGetInferenceServicesAction() ); + List conditionalRestActions = UnifiedCompletionFeature.UNIFIED_COMPLETION_FEATURE_FLAG.isEnabled() + ? List.of(new RestUnifiedCompletionInferenceAction()) + : List.of(); + + return Stream.concat(availableRestActions.stream(), conditionalRestActions.stream()).toList(); } @Override diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/UnifiedCompletionFeature.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/UnifiedCompletionFeature.java new file mode 100644 index 000000000000..3e13d0c1e39d --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/UnifiedCompletionFeature.java @@ -0,0 +1,20 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference; + +import org.elasticsearch.common.util.FeatureFlag; + +/** + * Unified Completion feature flag. When the feature is complete, this flag will be removed. + * Enable feature via JVM option: `-Des.inference_unified_feature_flag_enabled=true`. + */ +public class UnifiedCompletionFeature { + public static final FeatureFlag UNIFIED_COMPLETION_FEATURE_FLAG = new FeatureFlag("inference_unified"); + + private UnifiedCompletionFeature() {} +} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/BaseTransportInferenceAction.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/BaseTransportInferenceAction.java new file mode 100644 index 000000000000..2a0e8e177527 --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/BaseTransportInferenceAction.java @@ -0,0 +1,250 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference.action; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.common.xcontent.ChunkedToXContent; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.inference.InferenceService; +import org.elasticsearch.inference.InferenceServiceRegistry; +import org.elasticsearch.inference.InferenceServiceResults; +import org.elasticsearch.inference.Model; +import org.elasticsearch.inference.TaskType; +import org.elasticsearch.inference.UnparsedModel; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.core.inference.action.BaseInferenceActionRequest; +import org.elasticsearch.xpack.core.inference.action.InferenceAction; +import org.elasticsearch.xpack.inference.action.task.StreamingTaskManager; +import org.elasticsearch.xpack.inference.common.DelegatingProcessor; +import org.elasticsearch.xpack.inference.registry.ModelRegistry; +import org.elasticsearch.xpack.inference.telemetry.InferenceStats; +import org.elasticsearch.xpack.inference.telemetry.InferenceTimer; + +import java.util.function.Supplier; +import java.util.stream.Collectors; + +import static org.elasticsearch.core.Strings.format; +import static org.elasticsearch.xpack.inference.telemetry.InferenceStats.modelAttributes; +import static org.elasticsearch.xpack.inference.telemetry.InferenceStats.responseAttributes; + +public abstract class BaseTransportInferenceAction extends HandledTransportAction< + Request, + InferenceAction.Response> { + + private static final Logger log = LogManager.getLogger(BaseTransportInferenceAction.class); + private static final String STREAMING_INFERENCE_TASK_TYPE = "streaming_inference"; + private static final String STREAMING_TASK_ACTION = "xpack/inference/streaming_inference[n]"; + private final ModelRegistry modelRegistry; + private final InferenceServiceRegistry serviceRegistry; + private final InferenceStats inferenceStats; + private final StreamingTaskManager streamingTaskManager; + + public BaseTransportInferenceAction( + String inferenceActionName, + TransportService transportService, + ActionFilters actionFilters, + ModelRegistry modelRegistry, + InferenceServiceRegistry serviceRegistry, + InferenceStats inferenceStats, + StreamingTaskManager streamingTaskManager, + Writeable.Reader requestReader + ) { + super(inferenceActionName, transportService, actionFilters, requestReader, EsExecutors.DIRECT_EXECUTOR_SERVICE); + this.modelRegistry = modelRegistry; + this.serviceRegistry = serviceRegistry; + this.inferenceStats = inferenceStats; + this.streamingTaskManager = streamingTaskManager; + } + + @Override + protected void doExecute(Task task, Request request, ActionListener listener) { + var timer = InferenceTimer.start(); + + var getModelListener = ActionListener.wrap((UnparsedModel unparsedModel) -> { + var service = serviceRegistry.getService(unparsedModel.service()); + try { + validationHelper(service::isEmpty, () -> unknownServiceException(unparsedModel.service(), request.getInferenceEntityId())); + validationHelper( + () -> request.getTaskType().isAnyOrSame(unparsedModel.taskType()) == false, + () -> requestModelTaskTypeMismatchException(request.getTaskType(), unparsedModel.taskType()) + ); + validationHelper( + () -> isInvalidTaskTypeForInferenceEndpoint(request, unparsedModel), + () -> createInvalidTaskTypeException(request, unparsedModel) + ); + } catch (Exception e) { + recordMetrics(unparsedModel, timer, e); + listener.onFailure(e); + return; + } + + var model = service.get() + .parsePersistedConfigWithSecrets( + unparsedModel.inferenceEntityId(), + unparsedModel.taskType(), + unparsedModel.settings(), + unparsedModel.secrets() + ); + inferOnServiceWithMetrics(model, request, service.get(), timer, listener); + }, e -> { + try { + inferenceStats.inferenceDuration().record(timer.elapsedMillis(), responseAttributes(e)); + } catch (Exception metricsException) { + log.atDebug().withThrowable(metricsException).log("Failed to record metrics when the model is missing, dropping metrics"); + } + listener.onFailure(e); + }); + + modelRegistry.getModelWithSecrets(request.getInferenceEntityId(), getModelListener); + } + + private static void validationHelper(Supplier validationFailure, Supplier exceptionCreator) { + if (validationFailure.get()) { + throw exceptionCreator.get(); + } + } + + protected abstract boolean isInvalidTaskTypeForInferenceEndpoint(Request request, UnparsedModel unparsedModel); + + protected abstract ElasticsearchStatusException createInvalidTaskTypeException(Request request, UnparsedModel unparsedModel); + + private void recordMetrics(UnparsedModel model, InferenceTimer timer, @Nullable Throwable t) { + try { + inferenceStats.inferenceDuration().record(timer.elapsedMillis(), responseAttributes(model, t)); + } catch (Exception e) { + log.atDebug().withThrowable(e).log("Failed to record metrics with an unparsed model, dropping metrics"); + } + } + + private void inferOnServiceWithMetrics( + Model model, + Request request, + InferenceService service, + InferenceTimer timer, + ActionListener listener + ) { + inferenceStats.requestCount().incrementBy(1, modelAttributes(model)); + inferOnService(model, request, service, ActionListener.wrap(inferenceResults -> { + if (request.isStreaming()) { + var taskProcessor = streamingTaskManager.create(STREAMING_INFERENCE_TASK_TYPE, STREAMING_TASK_ACTION); + inferenceResults.publisher().subscribe(taskProcessor); + + var instrumentedStream = new PublisherWithMetrics(timer, model); + taskProcessor.subscribe(instrumentedStream); + + listener.onResponse(new InferenceAction.Response(inferenceResults, instrumentedStream)); + } else { + recordMetrics(model, timer, null); + listener.onResponse(new InferenceAction.Response(inferenceResults)); + } + }, e -> { + recordMetrics(model, timer, e); + listener.onFailure(e); + })); + } + + private void recordMetrics(Model model, InferenceTimer timer, @Nullable Throwable t) { + try { + inferenceStats.inferenceDuration().record(timer.elapsedMillis(), responseAttributes(model, t)); + } catch (Exception e) { + log.atDebug().withThrowable(e).log("Failed to record metrics with a parsed model, dropping metrics"); + } + } + + private void inferOnService(Model model, Request request, InferenceService service, ActionListener listener) { + if (request.isStreaming() == false || service.canStream(request.getTaskType())) { + doInference(model, request, service, listener); + } else { + listener.onFailure(unsupportedStreamingTaskException(request, service)); + } + } + + protected abstract void doInference( + Model model, + Request request, + InferenceService service, + ActionListener listener + ); + + private ElasticsearchStatusException unsupportedStreamingTaskException(Request request, InferenceService service) { + var supportedTasks = service.supportedStreamingTasks(); + if (supportedTasks.isEmpty()) { + return new ElasticsearchStatusException( + format("Streaming is not allowed for service [%s].", service.name()), + RestStatus.METHOD_NOT_ALLOWED + ); + } else { + var validTasks = supportedTasks.stream().map(TaskType::toString).collect(Collectors.joining(",")); + return new ElasticsearchStatusException( + format( + "Streaming is not allowed for service [%s] and task [%s]. Supported tasks: [%s]", + service.name(), + request.getTaskType(), + validTasks + ), + RestStatus.METHOD_NOT_ALLOWED + ); + } + } + + private static ElasticsearchStatusException unknownServiceException(String service, String inferenceId) { + return new ElasticsearchStatusException("Unknown service [{}] for model [{}]. ", RestStatus.BAD_REQUEST, service, inferenceId); + } + + private static ElasticsearchStatusException requestModelTaskTypeMismatchException(TaskType requested, TaskType expected) { + return new ElasticsearchStatusException( + "Incompatible task_type, the requested type [{}] does not match the model type [{}]", + RestStatus.BAD_REQUEST, + requested, + expected + ); + } + + private class PublisherWithMetrics extends DelegatingProcessor { + + private final InferenceTimer timer; + private final Model model; + + private PublisherWithMetrics(InferenceTimer timer, Model model) { + this.timer = timer; + this.model = model; + } + + @Override + protected void next(ChunkedToXContent item) { + downstream().onNext(item); + } + + @Override + public void onError(Throwable throwable) { + recordMetrics(model, timer, throwable); + super.onError(throwable); + } + + @Override + protected void onCancel() { + recordMetrics(model, timer, null); + super.onCancel(); + } + + @Override + public void onComplete() { + recordMetrics(model, timer, null); + super.onComplete(); + } + } +} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/TransportInferenceAction.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/TransportInferenceAction.java index ba9ab3c13373..08e6d869a553 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/TransportInferenceAction.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/TransportInferenceAction.java @@ -7,47 +7,22 @@ package org.elasticsearch.xpack.inference.action; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; -import org.elasticsearch.action.support.HandledTransportAction; -import org.elasticsearch.common.util.concurrent.EsExecutors; -import org.elasticsearch.common.xcontent.ChunkedToXContent; -import org.elasticsearch.core.Nullable; import org.elasticsearch.inference.InferenceService; import org.elasticsearch.inference.InferenceServiceRegistry; import org.elasticsearch.inference.InferenceServiceResults; import org.elasticsearch.inference.Model; -import org.elasticsearch.inference.TaskType; import org.elasticsearch.inference.UnparsedModel; import org.elasticsearch.injection.guice.Inject; -import org.elasticsearch.rest.RestStatus; -import org.elasticsearch.tasks.Task; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xpack.core.inference.action.InferenceAction; import org.elasticsearch.xpack.inference.action.task.StreamingTaskManager; -import org.elasticsearch.xpack.inference.common.DelegatingProcessor; import org.elasticsearch.xpack.inference.registry.ModelRegistry; import org.elasticsearch.xpack.inference.telemetry.InferenceStats; -import org.elasticsearch.xpack.inference.telemetry.InferenceTimer; -import java.util.stream.Collectors; - -import static org.elasticsearch.core.Strings.format; -import static org.elasticsearch.xpack.inference.telemetry.InferenceStats.modelAttributes; -import static org.elasticsearch.xpack.inference.telemetry.InferenceStats.responseAttributes; - -public class TransportInferenceAction extends HandledTransportAction { - private static final Logger log = LogManager.getLogger(TransportInferenceAction.class); - private static final String STREAMING_INFERENCE_TASK_TYPE = "streaming_inference"; - private static final String STREAMING_TASK_ACTION = "xpack/inference/streaming_inference[n]"; - - private final ModelRegistry modelRegistry; - private final InferenceServiceRegistry serviceRegistry; - private final InferenceStats inferenceStats; - private final StreamingTaskManager streamingTaskManager; +public class TransportInferenceAction extends BaseTransportInferenceAction { @Inject public TransportInferenceAction( @@ -58,184 +33,44 @@ public TransportInferenceAction( InferenceStats inferenceStats, StreamingTaskManager streamingTaskManager ) { - super(InferenceAction.NAME, transportService, actionFilters, InferenceAction.Request::new, EsExecutors.DIRECT_EXECUTOR_SERVICE); - this.modelRegistry = modelRegistry; - this.serviceRegistry = serviceRegistry; - this.inferenceStats = inferenceStats; - this.streamingTaskManager = streamingTaskManager; + super( + InferenceAction.NAME, + transportService, + actionFilters, + modelRegistry, + serviceRegistry, + inferenceStats, + streamingTaskManager, + InferenceAction.Request::new + ); } @Override - protected void doExecute(Task task, InferenceAction.Request request, ActionListener listener) { - var timer = InferenceTimer.start(); - - var getModelListener = ActionListener.wrap((UnparsedModel unparsedModel) -> { - var service = serviceRegistry.getService(unparsedModel.service()); - if (service.isEmpty()) { - var e = unknownServiceException(unparsedModel.service(), request.getInferenceEntityId()); - recordMetrics(unparsedModel, timer, e); - listener.onFailure(e); - return; - } - - if (request.getTaskType().isAnyOrSame(unparsedModel.taskType()) == false) { - // not the wildcard task type and not the model task type - var e = incompatibleTaskTypeException(request.getTaskType(), unparsedModel.taskType()); - recordMetrics(unparsedModel, timer, e); - listener.onFailure(e); - return; - } - - var model = service.get() - .parsePersistedConfigWithSecrets( - unparsedModel.inferenceEntityId(), - unparsedModel.taskType(), - unparsedModel.settings(), - unparsedModel.secrets() - ); - inferOnServiceWithMetrics(model, request, service.get(), timer, listener); - }, e -> { - try { - inferenceStats.inferenceDuration().record(timer.elapsedMillis(), responseAttributes(e)); - } catch (Exception metricsException) { - log.atDebug().withThrowable(metricsException).log("Failed to record metrics when the model is missing, dropping metrics"); - } - listener.onFailure(e); - }); - - modelRegistry.getModelWithSecrets(request.getInferenceEntityId(), getModelListener); - } - - private void recordMetrics(UnparsedModel model, InferenceTimer timer, @Nullable Throwable t) { - try { - inferenceStats.inferenceDuration().record(timer.elapsedMillis(), responseAttributes(model, t)); - } catch (Exception e) { - log.atDebug().withThrowable(e).log("Failed to record metrics with an unparsed model, dropping metrics"); - } - } - - private void inferOnServiceWithMetrics( - Model model, - InferenceAction.Request request, - InferenceService service, - InferenceTimer timer, - ActionListener listener - ) { - inferenceStats.requestCount().incrementBy(1, modelAttributes(model)); - inferOnService(model, request, service, ActionListener.wrap(inferenceResults -> { - if (request.isStreaming()) { - var taskProcessor = streamingTaskManager.create(STREAMING_INFERENCE_TASK_TYPE, STREAMING_TASK_ACTION); - inferenceResults.publisher().subscribe(taskProcessor); - - var instrumentedStream = new PublisherWithMetrics(timer, model); - taskProcessor.subscribe(instrumentedStream); - - listener.onResponse(new InferenceAction.Response(inferenceResults, instrumentedStream)); - } else { - recordMetrics(model, timer, null); - listener.onResponse(new InferenceAction.Response(inferenceResults)); - } - }, e -> { - recordMetrics(model, timer, e); - listener.onFailure(e); - })); + protected boolean isInvalidTaskTypeForInferenceEndpoint(InferenceAction.Request request, UnparsedModel unparsedModel) { + return false; } - private void recordMetrics(Model model, InferenceTimer timer, @Nullable Throwable t) { - try { - inferenceStats.inferenceDuration().record(timer.elapsedMillis(), responseAttributes(model, t)); - } catch (Exception e) { - log.atDebug().withThrowable(e).log("Failed to record metrics with a parsed model, dropping metrics"); - } + @Override + protected ElasticsearchStatusException createInvalidTaskTypeException(InferenceAction.Request request, UnparsedModel unparsedModel) { + return null; } - private void inferOnService( + @Override + protected void doInference( Model model, InferenceAction.Request request, InferenceService service, ActionListener listener ) { - if (request.isStreaming() == false || service.canStream(request.getTaskType())) { - service.infer( - model, - request.getQuery(), - request.getInput(), - request.isStreaming(), - request.getTaskSettings(), - request.getInputType(), - request.getInferenceTimeout(), - listener - ); - } else { - listener.onFailure(unsupportedStreamingTaskException(request, service)); - } - } - - private ElasticsearchStatusException unsupportedStreamingTaskException(InferenceAction.Request request, InferenceService service) { - var supportedTasks = service.supportedStreamingTasks(); - if (supportedTasks.isEmpty()) { - return new ElasticsearchStatusException( - format("Streaming is not allowed for service [%s].", service.name()), - RestStatus.METHOD_NOT_ALLOWED - ); - } else { - var validTasks = supportedTasks.stream().map(TaskType::toString).collect(Collectors.joining(",")); - return new ElasticsearchStatusException( - format( - "Streaming is not allowed for service [%s] and task [%s]. Supported tasks: [%s]", - service.name(), - request.getTaskType(), - validTasks - ), - RestStatus.METHOD_NOT_ALLOWED - ); - } - } - - private static ElasticsearchStatusException unknownServiceException(String service, String inferenceId) { - return new ElasticsearchStatusException("Unknown service [{}] for model [{}]. ", RestStatus.BAD_REQUEST, service, inferenceId); - } - - private static ElasticsearchStatusException incompatibleTaskTypeException(TaskType requested, TaskType expected) { - return new ElasticsearchStatusException( - "Incompatible task_type, the requested type [{}] does not match the model type [{}]", - RestStatus.BAD_REQUEST, - requested, - expected + service.infer( + model, + request.getQuery(), + request.getInput(), + request.isStreaming(), + request.getTaskSettings(), + request.getInputType(), + request.getInferenceTimeout(), + listener ); } - - private class PublisherWithMetrics extends DelegatingProcessor { - private final InferenceTimer timer; - private final Model model; - - private PublisherWithMetrics(InferenceTimer timer, Model model) { - this.timer = timer; - this.model = model; - } - - @Override - protected void next(ChunkedToXContent item) { - downstream().onNext(item); - } - - @Override - public void onError(Throwable throwable) { - recordMetrics(model, timer, throwable); - super.onError(throwable); - } - - @Override - protected void onCancel() { - recordMetrics(model, timer, null); - super.onCancel(); - } - - @Override - public void onComplete() { - recordMetrics(model, timer, null); - super.onComplete(); - } - } - } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/TransportUnifiedCompletionInferenceAction.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/TransportUnifiedCompletionInferenceAction.java new file mode 100644 index 000000000000..f0906231d8f4 --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/action/TransportUnifiedCompletionInferenceAction.java @@ -0,0 +1,77 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference.action; + +import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.inference.InferenceService; +import org.elasticsearch.inference.InferenceServiceRegistry; +import org.elasticsearch.inference.InferenceServiceResults; +import org.elasticsearch.inference.Model; +import org.elasticsearch.inference.TaskType; +import org.elasticsearch.inference.UnparsedModel; +import org.elasticsearch.injection.guice.Inject; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.core.inference.action.UnifiedCompletionAction; +import org.elasticsearch.xpack.inference.action.task.StreamingTaskManager; +import org.elasticsearch.xpack.inference.registry.ModelRegistry; +import org.elasticsearch.xpack.inference.telemetry.InferenceStats; + +public class TransportUnifiedCompletionInferenceAction extends BaseTransportInferenceAction { + + @Inject + public TransportUnifiedCompletionInferenceAction( + TransportService transportService, + ActionFilters actionFilters, + ModelRegistry modelRegistry, + InferenceServiceRegistry serviceRegistry, + InferenceStats inferenceStats, + StreamingTaskManager streamingTaskManager + ) { + super( + UnifiedCompletionAction.NAME, + transportService, + actionFilters, + modelRegistry, + serviceRegistry, + inferenceStats, + streamingTaskManager, + UnifiedCompletionAction.Request::new + ); + } + + @Override + protected boolean isInvalidTaskTypeForInferenceEndpoint(UnifiedCompletionAction.Request request, UnparsedModel unparsedModel) { + return request.getTaskType().isAnyOrSame(TaskType.COMPLETION) == false || unparsedModel.taskType() != TaskType.COMPLETION; + } + + @Override + protected ElasticsearchStatusException createInvalidTaskTypeException( + UnifiedCompletionAction.Request request, + UnparsedModel unparsedModel + ) { + return new ElasticsearchStatusException( + "Incompatible task_type for unified API, the requested type [{}] must be one of [{}]", + RestStatus.BAD_REQUEST, + request.getTaskType(), + TaskType.COMPLETION.toString() + ); + } + + @Override + protected void doInference( + Model model, + UnifiedCompletionAction.Request request, + InferenceService service, + ActionListener listener + ) { + service.unifiedCompletionInfer(model, request.getUnifiedCompletionRequest(), null, listener); + } +} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/common/DelegatingProcessor.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/common/DelegatingProcessor.java index 03e794e42c3a..eda3fc0f3bfd 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/common/DelegatingProcessor.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/common/DelegatingProcessor.java @@ -9,7 +9,14 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; - +import org.elasticsearch.xcontent.XContentParserConfiguration; +import org.elasticsearch.xpack.inference.external.response.streaming.ServerSentEvent; +import org.elasticsearch.xpack.inference.external.response.streaming.ServerSentEventField; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.Deque; +import java.util.Iterator; import java.util.concurrent.Flow; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; @@ -25,6 +32,33 @@ public abstract class DelegatingProcessor implements Flow.Processor private Flow.Subscriber downstream; private Flow.Subscription upstream; + public static Deque parseEvent( + Deque item, + ParseChunkFunction parseFunction, + XContentParserConfiguration parserConfig, + Logger logger + ) throws Exception { + var results = new ArrayDeque(item.size()); + for (ServerSentEvent event : item) { + if (ServerSentEventField.DATA == event.name() && event.hasValue()) { + try { + var delta = parseFunction.apply(parserConfig, event); + delta.forEachRemaining(results::offer); + } catch (Exception e) { + logger.warn("Failed to parse event from inference provider: {}", event); + throw e; + } + } + } + + return results; + } + + @FunctionalInterface + public interface ParseChunkFunction { + Iterator apply(XContentParserConfiguration parserConfig, ServerSentEvent event) throws IOException; + } + @Override public void subscribe(Flow.Subscriber subscriber) { if (downstream != null) { @@ -51,7 +85,7 @@ public void request(long n) { if (isClosed.get()) { downstream.onComplete(); } else if (upstream != null) { - upstream.request(n); + upstreamRequest(n); } else { pendingRequests.accumulateAndGet(n, Long::sum); } @@ -67,6 +101,13 @@ public void cancel() { }; } + /** + * Guaranteed to be called when the upstream is set and this processor had not been closed. + */ + protected void upstreamRequest(long n) { + upstream.request(n); + } + protected void onCancel() {} @Override diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/SingleInputSenderExecutableAction.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/SingleInputSenderExecutableAction.java index 4e97554b5644..b43e5ab70e2f 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/SingleInputSenderExecutableAction.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/SingleInputSenderExecutableAction.java @@ -12,7 +12,6 @@ import org.elasticsearch.core.TimeValue; import org.elasticsearch.inference.InferenceServiceResults; import org.elasticsearch.rest.RestStatus; -import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.InferenceInputs; import org.elasticsearch.xpack.inference.external.http.sender.RequestManager; import org.elasticsearch.xpack.inference.external.http.sender.Sender; @@ -34,13 +33,7 @@ public SingleInputSenderExecutableAction( @Override public void execute(InferenceInputs inferenceInputs, TimeValue timeout, ActionListener listener) { - if (inferenceInputs instanceof DocumentsOnlyInput == false) { - listener.onFailure(new ElasticsearchStatusException("Invalid inference input type", RestStatus.INTERNAL_SERVER_ERROR)); - return; - } - - var docsOnlyInput = (DocumentsOnlyInput) inferenceInputs; - if (docsOnlyInput.getInputs().size() > 1) { + if (inferenceInputs.inputSize() > 1) { listener.onFailure( new ElasticsearchStatusException(requestTypeForInputValidationError + " only accepts 1 input", RestStatus.BAD_REQUEST) ); diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/openai/OpenAiActionCreator.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/openai/OpenAiActionCreator.java index 9c83264b5581..bd5c53d589df 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/openai/OpenAiActionCreator.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/action/openai/OpenAiActionCreator.java @@ -26,7 +26,7 @@ * Provides a way to construct an {@link ExecutableAction} using the visitor pattern based on the openai model type. */ public class OpenAiActionCreator implements OpenAiActionVisitor { - private static final String COMPLETION_ERROR_PREFIX = "OpenAI chat completions"; + public static final String COMPLETION_ERROR_PREFIX = "OpenAI chat completions"; private final Sender sender; private final ServiceComponents serviceComponents; diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AlibabaCloudSearchCompletionRequestManager.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AlibabaCloudSearchCompletionRequestManager.java index a0a44e62f9f7..e7a960f1316f 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AlibabaCloudSearchCompletionRequestManager.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AlibabaCloudSearchCompletionRequestManager.java @@ -69,7 +69,7 @@ public void execute( Supplier hasRequestCompletedFunction, ActionListener listener ) { - List input = DocumentsOnlyInput.of(inferenceInputs).getInputs(); + List input = inferenceInputs.castTo(ChatCompletionInput.class).getInputs(); AlibabaCloudSearchCompletionRequest request = new AlibabaCloudSearchCompletionRequest(account, input, model); execute(new ExecutableInferenceRequest(requestSender, logger, request, HANDLER, hasRequestCompletedFunction, listener)); } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AmazonBedrockChatCompletionRequestManager.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AmazonBedrockChatCompletionRequestManager.java index 69a5c665feb8..3929585a0745 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AmazonBedrockChatCompletionRequestManager.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AmazonBedrockChatCompletionRequestManager.java @@ -44,10 +44,10 @@ public void execute( Supplier hasRequestCompletedFunction, ActionListener listener ) { - var docsOnly = DocumentsOnlyInput.of(inferenceInputs); - var docsInput = docsOnly.getInputs(); - var stream = docsOnly.stream(); - var requestEntity = AmazonBedrockChatCompletionEntityFactory.createEntity(model, docsInput); + var chatCompletionInput = inferenceInputs.castTo(ChatCompletionInput.class); + var inputs = chatCompletionInput.getInputs(); + var stream = chatCompletionInput.stream(); + var requestEntity = AmazonBedrockChatCompletionEntityFactory.createEntity(model, inputs); var request = new AmazonBedrockChatCompletionRequest(model, requestEntity, timeout, stream); var responseHandler = new AmazonBedrockChatCompletionResponseHandler(); diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AnthropicCompletionRequestManager.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AnthropicCompletionRequestManager.java index 5418b3dd9840..6d4aeb9e31ba 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AnthropicCompletionRequestManager.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AnthropicCompletionRequestManager.java @@ -46,10 +46,10 @@ public void execute( Supplier hasRequestCompletedFunction, ActionListener listener ) { - var docsOnly = DocumentsOnlyInput.of(inferenceInputs); - var docsInput = docsOnly.getInputs(); - var stream = docsOnly.stream(); - AnthropicChatCompletionRequest request = new AnthropicChatCompletionRequest(docsInput, model, stream); + var chatCompletionInput = inferenceInputs.castTo(ChatCompletionInput.class); + var inputs = chatCompletionInput.getInputs(); + var stream = chatCompletionInput.stream(); + AnthropicChatCompletionRequest request = new AnthropicChatCompletionRequest(inputs, model, stream); execute(new ExecutableInferenceRequest(requestSender, logger, request, HANDLER, hasRequestCompletedFunction, listener)); } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AzureAiStudioChatCompletionRequestManager.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AzureAiStudioChatCompletionRequestManager.java index 21cec68b14a4..affd2e3a7760 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AzureAiStudioChatCompletionRequestManager.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AzureAiStudioChatCompletionRequestManager.java @@ -41,10 +41,10 @@ public void execute( Supplier hasRequestCompletedFunction, ActionListener listener ) { - var docsOnly = DocumentsOnlyInput.of(inferenceInputs); - var docsInput = docsOnly.getInputs(); - var stream = docsOnly.stream(); - AzureAiStudioChatCompletionRequest request = new AzureAiStudioChatCompletionRequest(model, docsInput, stream); + var chatCompletionInput = inferenceInputs.castTo(ChatCompletionInput.class); + var inputs = chatCompletionInput.getInputs(); + var stream = chatCompletionInput.stream(); + AzureAiStudioChatCompletionRequest request = new AzureAiStudioChatCompletionRequest(model, inputs, stream); execute(new ExecutableInferenceRequest(requestSender, logger, request, HANDLER, hasRequestCompletedFunction, listener)); } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AzureOpenAiCompletionRequestManager.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AzureOpenAiCompletionRequestManager.java index d036559ec3dc..c2f5f3e9db5e 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AzureOpenAiCompletionRequestManager.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/AzureOpenAiCompletionRequestManager.java @@ -46,10 +46,10 @@ public void execute( Supplier hasRequestCompletedFunction, ActionListener listener ) { - var docsOnly = DocumentsOnlyInput.of(inferenceInputs); - var docsInput = docsOnly.getInputs(); - var stream = docsOnly.stream(); - AzureOpenAiCompletionRequest request = new AzureOpenAiCompletionRequest(docsInput, model, stream); + var chatCompletionInput = inferenceInputs.castTo(ChatCompletionInput.class); + var inputs = chatCompletionInput.getInputs(); + var stream = chatCompletionInput.stream(); + AzureOpenAiCompletionRequest request = new AzureOpenAiCompletionRequest(inputs, model, stream); execute(new ExecutableInferenceRequest(requestSender, logger, request, HANDLER, hasRequestCompletedFunction, listener)); } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/ChatCompletionInput.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/ChatCompletionInput.java new file mode 100644 index 000000000000..928da95d9c2f --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/ChatCompletionInput.java @@ -0,0 +1,39 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference.external.http.sender; + +import java.util.List; +import java.util.Objects; + +/** + * This class encapsulates the input text passed by the request and indicates whether the response should be streamed. + * The main difference between this class and {@link UnifiedChatInput} is this should only be used for + * {@link org.elasticsearch.inference.TaskType#COMPLETION} originating through the + * {@link org.elasticsearch.inference.InferenceService#infer} code path. These are requests sent to the + * API without using the _unified route. + */ +public class ChatCompletionInput extends InferenceInputs { + private final List input; + + public ChatCompletionInput(List input) { + this(input, false); + } + + public ChatCompletionInput(List input, boolean stream) { + super(stream); + this.input = Objects.requireNonNull(input); + } + + public List getInputs() { + return this.input; + } + + public int inputSize() { + return input.size(); + } +} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/CohereCompletionRequestManager.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/CohereCompletionRequestManager.java index ae46fbe0fef8..40cd03c87664 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/CohereCompletionRequestManager.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/CohereCompletionRequestManager.java @@ -50,10 +50,10 @@ public void execute( Supplier hasRequestCompletedFunction, ActionListener listener ) { - var docsOnly = DocumentsOnlyInput.of(inferenceInputs); - var docsInput = docsOnly.getInputs(); - var stream = docsOnly.stream(); - CohereCompletionRequest request = new CohereCompletionRequest(docsInput, model, stream); + var chatCompletionInput = inferenceInputs.castTo(ChatCompletionInput.class); + var inputs = chatCompletionInput.getInputs(); + var stream = chatCompletionInput.stream(); + CohereCompletionRequest request = new CohereCompletionRequest(inputs, model, stream); execute(new ExecutableInferenceRequest(requestSender, logger, request, HANDLER, hasRequestCompletedFunction, listener)); } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/DocumentsOnlyInput.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/DocumentsOnlyInput.java index 8cf411d84c93..3feb79d3de6c 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/DocumentsOnlyInput.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/DocumentsOnlyInput.java @@ -14,30 +14,28 @@ public class DocumentsOnlyInput extends InferenceInputs { public static DocumentsOnlyInput of(InferenceInputs inferenceInputs) { if (inferenceInputs instanceof DocumentsOnlyInput == false) { - throw createUnsupportedTypeException(inferenceInputs); + throw createUnsupportedTypeException(inferenceInputs, DocumentsOnlyInput.class); } return (DocumentsOnlyInput) inferenceInputs; } private final List input; - private final boolean stream; public DocumentsOnlyInput(List input) { this(input, false); } public DocumentsOnlyInput(List input, boolean stream) { - super(); + super(stream); this.input = Objects.requireNonNull(input); - this.stream = stream; } public List getInputs() { return this.input; } - public boolean stream() { - return stream; + public int inputSize() { + return input.size(); } } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/GoogleAiStudioCompletionRequestManager.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/GoogleAiStudioCompletionRequestManager.java index abe50c6fae3f..0097f9c08ea2 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/GoogleAiStudioCompletionRequestManager.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/GoogleAiStudioCompletionRequestManager.java @@ -51,7 +51,10 @@ public void execute( Supplier hasRequestCompletedFunction, ActionListener listener ) { - GoogleAiStudioCompletionRequest request = new GoogleAiStudioCompletionRequest(DocumentsOnlyInput.of(inferenceInputs), model); + GoogleAiStudioCompletionRequest request = new GoogleAiStudioCompletionRequest( + inferenceInputs.castTo(ChatCompletionInput.class), + model + ); execute(new ExecutableInferenceRequest(requestSender, logger, request, HANDLER, hasRequestCompletedFunction, listener)); } } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/InferenceInputs.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/InferenceInputs.java index dd241857ef0c..e85ea6f1d9b3 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/InferenceInputs.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/InferenceInputs.java @@ -10,7 +10,29 @@ import org.elasticsearch.common.Strings; public abstract class InferenceInputs { - public static IllegalArgumentException createUnsupportedTypeException(InferenceInputs inferenceInputs) { - return new IllegalArgumentException(Strings.format("Unsupported inference inputs type: [%s]", inferenceInputs.getClass())); + private final boolean stream; + + public InferenceInputs(boolean stream) { + this.stream = stream; + } + + public static IllegalArgumentException createUnsupportedTypeException(InferenceInputs inferenceInputs, Class clazz) { + return new IllegalArgumentException( + Strings.format("Unable to convert inference inputs type: [%s] to [%s]", inferenceInputs.getClass(), clazz) + ); } + + public T castTo(Class clazz) { + if (clazz.isInstance(this) == false) { + throw createUnsupportedTypeException(this, clazz); + } + + return clazz.cast(this); + } + + public boolean stream() { + return stream; + } + + public abstract int inputSize(); } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/OpenAiCompletionRequestManager.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/OpenAiCompletionRequestManager.java index cea89332e5bf..4d730be6aa6b 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/OpenAiCompletionRequestManager.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/OpenAiCompletionRequestManager.java @@ -15,7 +15,7 @@ import org.elasticsearch.xpack.inference.external.http.retry.RequestSender; import org.elasticsearch.xpack.inference.external.http.retry.ResponseHandler; import org.elasticsearch.xpack.inference.external.openai.OpenAiChatCompletionResponseHandler; -import org.elasticsearch.xpack.inference.external.request.openai.OpenAiChatCompletionRequest; +import org.elasticsearch.xpack.inference.external.request.openai.OpenAiUnifiedChatCompletionRequest; import org.elasticsearch.xpack.inference.external.response.openai.OpenAiChatCompletionResponseEntity; import org.elasticsearch.xpack.inference.services.openai.completion.OpenAiChatCompletionModel; @@ -25,8 +25,8 @@ public class OpenAiCompletionRequestManager extends OpenAiRequestManager { private static final Logger logger = LogManager.getLogger(OpenAiCompletionRequestManager.class); - private static final ResponseHandler HANDLER = createCompletionHandler(); + static final String USER_ROLE = "user"; public static OpenAiCompletionRequestManager of(OpenAiChatCompletionModel model, ThreadPool threadPool) { return new OpenAiCompletionRequestManager(Objects.requireNonNull(model), Objects.requireNonNull(threadPool)); @@ -35,7 +35,7 @@ public static OpenAiCompletionRequestManager of(OpenAiChatCompletionModel model, private final OpenAiChatCompletionModel model; private OpenAiCompletionRequestManager(OpenAiChatCompletionModel model, ThreadPool threadPool) { - super(threadPool, model, OpenAiChatCompletionRequest::buildDefaultUri); + super(threadPool, model, OpenAiUnifiedChatCompletionRequest::buildDefaultUri); this.model = Objects.requireNonNull(model); } @@ -46,10 +46,8 @@ public void execute( Supplier hasRequestCompletedFunction, ActionListener listener ) { - var docsOnly = DocumentsOnlyInput.of(inferenceInputs); - var docsInput = docsOnly.getInputs(); - var stream = docsOnly.stream(); - OpenAiChatCompletionRequest request = new OpenAiChatCompletionRequest(docsInput, model, stream); + var chatCompletionInputs = inferenceInputs.castTo(ChatCompletionInput.class); + var request = new OpenAiUnifiedChatCompletionRequest(new UnifiedChatInput(chatCompletionInputs, USER_ROLE), model); execute(new ExecutableInferenceRequest(requestSender, logger, request, HANDLER, hasRequestCompletedFunction, listener)); } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/OpenAiUnifiedCompletionRequestManager.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/OpenAiUnifiedCompletionRequestManager.java new file mode 100644 index 000000000000..3b0f770e3e06 --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/OpenAiUnifiedCompletionRequestManager.java @@ -0,0 +1,61 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference.external.http.sender; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.inference.InferenceServiceResults; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.inference.external.http.retry.RequestSender; +import org.elasticsearch.xpack.inference.external.http.retry.ResponseHandler; +import org.elasticsearch.xpack.inference.external.openai.OpenAiUnifiedChatCompletionResponseHandler; +import org.elasticsearch.xpack.inference.external.request.openai.OpenAiUnifiedChatCompletionRequest; +import org.elasticsearch.xpack.inference.external.response.openai.OpenAiChatCompletionResponseEntity; +import org.elasticsearch.xpack.inference.services.openai.completion.OpenAiChatCompletionModel; + +import java.util.Objects; +import java.util.function.Supplier; + +public class OpenAiUnifiedCompletionRequestManager extends OpenAiRequestManager { + + private static final Logger logger = LogManager.getLogger(OpenAiUnifiedCompletionRequestManager.class); + + private static final ResponseHandler HANDLER = createCompletionHandler(); + + public static OpenAiUnifiedCompletionRequestManager of(OpenAiChatCompletionModel model, ThreadPool threadPool) { + return new OpenAiUnifiedCompletionRequestManager(Objects.requireNonNull(model), Objects.requireNonNull(threadPool)); + } + + private final OpenAiChatCompletionModel model; + + private OpenAiUnifiedCompletionRequestManager(OpenAiChatCompletionModel model, ThreadPool threadPool) { + super(threadPool, model, OpenAiUnifiedChatCompletionRequest::buildDefaultUri); + this.model = Objects.requireNonNull(model); + } + + @Override + public void execute( + InferenceInputs inferenceInputs, + RequestSender requestSender, + Supplier hasRequestCompletedFunction, + ActionListener listener + ) { + + OpenAiUnifiedChatCompletionRequest request = new OpenAiUnifiedChatCompletionRequest( + inferenceInputs.castTo(UnifiedChatInput.class), + model + ); + + execute(new ExecutableInferenceRequest(requestSender, logger, request, HANDLER, hasRequestCompletedFunction, listener)); + } + + private static ResponseHandler createCompletionHandler() { + return new OpenAiUnifiedChatCompletionResponseHandler("openai completion", OpenAiChatCompletionResponseEntity::fromResponse); + } +} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/QueryAndDocsInputs.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/QueryAndDocsInputs.java index 50bb77b307db..5af5245ac5b4 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/QueryAndDocsInputs.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/QueryAndDocsInputs.java @@ -14,7 +14,7 @@ public class QueryAndDocsInputs extends InferenceInputs { public static QueryAndDocsInputs of(InferenceInputs inferenceInputs) { if (inferenceInputs instanceof QueryAndDocsInputs == false) { - throw createUnsupportedTypeException(inferenceInputs); + throw createUnsupportedTypeException(inferenceInputs, QueryAndDocsInputs.class); } return (QueryAndDocsInputs) inferenceInputs; @@ -22,17 +22,15 @@ public static QueryAndDocsInputs of(InferenceInputs inferenceInputs) { private final String query; private final List chunks; - private final boolean stream; public QueryAndDocsInputs(String query, List chunks) { this(query, chunks, false); } public QueryAndDocsInputs(String query, List chunks, boolean stream) { - super(); + super(stream); this.query = Objects.requireNonNull(query); this.chunks = Objects.requireNonNull(chunks); - this.stream = stream; } public String getQuery() { @@ -43,8 +41,7 @@ public List getChunks() { return chunks; } - public boolean stream() { - return stream; + public int inputSize() { + return chunks.size(); } - } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/UnifiedChatInput.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/UnifiedChatInput.java new file mode 100644 index 000000000000..f89fa1ee37a6 --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/http/sender/UnifiedChatInput.java @@ -0,0 +1,62 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference.external.http.sender; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.inference.Model; +import org.elasticsearch.inference.UnifiedCompletionRequest; + +import java.util.List; +import java.util.Objects; + +/** + * This class encapsulates the unified request. + * The main difference between this class and {@link ChatCompletionInput} is this should only be used for + * {@link org.elasticsearch.inference.TaskType#COMPLETION} originating through the + * {@link org.elasticsearch.inference.InferenceService#unifiedCompletionInfer(Model, UnifiedCompletionRequest, TimeValue, ActionListener)} + * code path. These are requests sent to the API with the _unified route. + */ +public class UnifiedChatInput extends InferenceInputs { + private final UnifiedCompletionRequest request; + + public UnifiedChatInput(UnifiedCompletionRequest request, boolean stream) { + super(stream); + this.request = Objects.requireNonNull(request); + } + + public UnifiedChatInput(ChatCompletionInput completionInput, String roleValue) { + this(completionInput.getInputs(), roleValue, completionInput.stream()); + } + + public UnifiedChatInput(List inputs, String roleValue, boolean stream) { + this(UnifiedCompletionRequest.of(convertToMessages(inputs, roleValue)), stream); + } + + private static List convertToMessages(List inputs, String roleValue) { + return inputs.stream() + .map( + value -> new UnifiedCompletionRequest.Message( + new UnifiedCompletionRequest.ContentString(value), + roleValue, + null, + null, + null + ) + ) + .toList(); + } + + public UnifiedCompletionRequest getRequest() { + return request; + } + + public int inputSize() { + return request.messages().size(); + } +} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiStreamingProcessor.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiStreamingProcessor.java index 6e006fe25595..48c8132035b5 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiStreamingProcessor.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiStreamingProcessor.java @@ -18,10 +18,8 @@ import org.elasticsearch.xpack.core.inference.results.StreamingChatCompletionResults; import org.elasticsearch.xpack.inference.common.DelegatingProcessor; import org.elasticsearch.xpack.inference.external.response.streaming.ServerSentEvent; -import org.elasticsearch.xpack.inference.external.response.streaming.ServerSentEventField; import java.io.IOException; -import java.util.ArrayDeque; import java.util.Collections; import java.util.Deque; import java.util.Iterator; @@ -115,19 +113,7 @@ public class OpenAiStreamingProcessor extends DelegatingProcessor item) throws Exception { var parserConfig = XContentParserConfiguration.EMPTY.withDeprecationHandler(LoggingDeprecationHandler.INSTANCE); - - var results = new ArrayDeque(item.size()); - for (ServerSentEvent event : item) { - if (ServerSentEventField.DATA == event.name() && event.hasValue()) { - try { - var delta = parse(parserConfig, event); - delta.forEachRemaining(results::offer); - } catch (Exception e) { - log.warn("Failed to parse event from inference provider: {}", event); - throw e; - } - } - } + var results = parseEvent(item, OpenAiStreamingProcessor::parse, parserConfig, log); if (results.isEmpty()) { upstream().request(1); @@ -136,7 +122,7 @@ protected void next(Deque item) throws Exception { } } - private Iterator parse(XContentParserConfiguration parserConfig, ServerSentEvent event) + private static Iterator parse(XContentParserConfiguration parserConfig, ServerSentEvent event) throws IOException { if (DONE_MESSAGE.equalsIgnoreCase(event.value())) { return Collections.emptyIterator(); diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedChatCompletionResponseHandler.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedChatCompletionResponseHandler.java new file mode 100644 index 000000000000..fce2556efc5e --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedChatCompletionResponseHandler.java @@ -0,0 +1,34 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference.external.openai; + +import org.elasticsearch.inference.InferenceServiceResults; +import org.elasticsearch.xpack.core.inference.results.StreamingUnifiedChatCompletionResults; +import org.elasticsearch.xpack.inference.external.http.HttpResult; +import org.elasticsearch.xpack.inference.external.http.retry.ResponseParser; +import org.elasticsearch.xpack.inference.external.request.Request; +import org.elasticsearch.xpack.inference.external.response.streaming.ServerSentEventParser; +import org.elasticsearch.xpack.inference.external.response.streaming.ServerSentEventProcessor; + +import java.util.concurrent.Flow; + +public class OpenAiUnifiedChatCompletionResponseHandler extends OpenAiChatCompletionResponseHandler { + public OpenAiUnifiedChatCompletionResponseHandler(String requestType, ResponseParser parseFunction) { + super(requestType, parseFunction); + } + + @Override + public InferenceServiceResults parseResult(Request request, Flow.Publisher flow) { + var serverSentEventProcessor = new ServerSentEventProcessor(new ServerSentEventParser()); + var openAiProcessor = new OpenAiUnifiedStreamingProcessor(); + + flow.subscribe(serverSentEventProcessor); + serverSentEventProcessor.subscribe(openAiProcessor); + return new StreamingUnifiedChatCompletionResults(openAiProcessor); + } +} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedStreamingProcessor.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedStreamingProcessor.java new file mode 100644 index 000000000000..599d71df3dcf --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedStreamingProcessor.java @@ -0,0 +1,287 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference.external.openai; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.common.xcontent.ChunkedToXContent; +import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; +import org.elasticsearch.xcontent.ConstructingObjectParser; +import org.elasticsearch.xcontent.ParseField; +import org.elasticsearch.xcontent.XContentFactory; +import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xcontent.XContentParserConfiguration; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.core.inference.results.StreamingUnifiedChatCompletionResults; +import org.elasticsearch.xpack.inference.common.DelegatingProcessor; +import org.elasticsearch.xpack.inference.external.response.streaming.ServerSentEvent; + +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.Collections; +import java.util.Deque; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.LinkedBlockingDeque; + +import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; +import static org.elasticsearch.xpack.inference.external.response.XContentUtils.moveToFirstToken; + +public class OpenAiUnifiedStreamingProcessor extends DelegatingProcessor, ChunkedToXContent> { + public static final String FUNCTION_FIELD = "function"; + private static final Logger logger = LogManager.getLogger(OpenAiUnifiedStreamingProcessor.class); + + private static final String CHOICES_FIELD = "choices"; + private static final String DELTA_FIELD = "delta"; + private static final String CONTENT_FIELD = "content"; + private static final String DONE_MESSAGE = "[done]"; + private static final String REFUSAL_FIELD = "refusal"; + private static final String TOOL_CALLS_FIELD = "tool_calls"; + public static final String ROLE_FIELD = "role"; + public static final String FINISH_REASON_FIELD = "finish_reason"; + public static final String INDEX_FIELD = "index"; + public static final String OBJECT_FIELD = "object"; + public static final String MODEL_FIELD = "model"; + public static final String ID_FIELD = "id"; + public static final String CHOICE_FIELD = "choice"; + public static final String USAGE_FIELD = "usage"; + public static final String TYPE_FIELD = "type"; + public static final String NAME_FIELD = "name"; + public static final String ARGUMENTS_FIELD = "arguments"; + public static final String COMPLETION_TOKENS_FIELD = "completion_tokens"; + public static final String PROMPT_TOKENS_FIELD = "prompt_tokens"; + public static final String TOTAL_TOKENS_FIELD = "total_tokens"; + + private final Deque buffer = new LinkedBlockingDeque<>(); + + @Override + protected void upstreamRequest(long n) { + if (buffer.isEmpty()) { + super.upstreamRequest(n); + } else { + downstream().onNext(new StreamingUnifiedChatCompletionResults.Results(singleItem(buffer.poll()))); + } + } + + @Override + protected void next(Deque item) throws Exception { + var parserConfig = XContentParserConfiguration.EMPTY.withDeprecationHandler(LoggingDeprecationHandler.INSTANCE); + var results = parseEvent(item, OpenAiUnifiedStreamingProcessor::parse, parserConfig, logger); + + if (results.isEmpty()) { + upstream().request(1); + } else if (results.size() == 1) { + downstream().onNext(new StreamingUnifiedChatCompletionResults.Results(results)); + } else { + // results > 1, but openai spec only wants 1 chunk per SSE event + var firstItem = singleItem(results.poll()); + while (results.isEmpty() == false) { + buffer.offer(results.poll()); + } + downstream().onNext(new StreamingUnifiedChatCompletionResults.Results(firstItem)); + } + } + + private static Iterator parse( + XContentParserConfiguration parserConfig, + ServerSentEvent event + ) throws IOException { + if (DONE_MESSAGE.equalsIgnoreCase(event.value())) { + return Collections.emptyIterator(); + } + + try (XContentParser jsonParser = XContentFactory.xContent(XContentType.JSON).createParser(parserConfig, event.value())) { + moveToFirstToken(jsonParser); + + XContentParser.Token token = jsonParser.currentToken(); + ensureExpectedToken(XContentParser.Token.START_OBJECT, token, jsonParser); + + StreamingUnifiedChatCompletionResults.ChatCompletionChunk chunk = ChatCompletionChunkParser.parse(jsonParser); + + return Collections.singleton(chunk).iterator(); + } + } + + public static class ChatCompletionChunkParser { + @SuppressWarnings("unchecked") + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>( + "chat_completion_chunk", + true, + args -> new StreamingUnifiedChatCompletionResults.ChatCompletionChunk( + (String) args[0], + (List) args[1], + (String) args[2], + (String) args[3], + (StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Usage) args[4] + ) + ); + + static { + PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField(ID_FIELD)); + PARSER.declareObjectArray( + ConstructingObjectParser.constructorArg(), + (p, c) -> ChatCompletionChunkParser.ChoiceParser.parse(p), + new ParseField(CHOICES_FIELD) + ); + PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField(MODEL_FIELD)); + PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField(OBJECT_FIELD)); + PARSER.declareObjectOrNull( + ConstructingObjectParser.optionalConstructorArg(), + (p, c) -> ChatCompletionChunkParser.UsageParser.parse(p), + null, + new ParseField(USAGE_FIELD) + ); + } + + public static StreamingUnifiedChatCompletionResults.ChatCompletionChunk parse(XContentParser parser) throws IOException { + return PARSER.parse(parser, null); + } + + private static class ChoiceParser { + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>( + CHOICE_FIELD, + true, + args -> new StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice( + (StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta) args[0], + (String) args[1], + (int) args[2] + ) + ); + + static { + PARSER.declareObject( + ConstructingObjectParser.constructorArg(), + (p, c) -> ChatCompletionChunkParser.DeltaParser.parse(p), + new ParseField(DELTA_FIELD) + ); + PARSER.declareStringOrNull(ConstructingObjectParser.optionalConstructorArg(), new ParseField(FINISH_REASON_FIELD)); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), new ParseField(INDEX_FIELD)); + } + + public static StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice parse(XContentParser parser) { + return PARSER.apply(parser, null); + } + } + + private static class DeltaParser { + @SuppressWarnings("unchecked") + private static final ConstructingObjectParser< + StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta, + Void> PARSER = new ConstructingObjectParser<>( + DELTA_FIELD, + true, + args -> new StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta( + (String) args[0], + (String) args[1], + (String) args[2], + (List) args[3] + ) + ); + + static { + PARSER.declareStringOrNull(ConstructingObjectParser.optionalConstructorArg(), new ParseField(CONTENT_FIELD)); + PARSER.declareStringOrNull(ConstructingObjectParser.optionalConstructorArg(), new ParseField(REFUSAL_FIELD)); + PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), new ParseField(ROLE_FIELD)); + PARSER.declareObjectArray( + ConstructingObjectParser.optionalConstructorArg(), + (p, c) -> ChatCompletionChunkParser.ToolCallParser.parse(p), + new ParseField(TOOL_CALLS_FIELD) + ); + } + + public static StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta parse(XContentParser parser) + throws IOException { + return PARSER.parse(parser, null); + } + } + + private static class ToolCallParser { + private static final ConstructingObjectParser< + StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta.ToolCall, + Void> PARSER = new ConstructingObjectParser<>( + "tool_call", + true, + args -> new StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta.ToolCall( + (int) args[0], + (String) args[1], + (StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta.ToolCall.Function) args[2], + (String) args[3] + ) + ); + + static { + PARSER.declareInt(ConstructingObjectParser.constructorArg(), new ParseField(INDEX_FIELD)); + PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), new ParseField(ID_FIELD)); + PARSER.declareObject( + ConstructingObjectParser.optionalConstructorArg(), + (p, c) -> ChatCompletionChunkParser.FunctionParser.parse(p), + new ParseField(FUNCTION_FIELD) + ); + PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), new ParseField(TYPE_FIELD)); + } + + public static StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta.ToolCall parse(XContentParser parser) + throws IOException { + return PARSER.parse(parser, null); + } + } + + private static class FunctionParser { + private static final ConstructingObjectParser< + StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta.ToolCall.Function, + Void> PARSER = new ConstructingObjectParser<>( + FUNCTION_FIELD, + true, + args -> new StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta.ToolCall.Function( + (String) args[0], + (String) args[1] + ) + ); + + static { + PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), new ParseField(ARGUMENTS_FIELD)); + PARSER.declareString(ConstructingObjectParser.optionalConstructorArg(), new ParseField(NAME_FIELD)); + } + + public static StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta.ToolCall.Function parse( + XContentParser parser + ) throws IOException { + return PARSER.parse(parser, null); + } + } + + private static class UsageParser { + private static final ConstructingObjectParser PARSER = + new ConstructingObjectParser<>( + USAGE_FIELD, + true, + args -> new StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Usage((int) args[0], (int) args[1], (int) args[2]) + ); + + static { + PARSER.declareInt(ConstructingObjectParser.constructorArg(), new ParseField(COMPLETION_TOKENS_FIELD)); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), new ParseField(PROMPT_TOKENS_FIELD)); + PARSER.declareInt(ConstructingObjectParser.constructorArg(), new ParseField(TOTAL_TOKENS_FIELD)); + } + + public static StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Usage parse(XContentParser parser) throws IOException { + return PARSER.parse(parser, null); + } + } + } + + private Deque singleItem( + StreamingUnifiedChatCompletionResults.ChatCompletionChunk result + ) { + var deque = new ArrayDeque(1); + deque.offer(result); + return deque; + } +} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/googleaistudio/GoogleAiStudioCompletionRequest.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/googleaistudio/GoogleAiStudioCompletionRequest.java index 80770d63ef13..b1af18d03dda 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/googleaistudio/GoogleAiStudioCompletionRequest.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/googleaistudio/GoogleAiStudioCompletionRequest.java @@ -14,7 +14,7 @@ import org.elasticsearch.common.ValidationException; import org.elasticsearch.common.util.LazyInitializable; import org.elasticsearch.xcontent.XContentType; -import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; +import org.elasticsearch.xpack.inference.external.http.sender.ChatCompletionInput; import org.elasticsearch.xpack.inference.external.request.HttpRequest; import org.elasticsearch.xpack.inference.external.request.Request; import org.elasticsearch.xpack.inference.services.googleaistudio.completion.GoogleAiStudioCompletionModel; @@ -27,13 +27,13 @@ public class GoogleAiStudioCompletionRequest implements GoogleAiStudioRequest { private static final String ALT_PARAM = "alt"; private static final String SSE_VALUE = "sse"; - private final DocumentsOnlyInput input; + private final ChatCompletionInput input; private final LazyInitializable uri; private final GoogleAiStudioCompletionModel model; - public GoogleAiStudioCompletionRequest(DocumentsOnlyInput input, GoogleAiStudioCompletionModel model) { + public GoogleAiStudioCompletionRequest(ChatCompletionInput input, GoogleAiStudioCompletionModel model) { this.input = Objects.requireNonNull(input); this.model = Objects.requireNonNull(model); this.uri = new LazyInitializable<>(() -> model.uri(input.stream())); diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiChatCompletionRequestEntity.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiChatCompletionRequestEntity.java deleted file mode 100644 index 867a7ca80cbc..000000000000 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiChatCompletionRequestEntity.java +++ /dev/null @@ -1,79 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0; you may not use this file except in compliance with the Elastic License - * 2.0. - */ - -package org.elasticsearch.xpack.inference.external.request.openai; - -import org.elasticsearch.common.Strings; -import org.elasticsearch.xcontent.ToXContentObject; -import org.elasticsearch.xcontent.XContentBuilder; - -import java.io.IOException; -import java.util.List; -import java.util.Objects; - -public class OpenAiChatCompletionRequestEntity implements ToXContentObject { - - private static final String MESSAGES_FIELD = "messages"; - private static final String MODEL_FIELD = "model"; - - private static final String NUMBER_OF_RETURNED_CHOICES_FIELD = "n"; - - private static final String ROLE_FIELD = "role"; - private static final String USER_FIELD = "user"; - private static final String CONTENT_FIELD = "content"; - private static final String STREAM_FIELD = "stream"; - - private final List messages; - private final String model; - - private final String user; - private final boolean stream; - - public OpenAiChatCompletionRequestEntity(List messages, String model, String user, boolean stream) { - Objects.requireNonNull(messages); - Objects.requireNonNull(model); - - this.messages = messages; - this.model = model; - this.user = user; - this.stream = stream; - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.startObject(); - builder.startArray(MESSAGES_FIELD); - { - for (String message : messages) { - builder.startObject(); - - { - builder.field(ROLE_FIELD, USER_FIELD); - builder.field(CONTENT_FIELD, message); - } - - builder.endObject(); - } - } - builder.endArray(); - - builder.field(MODEL_FIELD, model); - builder.field(NUMBER_OF_RETURNED_CHOICES_FIELD, 1); - - if (Strings.isNullOrEmpty(user) == false) { - builder.field(USER_FIELD, user); - } - - if (stream) { - builder.field(STREAM_FIELD, true); - } - - builder.endObject(); - - return builder; - } -} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiChatCompletionRequest.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiUnifiedChatCompletionRequest.java similarity index 80% rename from x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiChatCompletionRequest.java rename to x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiUnifiedChatCompletionRequest.java index 99a025e70d00..2e6bdb748fd3 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiChatCompletionRequest.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiUnifiedChatCompletionRequest.java @@ -13,6 +13,7 @@ import org.apache.http.entity.ByteArrayEntity; import org.elasticsearch.common.Strings; import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; import org.elasticsearch.xpack.inference.external.openai.OpenAiAccount; import org.elasticsearch.xpack.inference.external.request.HttpRequest; import org.elasticsearch.xpack.inference.external.request.Request; @@ -21,24 +22,21 @@ import java.net.URI; import java.net.URISyntaxException; import java.nio.charset.StandardCharsets; -import java.util.List; import java.util.Objects; import static org.elasticsearch.xpack.inference.external.request.RequestUtils.createAuthBearerHeader; import static org.elasticsearch.xpack.inference.external.request.openai.OpenAiUtils.createOrgHeader; -public class OpenAiChatCompletionRequest implements OpenAiRequest { +public class OpenAiUnifiedChatCompletionRequest implements OpenAiRequest { private final OpenAiAccount account; - private final List input; private final OpenAiChatCompletionModel model; - private final boolean stream; + private final UnifiedChatInput unifiedChatInput; - public OpenAiChatCompletionRequest(List input, OpenAiChatCompletionModel model, boolean stream) { - this.account = OpenAiAccount.of(model, OpenAiChatCompletionRequest::buildDefaultUri); - this.input = Objects.requireNonNull(input); + public OpenAiUnifiedChatCompletionRequest(UnifiedChatInput unifiedChatInput, OpenAiChatCompletionModel model) { + this.account = OpenAiAccount.of(model, OpenAiUnifiedChatCompletionRequest::buildDefaultUri); + this.unifiedChatInput = Objects.requireNonNull(unifiedChatInput); this.model = Objects.requireNonNull(model); - this.stream = stream; } @Override @@ -46,9 +44,7 @@ public HttpRequest createHttpRequest() { HttpPost httpPost = new HttpPost(account.uri()); ByteArrayEntity byteEntity = new ByteArrayEntity( - Strings.toString( - new OpenAiChatCompletionRequestEntity(input, model.getServiceSettings().modelId(), model.getTaskSettings().user(), stream) - ).getBytes(StandardCharsets.UTF_8) + Strings.toString(new OpenAiUnifiedChatCompletionRequestEntity(unifiedChatInput, model)).getBytes(StandardCharsets.UTF_8) ); httpPost.setEntity(byteEntity); @@ -87,7 +83,7 @@ public String getInferenceEntityId() { @Override public boolean isStreaming() { - return stream; + return unifiedChatInput.stream(); } public static URI buildDefaultUri() throws URISyntaxException { diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiUnifiedChatCompletionRequestEntity.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiUnifiedChatCompletionRequestEntity.java new file mode 100644 index 000000000000..50339bf851f7 --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiUnifiedChatCompletionRequestEntity.java @@ -0,0 +1,185 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference.external.request.openai; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.inference.UnifiedCompletionRequest; +import org.elasticsearch.xcontent.ToXContentObject; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; +import org.elasticsearch.xpack.inference.services.openai.completion.OpenAiChatCompletionModel; + +import java.io.IOException; +import java.util.Objects; + +public class OpenAiUnifiedChatCompletionRequestEntity implements ToXContentObject { + + public static final String NAME_FIELD = "name"; + public static final String TOOL_CALL_ID_FIELD = "tool_call_id"; + public static final String TOOL_CALLS_FIELD = "tool_calls"; + public static final String ID_FIELD = "id"; + public static final String FUNCTION_FIELD = "function"; + public static final String ARGUMENTS_FIELD = "arguments"; + public static final String DESCRIPTION_FIELD = "description"; + public static final String PARAMETERS_FIELD = "parameters"; + public static final String STRICT_FIELD = "strict"; + public static final String TOP_P_FIELD = "top_p"; + public static final String USER_FIELD = "user"; + public static final String STREAM_FIELD = "stream"; + private static final String NUMBER_OF_RETURNED_CHOICES_FIELD = "n"; + private static final String MODEL_FIELD = "model"; + public static final String MESSAGES_FIELD = "messages"; + private static final String ROLE_FIELD = "role"; + private static final String CONTENT_FIELD = "content"; + private static final String MAX_COMPLETION_TOKENS_FIELD = "max_completion_tokens"; + private static final String STOP_FIELD = "stop"; + private static final String TEMPERATURE_FIELD = "temperature"; + private static final String TOOL_CHOICE_FIELD = "tool_choice"; + private static final String TOOL_FIELD = "tools"; + private static final String TEXT_FIELD = "text"; + private static final String TYPE_FIELD = "type"; + private static final String STREAM_OPTIONS_FIELD = "stream_options"; + private static final String INCLUDE_USAGE_FIELD = "include_usage"; + + private final UnifiedCompletionRequest unifiedRequest; + private final boolean stream; + private final OpenAiChatCompletionModel model; + + public OpenAiUnifiedChatCompletionRequestEntity(UnifiedChatInput unifiedChatInput, OpenAiChatCompletionModel model) { + Objects.requireNonNull(unifiedChatInput); + + this.unifiedRequest = unifiedChatInput.getRequest(); + this.stream = unifiedChatInput.stream(); + this.model = Objects.requireNonNull(model); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.startArray(MESSAGES_FIELD); + { + for (UnifiedCompletionRequest.Message message : unifiedRequest.messages()) { + builder.startObject(); + { + switch (message.content()) { + case UnifiedCompletionRequest.ContentString contentString -> builder.field(CONTENT_FIELD, contentString.content()); + case UnifiedCompletionRequest.ContentObjects contentObjects -> { + builder.startArray(CONTENT_FIELD); + for (UnifiedCompletionRequest.ContentObject contentObject : contentObjects.contentObjects()) { + builder.startObject(); + builder.field(TEXT_FIELD, contentObject.text()); + builder.field(TYPE_FIELD, contentObject.type()); + builder.endObject(); + } + builder.endArray(); + } + } + + builder.field(ROLE_FIELD, message.role()); + if (message.name() != null) { + builder.field(NAME_FIELD, message.name()); + } + if (message.toolCallId() != null) { + builder.field(TOOL_CALL_ID_FIELD, message.toolCallId()); + } + if (message.toolCalls() != null) { + builder.startArray(TOOL_CALLS_FIELD); + for (UnifiedCompletionRequest.ToolCall toolCall : message.toolCalls()) { + builder.startObject(); + { + builder.field(ID_FIELD, toolCall.id()); + builder.startObject(FUNCTION_FIELD); + { + builder.field(ARGUMENTS_FIELD, toolCall.function().arguments()); + builder.field(NAME_FIELD, toolCall.function().name()); + } + builder.endObject(); + builder.field(TYPE_FIELD, toolCall.type()); + } + builder.endObject(); + } + builder.endArray(); + } + } + builder.endObject(); + } + } + builder.endArray(); + + builder.field(MODEL_FIELD, model.getServiceSettings().modelId()); + if (unifiedRequest.maxCompletionTokens() != null) { + builder.field(MAX_COMPLETION_TOKENS_FIELD, unifiedRequest.maxCompletionTokens()); + } + + builder.field(NUMBER_OF_RETURNED_CHOICES_FIELD, 1); + + if (unifiedRequest.stop() != null && unifiedRequest.stop().isEmpty() == false) { + builder.field(STOP_FIELD, unifiedRequest.stop()); + } + if (unifiedRequest.temperature() != null) { + builder.field(TEMPERATURE_FIELD, unifiedRequest.temperature()); + } + if (unifiedRequest.toolChoice() != null) { + if (unifiedRequest.toolChoice() instanceof UnifiedCompletionRequest.ToolChoiceString) { + builder.field(TOOL_CHOICE_FIELD, ((UnifiedCompletionRequest.ToolChoiceString) unifiedRequest.toolChoice()).value()); + } else if (unifiedRequest.toolChoice() instanceof UnifiedCompletionRequest.ToolChoiceObject) { + builder.startObject(TOOL_CHOICE_FIELD); + { + builder.field(TYPE_FIELD, ((UnifiedCompletionRequest.ToolChoiceObject) unifiedRequest.toolChoice()).type()); + builder.startObject(FUNCTION_FIELD); + { + builder.field( + NAME_FIELD, + ((UnifiedCompletionRequest.ToolChoiceObject) unifiedRequest.toolChoice()).function().name() + ); + } + builder.endObject(); + } + builder.endObject(); + } + } + if (unifiedRequest.tools() != null && unifiedRequest.tools().isEmpty() == false) { + builder.startArray(TOOL_FIELD); + for (UnifiedCompletionRequest.Tool t : unifiedRequest.tools()) { + builder.startObject(); + { + builder.field(TYPE_FIELD, t.type()); + builder.startObject(FUNCTION_FIELD); + { + builder.field(DESCRIPTION_FIELD, t.function().description()); + builder.field(NAME_FIELD, t.function().name()); + builder.field(PARAMETERS_FIELD, t.function().parameters()); + if (t.function().strict() != null) { + builder.field(STRICT_FIELD, t.function().strict()); + } + } + builder.endObject(); + } + builder.endObject(); + } + builder.endArray(); + } + if (unifiedRequest.topP() != null) { + builder.field(TOP_P_FIELD, unifiedRequest.topP()); + } + + if (Strings.isNullOrEmpty(model.getTaskSettings().user()) == false) { + builder.field(USER_FIELD, model.getTaskSettings().user()); + } + + builder.field(STREAM_FIELD, stream); + if (stream) { + builder.startObject(STREAM_OPTIONS_FIELD); + builder.field(INCLUDE_USAGE_FIELD, true); + builder.endObject(); + } + builder.endObject(); + + return builder; + } +} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/BaseInferenceAction.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/BaseInferenceAction.java index e72e68052f64..d911158e8229 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/BaseInferenceAction.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/BaseInferenceAction.java @@ -9,6 +9,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.client.internal.node.NodeClient; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.inference.TaskType; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestChannel; @@ -21,27 +22,32 @@ import static org.elasticsearch.xpack.inference.rest.Paths.TASK_TYPE_OR_INFERENCE_ID; abstract class BaseInferenceAction extends BaseRestHandler { - @Override - protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException { - String inferenceEntityId; - TaskType taskType; + static Params parseParams(RestRequest restRequest) { if (restRequest.hasParam(INFERENCE_ID)) { - inferenceEntityId = restRequest.param(INFERENCE_ID); - taskType = TaskType.fromStringOrStatusException(restRequest.param(TASK_TYPE_OR_INFERENCE_ID)); + var inferenceEntityId = restRequest.param(INFERENCE_ID); + var taskType = TaskType.fromStringOrStatusException(restRequest.param(TASK_TYPE_OR_INFERENCE_ID)); + return new Params(inferenceEntityId, taskType); } else { - inferenceEntityId = restRequest.param(TASK_TYPE_OR_INFERENCE_ID); - taskType = TaskType.ANY; + return new Params(restRequest.param(TASK_TYPE_OR_INFERENCE_ID), TaskType.ANY); } + } + + record Params(String inferenceEntityId, TaskType taskType) {} + + static TimeValue parseTimeout(RestRequest restRequest) { + return restRequest.paramAsTime(InferenceAction.Request.TIMEOUT.getPreferredName(), InferenceAction.Request.DEFAULT_TIMEOUT); + } + + @Override + protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException { + var params = parseParams(restRequest); InferenceAction.Request.Builder requestBuilder; try (var parser = restRequest.contentParser()) { - requestBuilder = InferenceAction.Request.parseRequest(inferenceEntityId, taskType, parser); + requestBuilder = InferenceAction.Request.parseRequest(params.inferenceEntityId(), params.taskType(), parser); } - var inferTimeout = restRequest.paramAsTime( - InferenceAction.Request.TIMEOUT.getPreferredName(), - InferenceAction.Request.DEFAULT_TIMEOUT - ); + var inferTimeout = parseTimeout(restRequest); requestBuilder.setInferenceTimeout(inferTimeout); var request = prepareInferenceRequest(requestBuilder); return channel -> client.execute(InferenceAction.INSTANCE, request, listener(channel)); diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/Paths.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/Paths.java index 55d6443b43c0..c46f211bb26a 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/Paths.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/Paths.java @@ -30,6 +30,12 @@ public final class Paths { + "}/{" + INFERENCE_ID + "}/_stream"; + static final String UNIFIED_INFERENCE_ID_PATH = "_inference/{" + TASK_TYPE_OR_INFERENCE_ID + "}/_unified"; + static final String UNIFIED_TASK_TYPE_INFERENCE_ID_PATH = "_inference/{" + + TASK_TYPE_OR_INFERENCE_ID + + "}/{" + + INFERENCE_ID + + "}/_unified"; private Paths() { diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/RestUnifiedCompletionInferenceAction.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/RestUnifiedCompletionInferenceAction.java new file mode 100644 index 000000000000..5c71b560a6b9 --- /dev/null +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/rest/RestUnifiedCompletionInferenceAction.java @@ -0,0 +1,49 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference.rest; + +import org.elasticsearch.client.internal.node.NodeClient; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.Scope; +import org.elasticsearch.rest.ServerlessScope; +import org.elasticsearch.xpack.core.inference.action.UnifiedCompletionAction; + +import java.io.IOException; +import java.util.List; + +import static org.elasticsearch.rest.RestRequest.Method.POST; +import static org.elasticsearch.xpack.inference.rest.Paths.UNIFIED_INFERENCE_ID_PATH; +import static org.elasticsearch.xpack.inference.rest.Paths.UNIFIED_TASK_TYPE_INFERENCE_ID_PATH; + +@ServerlessScope(Scope.PUBLIC) +public class RestUnifiedCompletionInferenceAction extends BaseRestHandler { + @Override + public String getName() { + return "unified_inference_action"; + } + + @Override + public List routes() { + return List.of(new Route(POST, UNIFIED_INFERENCE_ID_PATH), new Route(POST, UNIFIED_TASK_TYPE_INFERENCE_ID_PATH)); + } + + @Override + protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException { + var params = BaseInferenceAction.parseParams(restRequest); + + var inferTimeout = BaseInferenceAction.parseTimeout(restRequest); + + UnifiedCompletionAction.Request request; + try (var parser = restRequest.contentParser()) { + request = UnifiedCompletionAction.Request.parseRequest(params.inferenceEntityId(), params.taskType(), inferTimeout, parser); + } + + return channel -> client.execute(UnifiedCompletionAction.INSTANCE, request, new ServerSentEventsRestActionListener(channel)); + } +} diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/SenderService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/SenderService.java index 8e2dac1ef9db..e9b75e9ec779 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/SenderService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/SenderService.java @@ -7,9 +7,11 @@ package org.elasticsearch.xpack.inference.services; +import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.core.IOUtils; import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.Strings; import org.elasticsearch.core.TimeValue; import org.elasticsearch.inference.ChunkedInferenceServiceResults; import org.elasticsearch.inference.InferenceService; @@ -17,11 +19,15 @@ import org.elasticsearch.inference.InputType; import org.elasticsearch.inference.Model; import org.elasticsearch.inference.TaskType; +import org.elasticsearch.inference.UnifiedCompletionRequest; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.xpack.inference.external.http.sender.ChatCompletionInput; import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; import org.elasticsearch.xpack.inference.external.http.sender.InferenceInputs; import org.elasticsearch.xpack.inference.external.http.sender.QueryAndDocsInputs; import org.elasticsearch.xpack.inference.external.http.sender.Sender; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; import java.io.IOException; import java.util.EnumSet; @@ -61,11 +67,31 @@ public void infer( ActionListener listener ) { init(); - if (query != null) { - doInfer(model, new QueryAndDocsInputs(query, input, stream), taskSettings, inputType, timeout, listener); - } else { - doInfer(model, new DocumentsOnlyInput(input, stream), taskSettings, inputType, timeout, listener); - } + var inferenceInput = createInput(model, input, query, stream); + doInfer(model, inferenceInput, taskSettings, inputType, timeout, listener); + } + + private static InferenceInputs createInput(Model model, List input, @Nullable String query, boolean stream) { + return switch (model.getTaskType()) { + case COMPLETION -> new ChatCompletionInput(input, stream); + case RERANK -> new QueryAndDocsInputs(query, input, stream); + case TEXT_EMBEDDING, SPARSE_EMBEDDING -> new DocumentsOnlyInput(input, stream); + default -> throw new ElasticsearchStatusException( + Strings.format("Invalid task type received when determining input type: [%s]", model.getTaskType().toString()), + RestStatus.BAD_REQUEST + ); + }; + } + + @Override + public void unifiedCompletionInfer( + Model model, + UnifiedCompletionRequest request, + TimeValue timeout, + ActionListener listener + ) { + init(); + doUnifiedCompletionInfer(model, new UnifiedChatInput(request, true), timeout, listener); } @Override @@ -92,6 +118,13 @@ protected abstract void doInfer( ActionListener listener ); + protected abstract void doUnifiedCompletionInfer( + Model model, + UnifiedChatInput inputs, + TimeValue timeout, + ActionListener listener + ); + protected abstract void doChunkedInfer( Model model, DocumentsOnlyInput inputs, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ServiceUtils.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ServiceUtils.java index ec4b8d9bb4d3..7d05bac363fb 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ServiceUtils.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ServiceUtils.java @@ -776,5 +776,9 @@ public static T nonNullOrDefault(@Nullable T requestValue, @Nullable T origi return requestValue == null ? originalSettingsValue : requestValue; } + public static void throwUnsupportedUnifiedCompletionOperation(String serviceName) { + throw new UnsupportedOperationException(Strings.format("The %s service does not support unified completion", serviceName)); + } + private ServiceUtils() {} } diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/alibabacloudsearch/AlibabaCloudSearchService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/alibabacloudsearch/AlibabaCloudSearchService.java index 5adc2a11b19d..ffd26b9ac534 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/alibabacloudsearch/AlibabaCloudSearchService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/alibabacloudsearch/AlibabaCloudSearchService.java @@ -37,6 +37,7 @@ import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; import org.elasticsearch.xpack.inference.external.http.sender.InferenceInputs; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; import org.elasticsearch.xpack.inference.external.request.alibabacloudsearch.AlibabaCloudSearchUtils; import org.elasticsearch.xpack.inference.services.ConfigurationParseContext; import org.elasticsearch.xpack.inference.services.SenderService; @@ -57,14 +58,13 @@ import java.util.Map; import java.util.stream.Stream; -import static org.elasticsearch.inference.TaskType.SPARSE_EMBEDDING; -import static org.elasticsearch.inference.TaskType.TEXT_EMBEDDING; import static org.elasticsearch.xpack.inference.services.ServiceUtils.createInvalidModelException; import static org.elasticsearch.xpack.inference.services.ServiceUtils.parsePersistedConfigErrorMsg; import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMap; import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrDefaultEmpty; import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrThrowIfNull; import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwIfNotEmptyMap; +import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwUnsupportedUnifiedCompletionOperation; import static org.elasticsearch.xpack.inference.services.alibabacloudsearch.AlibabaCloudSearchServiceFields.EMBEDDING_MAX_BATCH_SIZE; import static org.elasticsearch.xpack.inference.services.alibabacloudsearch.AlibabaCloudSearchServiceSettings.HOST; import static org.elasticsearch.xpack.inference.services.alibabacloudsearch.AlibabaCloudSearchServiceSettings.HTTP_SCHEMA_NAME; @@ -261,6 +261,16 @@ public AlibabaCloudSearchModel parsePersistedConfig(String inferenceEntityId, Ta ); } + @Override + protected void doUnifiedCompletionInfer( + Model model, + UnifiedChatInput inputs, + TimeValue timeout, + ActionListener listener + ) { + throwUnsupportedUnifiedCompletionOperation(NAME); + } + @Override public void doInfer( Model model, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/amazonbedrock/AmazonBedrockService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/amazonbedrock/AmazonBedrockService.java index 48b3c3df03e1..d224e50bb650 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/amazonbedrock/AmazonBedrockService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/amazonbedrock/AmazonBedrockService.java @@ -40,6 +40,7 @@ import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; import org.elasticsearch.xpack.inference.external.http.sender.InferenceInputs; import org.elasticsearch.xpack.inference.external.http.sender.Sender; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; import org.elasticsearch.xpack.inference.services.ConfigurationParseContext; import org.elasticsearch.xpack.inference.services.SenderService; import org.elasticsearch.xpack.inference.services.ServiceComponents; @@ -64,6 +65,7 @@ import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrDefaultEmpty; import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrThrowIfNull; import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwIfNotEmptyMap; +import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwUnsupportedUnifiedCompletionOperation; import static org.elasticsearch.xpack.inference.services.amazonbedrock.AmazonBedrockConstants.MODEL_FIELD; import static org.elasticsearch.xpack.inference.services.amazonbedrock.AmazonBedrockConstants.PROVIDER_FIELD; import static org.elasticsearch.xpack.inference.services.amazonbedrock.AmazonBedrockConstants.REGION_FIELD; @@ -89,6 +91,16 @@ public AmazonBedrockService( this.amazonBedrockSender = amazonBedrockFactory.createSender(); } + @Override + protected void doUnifiedCompletionInfer( + Model model, + UnifiedChatInput inputs, + TimeValue timeout, + ActionListener listener + ) { + throwUnsupportedUnifiedCompletionOperation(NAME); + } + @Override protected void doInfer( Model model, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/anthropic/AnthropicService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/anthropic/AnthropicService.java index b3d503de8e3e..f1840af18779 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/anthropic/AnthropicService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/anthropic/AnthropicService.java @@ -32,6 +32,7 @@ import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; import org.elasticsearch.xpack.inference.external.http.sender.InferenceInputs; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; import org.elasticsearch.xpack.inference.services.ConfigurationParseContext; import org.elasticsearch.xpack.inference.services.SenderService; import org.elasticsearch.xpack.inference.services.ServiceComponents; @@ -52,6 +53,7 @@ import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrDefaultEmpty; import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrThrowIfNull; import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwIfNotEmptyMap; +import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwUnsupportedUnifiedCompletionOperation; public class AnthropicService extends SenderService { public static final String NAME = "anthropic"; @@ -192,6 +194,16 @@ public EnumSet supportedTaskTypes() { return supportedTaskTypes; } + @Override + protected void doUnifiedCompletionInfer( + Model model, + UnifiedChatInput inputs, + TimeValue timeout, + ActionListener listener + ) { + throwUnsupportedUnifiedCompletionOperation(NAME); + } + @Override public void doInfer( Model model, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/azureaistudio/AzureAiStudioService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/azureaistudio/AzureAiStudioService.java index bba331fc0b5d..f8ea11e4b15a 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/azureaistudio/AzureAiStudioService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/azureaistudio/AzureAiStudioService.java @@ -38,6 +38,7 @@ import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; import org.elasticsearch.xpack.inference.external.http.sender.InferenceInputs; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; import org.elasticsearch.xpack.inference.services.ConfigurationParseContext; import org.elasticsearch.xpack.inference.services.SenderService; import org.elasticsearch.xpack.inference.services.ServiceComponents; @@ -63,6 +64,7 @@ import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrDefaultEmpty; import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrThrowIfNull; import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwIfNotEmptyMap; +import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwUnsupportedUnifiedCompletionOperation; import static org.elasticsearch.xpack.inference.services.azureaistudio.AzureAiStudioConstants.ENDPOINT_TYPE_FIELD; import static org.elasticsearch.xpack.inference.services.azureaistudio.AzureAiStudioConstants.PROVIDER_FIELD; import static org.elasticsearch.xpack.inference.services.azureaistudio.AzureAiStudioConstants.TARGET_FIELD; @@ -81,6 +83,16 @@ public AzureAiStudioService(HttpRequestSender.Factory factory, ServiceComponents super(factory, serviceComponents); } + @Override + protected void doUnifiedCompletionInfer( + Model model, + UnifiedChatInput inputs, + TimeValue timeout, + ActionListener listener + ) { + throwUnsupportedUnifiedCompletionOperation(NAME); + } + @Override protected void doInfer( Model model, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/azureopenai/AzureOpenAiService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/azureopenai/AzureOpenAiService.java index 16c94dfa9ad9..a38c265d2613 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/azureopenai/AzureOpenAiService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/azureopenai/AzureOpenAiService.java @@ -36,6 +36,7 @@ import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; import org.elasticsearch.xpack.inference.external.http.sender.InferenceInputs; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; import org.elasticsearch.xpack.inference.services.ConfigurationParseContext; import org.elasticsearch.xpack.inference.services.SenderService; import org.elasticsearch.xpack.inference.services.ServiceComponents; @@ -58,6 +59,7 @@ import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrDefaultEmpty; import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrThrowIfNull; import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwIfNotEmptyMap; +import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwUnsupportedUnifiedCompletionOperation; import static org.elasticsearch.xpack.inference.services.azureopenai.AzureOpenAiServiceFields.API_VERSION; import static org.elasticsearch.xpack.inference.services.azureopenai.AzureOpenAiServiceFields.DEPLOYMENT_ID; import static org.elasticsearch.xpack.inference.services.azureopenai.AzureOpenAiServiceFields.RESOURCE_NAME; @@ -233,6 +235,16 @@ public EnumSet supportedTaskTypes() { return supportedTaskTypes; } + @Override + protected void doUnifiedCompletionInfer( + Model model, + UnifiedChatInput inputs, + TimeValue timeout, + ActionListener listener + ) { + throwUnsupportedUnifiedCompletionOperation(NAME); + } + @Override protected void doInfer( Model model, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/cohere/CohereService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/cohere/CohereService.java index b3d8b3b6efce..ccb8d79dacd6 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/cohere/CohereService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/cohere/CohereService.java @@ -34,6 +34,7 @@ import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; import org.elasticsearch.xpack.inference.external.http.sender.InferenceInputs; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; import org.elasticsearch.xpack.inference.services.ConfigurationParseContext; import org.elasticsearch.xpack.inference.services.SenderService; import org.elasticsearch.xpack.inference.services.ServiceComponents; @@ -58,6 +59,7 @@ import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrDefaultEmpty; import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrThrowIfNull; import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwIfNotEmptyMap; +import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwUnsupportedUnifiedCompletionOperation; import static org.elasticsearch.xpack.inference.services.cohere.CohereServiceFields.EMBEDDING_MAX_BATCH_SIZE; public class CohereService extends SenderService { @@ -232,6 +234,16 @@ public EnumSet supportedTaskTypes() { return supportedTaskTypes; } + @Override + protected void doUnifiedCompletionInfer( + Model model, + UnifiedChatInput inputs, + TimeValue timeout, + ActionListener listener + ) { + throwUnsupportedUnifiedCompletionOperation(NAME); + } + @Override public void doInfer( Model model, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elastic/ElasticInferenceService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elastic/ElasticInferenceService.java index b256861e7dd2..fe8ee52eb881 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elastic/ElasticInferenceService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elastic/ElasticInferenceService.java @@ -38,6 +38,7 @@ import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; import org.elasticsearch.xpack.inference.external.http.sender.InferenceInputs; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; import org.elasticsearch.xpack.inference.services.ConfigurationParseContext; import org.elasticsearch.xpack.inference.services.SenderService; import org.elasticsearch.xpack.inference.services.ServiceComponents; @@ -57,6 +58,7 @@ import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrDefaultEmpty; import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrThrowIfNull; import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwIfNotEmptyMap; +import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwUnsupportedUnifiedCompletionOperation; public class ElasticInferenceService extends SenderService { @@ -76,6 +78,16 @@ public ElasticInferenceService( this.elasticInferenceServiceComponents = elasticInferenceServiceComponents; } + @Override + protected void doUnifiedCompletionInfer( + Model model, + UnifiedChatInput inputs, + TimeValue timeout, + ActionListener listener + ) { + throwUnsupportedUnifiedCompletionOperation(NAME); + } + @Override protected void doInfer( Model model, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalService.java index 0e64842f873d..5f613d6be586 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/elasticsearch/ElasticsearchInternalService.java @@ -31,6 +31,7 @@ import org.elasticsearch.inference.SettingsConfiguration; import org.elasticsearch.inference.TaskSettingsConfiguration; import org.elasticsearch.inference.TaskType; +import org.elasticsearch.inference.UnifiedCompletionRequest; import org.elasticsearch.inference.configuration.SettingsConfigurationDisplayType; import org.elasticsearch.inference.configuration.SettingsConfigurationFieldType; import org.elasticsearch.inference.configuration.SettingsConfigurationSelectOption; @@ -77,6 +78,7 @@ import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrDefaultEmpty; import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrThrowIfNull; import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwIfNotEmptyMap; +import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwUnsupportedUnifiedCompletionOperation; import static org.elasticsearch.xpack.inference.services.elasticsearch.ElasticsearchInternalServiceSettings.MODEL_ID; import static org.elasticsearch.xpack.inference.services.elasticsearch.ElasticsearchInternalServiceSettings.NUM_ALLOCATIONS; import static org.elasticsearch.xpack.inference.services.elasticsearch.ElasticsearchInternalServiceSettings.NUM_THREADS; @@ -578,6 +580,16 @@ private static CustomElandEmbeddingModel updateModelWithEmbeddingDetails(CustomE ); } + @Override + public void unifiedCompletionInfer( + Model model, + UnifiedCompletionRequest request, + TimeValue timeout, + ActionListener listener + ) { + throwUnsupportedUnifiedCompletionOperation(NAME); + } + @Override public void infer( Model model, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/googleaistudio/GoogleAiStudioService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/googleaistudio/GoogleAiStudioService.java index 57a8a66a3f3a..b681722a8213 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/googleaistudio/GoogleAiStudioService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/googleaistudio/GoogleAiStudioService.java @@ -39,6 +39,7 @@ import org.elasticsearch.xpack.inference.external.http.sender.GoogleAiStudioEmbeddingsRequestManager; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; import org.elasticsearch.xpack.inference.external.http.sender.InferenceInputs; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; import org.elasticsearch.xpack.inference.services.ConfigurationParseContext; import org.elasticsearch.xpack.inference.services.SenderService; import org.elasticsearch.xpack.inference.services.ServiceComponents; @@ -64,6 +65,7 @@ import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrDefaultEmpty; import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrThrowIfNull; import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwIfNotEmptyMap; +import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwUnsupportedUnifiedCompletionOperation; import static org.elasticsearch.xpack.inference.services.googleaistudio.GoogleAiStudioServiceFields.EMBEDDING_MAX_BATCH_SIZE; public class GoogleAiStudioService extends SenderService { @@ -282,9 +284,8 @@ protected void doInfer( ) { if (model instanceof GoogleAiStudioCompletionModel completionModel) { var requestManager = new GoogleAiStudioCompletionRequestManager(completionModel, getServiceComponents().threadPool()); - var docsOnly = DocumentsOnlyInput.of(inputs); var failedToSendRequestErrorMessage = constructFailedToSendRequestMessage( - completionModel.uri(docsOnly.stream()), + completionModel.uri(inputs.stream()), "Google AI Studio completion" ); var action = new SingleInputSenderExecutableAction( @@ -308,6 +309,16 @@ protected void doInfer( } } + @Override + protected void doUnifiedCompletionInfer( + Model model, + UnifiedChatInput inputs, + TimeValue timeout, + ActionListener listener + ) { + throwUnsupportedUnifiedCompletionOperation(NAME); + } + @Override protected void doChunkedInfer( Model model, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/googlevertexai/GoogleVertexAiService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/googlevertexai/GoogleVertexAiService.java index 857d475499aa..87a2d98dca92 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/googlevertexai/GoogleVertexAiService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/googlevertexai/GoogleVertexAiService.java @@ -35,6 +35,7 @@ import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; import org.elasticsearch.xpack.inference.external.http.sender.InferenceInputs; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; import org.elasticsearch.xpack.inference.services.ConfigurationParseContext; import org.elasticsearch.xpack.inference.services.SenderService; import org.elasticsearch.xpack.inference.services.ServiceComponents; @@ -57,6 +58,7 @@ import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrDefaultEmpty; import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrThrowIfNull; import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwIfNotEmptyMap; +import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwUnsupportedUnifiedCompletionOperation; import static org.elasticsearch.xpack.inference.services.googlevertexai.GoogleVertexAiServiceFields.EMBEDDING_MAX_BATCH_SIZE; import static org.elasticsearch.xpack.inference.services.googlevertexai.GoogleVertexAiServiceFields.LOCATION; import static org.elasticsearch.xpack.inference.services.googlevertexai.GoogleVertexAiServiceFields.PROJECT_ID; @@ -206,6 +208,16 @@ protected void doInfer( action.execute(inputs, timeout, listener); } + @Override + protected void doUnifiedCompletionInfer( + Model model, + UnifiedChatInput inputs, + TimeValue timeout, + ActionListener listener + ) { + throwUnsupportedUnifiedCompletionOperation(NAME); + } + @Override protected void doChunkedInfer( Model model, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/HuggingFaceService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/HuggingFaceService.java index 51cca72f2605..b74ec01cd76e 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/HuggingFaceService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/HuggingFaceService.java @@ -18,6 +18,7 @@ import org.elasticsearch.inference.ChunkingSettings; import org.elasticsearch.inference.EmptySettingsConfiguration; import org.elasticsearch.inference.InferenceServiceConfiguration; +import org.elasticsearch.inference.InferenceServiceResults; import org.elasticsearch.inference.InputType; import org.elasticsearch.inference.Model; import org.elasticsearch.inference.SettingsConfiguration; @@ -31,6 +32,7 @@ import org.elasticsearch.xpack.inference.external.action.huggingface.HuggingFaceActionCreator; import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; import org.elasticsearch.xpack.inference.services.ConfigurationParseContext; import org.elasticsearch.xpack.inference.services.ServiceComponents; import org.elasticsearch.xpack.inference.services.ServiceUtils; @@ -47,6 +49,7 @@ import static org.elasticsearch.xpack.inference.services.ServiceFields.URL; import static org.elasticsearch.xpack.inference.services.ServiceUtils.createInvalidModelException; +import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwUnsupportedUnifiedCompletionOperation; public class HuggingFaceService extends HuggingFaceBaseService { public static final String NAME = "hugging_face"; @@ -139,6 +142,16 @@ protected void doChunkedInfer( } } + @Override + protected void doUnifiedCompletionInfer( + Model model, + UnifiedChatInput inputs, + TimeValue timeout, + ActionListener listener + ) { + throwUnsupportedUnifiedCompletionOperation(NAME); + } + @Override public InferenceServiceConfiguration getConfiguration() { return Configuration.get(); diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/elser/HuggingFaceElserService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/elser/HuggingFaceElserService.java index 75920efa251f..5b038781b96a 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/elser/HuggingFaceElserService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/huggingface/elser/HuggingFaceElserService.java @@ -36,6 +36,7 @@ import org.elasticsearch.xpack.core.ml.inference.results.ErrorInferenceResults; import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; import org.elasticsearch.xpack.inference.services.ConfigurationParseContext; import org.elasticsearch.xpack.inference.services.ServiceComponents; import org.elasticsearch.xpack.inference.services.huggingface.HuggingFaceBaseService; @@ -49,6 +50,7 @@ import java.util.Map; import static org.elasticsearch.xpack.core.inference.results.ResultUtils.createInvalidChunkedResultException; +import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwUnsupportedUnifiedCompletionOperation; import static org.elasticsearch.xpack.inference.services.huggingface.elser.HuggingFaceElserServiceSettings.URL; public class HuggingFaceElserService extends HuggingFaceBaseService { @@ -81,6 +83,16 @@ protected HuggingFaceModel createModel( }; } + @Override + protected void doUnifiedCompletionInfer( + Model model, + UnifiedChatInput inputs, + TimeValue timeout, + ActionListener listener + ) { + throwUnsupportedUnifiedCompletionOperation(NAME); + } + @Override protected void doChunkedInfer( Model model, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxService.java index 981a3e95808e..cc66d5fd7ee7 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/ibmwatsonx/IbmWatsonxService.java @@ -37,6 +37,7 @@ import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; import org.elasticsearch.xpack.inference.external.http.sender.InferenceInputs; import org.elasticsearch.xpack.inference.external.http.sender.Sender; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; import org.elasticsearch.xpack.inference.services.ConfigurationParseContext; import org.elasticsearch.xpack.inference.services.SenderService; import org.elasticsearch.xpack.inference.services.ServiceComponents; @@ -57,6 +58,7 @@ import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrDefaultEmpty; import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrThrowIfNull; import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwIfNotEmptyMap; +import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwUnsupportedUnifiedCompletionOperation; import static org.elasticsearch.xpack.inference.services.huggingface.elser.HuggingFaceElserServiceSettings.URL; import static org.elasticsearch.xpack.inference.services.ibmwatsonx.IbmWatsonxServiceFields.API_VERSION; import static org.elasticsearch.xpack.inference.services.ibmwatsonx.IbmWatsonxServiceFields.EMBEDDING_MAX_BATCH_SIZE; @@ -276,6 +278,16 @@ protected void doInfer( action.execute(input, timeout, listener); } + @Override + protected void doUnifiedCompletionInfer( + Model model, + UnifiedChatInput inputs, + TimeValue timeout, + ActionListener listener + ) { + throwUnsupportedUnifiedCompletionOperation(NAME); + } + @Override protected void doChunkedInfer( Model model, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/mistral/MistralService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/mistral/MistralService.java index fe0edb851902..881e7d36f2a2 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/mistral/MistralService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/mistral/MistralService.java @@ -36,6 +36,7 @@ import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; import org.elasticsearch.xpack.inference.external.http.sender.InferenceInputs; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; import org.elasticsearch.xpack.inference.services.ConfigurationParseContext; import org.elasticsearch.xpack.inference.services.SenderService; import org.elasticsearch.xpack.inference.services.ServiceComponents; @@ -58,6 +59,7 @@ import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrDefaultEmpty; import static org.elasticsearch.xpack.inference.services.ServiceUtils.removeFromMapOrThrowIfNull; import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwIfNotEmptyMap; +import static org.elasticsearch.xpack.inference.services.ServiceUtils.throwUnsupportedUnifiedCompletionOperation; import static org.elasticsearch.xpack.inference.services.mistral.MistralConstants.MODEL_FIELD; public class MistralService extends SenderService { @@ -88,6 +90,16 @@ protected void doInfer( } } + @Override + protected void doUnifiedCompletionInfer( + Model model, + UnifiedChatInput inputs, + TimeValue timeout, + ActionListener listener + ) { + throwUnsupportedUnifiedCompletionOperation(NAME); + } + @Override protected void doChunkedInfer( Model model, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/openai/OpenAiService.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/openai/OpenAiService.java index 20ff1c617d21..7b51b068708c 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/openai/OpenAiService.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/openai/OpenAiService.java @@ -32,10 +32,13 @@ import org.elasticsearch.rest.RestStatus; import org.elasticsearch.xpack.inference.chunking.ChunkingSettingsBuilder; import org.elasticsearch.xpack.inference.chunking.EmbeddingRequestChunker; +import org.elasticsearch.xpack.inference.external.action.SenderExecutableAction; import org.elasticsearch.xpack.inference.external.action.openai.OpenAiActionCreator; import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; import org.elasticsearch.xpack.inference.external.http.sender.InferenceInputs; +import org.elasticsearch.xpack.inference.external.http.sender.OpenAiUnifiedCompletionRequestManager; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; import org.elasticsearch.xpack.inference.services.ConfigurationParseContext; import org.elasticsearch.xpack.inference.services.SenderService; import org.elasticsearch.xpack.inference.services.ServiceComponents; @@ -53,6 +56,8 @@ import java.util.Map; import java.util.Set; +import static org.elasticsearch.xpack.inference.external.action.ActionUtils.constructFailedToSendRequestMessage; +import static org.elasticsearch.xpack.inference.external.action.openai.OpenAiActionCreator.COMPLETION_ERROR_PREFIX; import static org.elasticsearch.xpack.inference.services.ServiceFields.MODEL_ID; import static org.elasticsearch.xpack.inference.services.ServiceFields.URL; import static org.elasticsearch.xpack.inference.services.ServiceUtils.createInvalidModelException; @@ -257,6 +262,28 @@ public void doInfer( action.execute(inputs, timeout, listener); } + @Override + public void doUnifiedCompletionInfer( + Model model, + UnifiedChatInput inputs, + TimeValue timeout, + ActionListener listener + ) { + if (model instanceof OpenAiChatCompletionModel == false) { + listener.onFailure(createInvalidModelException(model)); + return; + } + + OpenAiChatCompletionModel openAiModel = (OpenAiChatCompletionModel) model; + + var overriddenModel = OpenAiChatCompletionModel.of(openAiModel, inputs.getRequest()); + var requestCreator = OpenAiUnifiedCompletionRequestManager.of(overriddenModel, getServiceComponents().threadPool()); + var errorMessage = constructFailedToSendRequestMessage(overriddenModel.getServiceSettings().uri(), COMPLETION_ERROR_PREFIX); + var action = new SenderExecutableAction(getSender(), requestCreator, errorMessage); + + action.execute(inputs, timeout, listener); + } + @Override protected void doChunkedInfer( Model model, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/openai/completion/OpenAiChatCompletionModel.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/openai/completion/OpenAiChatCompletionModel.java index e721cd2955cf..7d79d64b3a77 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/openai/completion/OpenAiChatCompletionModel.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/openai/completion/OpenAiChatCompletionModel.java @@ -13,6 +13,7 @@ import org.elasticsearch.inference.ModelSecrets; import org.elasticsearch.inference.SettingsConfiguration; import org.elasticsearch.inference.TaskType; +import org.elasticsearch.inference.UnifiedCompletionRequest; import org.elasticsearch.inference.configuration.SettingsConfigurationDisplayType; import org.elasticsearch.inference.configuration.SettingsConfigurationFieldType; import org.elasticsearch.xpack.inference.external.action.ExecutableAction; @@ -24,6 +25,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import static org.elasticsearch.xpack.inference.services.openai.OpenAiServiceFields.USER; @@ -38,6 +40,26 @@ public static OpenAiChatCompletionModel of(OpenAiChatCompletionModel model, Map< return new OpenAiChatCompletionModel(model, OpenAiChatCompletionTaskSettings.of(model.getTaskSettings(), requestTaskSettings)); } + public static OpenAiChatCompletionModel of(OpenAiChatCompletionModel model, UnifiedCompletionRequest request) { + var originalModelServiceSettings = model.getServiceSettings(); + var overriddenServiceSettings = new OpenAiChatCompletionServiceSettings( + Objects.requireNonNullElse(request.model(), originalModelServiceSettings.modelId()), + originalModelServiceSettings.uri(), + originalModelServiceSettings.organizationId(), + originalModelServiceSettings.maxInputTokens(), + originalModelServiceSettings.rateLimitSettings() + ); + + return new OpenAiChatCompletionModel( + model.getInferenceEntityId(), + model.getTaskType(), + model.getConfigurations().getService(), + overriddenServiceSettings, + model.getTaskSettings(), + model.getSecretSettings() + ); + } + public OpenAiChatCompletionModel( String inferenceEntityId, TaskType taskType, diff --git a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/openai/completion/OpenAiChatCompletionRequestTaskSettings.java b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/openai/completion/OpenAiChatCompletionRequestTaskSettings.java index 8029d8579bab..7ef7f85d71a6 100644 --- a/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/openai/completion/OpenAiChatCompletionRequestTaskSettings.java +++ b/x-pack/plugin/inference/src/main/java/org/elasticsearch/xpack/inference/services/openai/completion/OpenAiChatCompletionRequestTaskSettings.java @@ -48,5 +48,4 @@ public static OpenAiChatCompletionRequestTaskSettings fromMap(Map TaskType.fromStringOrStatusException(null)); + assertThat(exception.getMessage(), Matchers.is("Task type must not be null")); + + exception = expectThrows(ElasticsearchStatusException.class, () -> TaskType.fromStringOrStatusException("blah")); + assertThat(exception.getMessage(), Matchers.is("Unknown task_type [blah]")); + + assertThat(TaskType.fromStringOrStatusException("any"), Matchers.is(TaskType.ANY)); + } + +} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/Utils.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/Utils.java index 5abb9000f4d0..9395ae222e9b 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/Utils.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/Utils.java @@ -19,6 +19,7 @@ import org.elasticsearch.inference.ModelConfigurations; import org.elasticsearch.inference.ModelSecrets; import org.elasticsearch.inference.SimilarityMeasure; +import org.elasticsearch.inference.TaskType; import org.elasticsearch.threadpool.ScalingExecutorBuilder; import org.elasticsearch.xpack.core.inference.results.ChatCompletionResults; import org.elasticsearch.xpack.inference.common.Truncator; @@ -160,9 +161,11 @@ public static Model getInvalidModel(String inferenceEntityId, String serviceName var mockConfigs = mock(ModelConfigurations.class); when(mockConfigs.getInferenceEntityId()).thenReturn(inferenceEntityId); when(mockConfigs.getService()).thenReturn(serviceName); + when(mockConfigs.getTaskType()).thenReturn(TaskType.TEXT_EMBEDDING); var mockModel = mock(Model.class); when(mockModel.getConfigurations()).thenReturn(mockConfigs); + when(mockModel.getTaskType()).thenReturn(TaskType.TEXT_EMBEDDING); return mockModel; } diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/BaseTransportInferenceActionTestCase.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/BaseTransportInferenceActionTestCase.java new file mode 100644 index 000000000000..47f3a0e0b57a --- /dev/null +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/BaseTransportInferenceActionTestCase.java @@ -0,0 +1,364 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference.action; + +import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.common.xcontent.ChunkedToXContent; +import org.elasticsearch.inference.InferenceService; +import org.elasticsearch.inference.InferenceServiceRegistry; +import org.elasticsearch.inference.InferenceServiceResults; +import org.elasticsearch.inference.Model; +import org.elasticsearch.inference.ModelConfigurations; +import org.elasticsearch.inference.TaskType; +import org.elasticsearch.inference.UnparsedModel; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.core.inference.action.BaseInferenceActionRequest; +import org.elasticsearch.xpack.core.inference.action.InferenceAction; +import org.elasticsearch.xpack.inference.action.task.StreamingTaskManager; +import org.elasticsearch.xpack.inference.registry.ModelRegistry; +import org.elasticsearch.xpack.inference.telemetry.InferenceStats; +import org.junit.Before; +import org.mockito.ArgumentCaptor; + +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.Flow; +import java.util.function.Consumer; + +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.isA; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.assertArg; +import static org.mockito.ArgumentMatchers.same; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public abstract class BaseTransportInferenceActionTestCase extends ESTestCase { + private ModelRegistry modelRegistry; + private StreamingTaskManager streamingTaskManager; + private BaseTransportInferenceAction action; + + protected static final String serviceId = "serviceId"; + protected static final TaskType taskType = TaskType.COMPLETION; + protected static final String inferenceId = "inferenceEntityId"; + protected InferenceServiceRegistry serviceRegistry; + protected InferenceStats inferenceStats; + + @Before + public void setUp() throws Exception { + super.setUp(); + TransportService transportService = mock(); + ActionFilters actionFilters = mock(); + modelRegistry = mock(); + serviceRegistry = mock(); + inferenceStats = new InferenceStats(mock(), mock()); + streamingTaskManager = mock(); + action = createAction(transportService, actionFilters, modelRegistry, serviceRegistry, inferenceStats, streamingTaskManager); + } + + protected abstract BaseTransportInferenceAction createAction( + TransportService transportService, + ActionFilters actionFilters, + ModelRegistry modelRegistry, + InferenceServiceRegistry serviceRegistry, + InferenceStats inferenceStats, + StreamingTaskManager streamingTaskManager + ); + + protected abstract Request createRequest(); + + public void testMetricsAfterModelRegistryError() { + var expectedException = new IllegalStateException("hello"); + var expectedError = expectedException.getClass().getSimpleName(); + + doAnswer(ans -> { + ActionListener listener = ans.getArgument(1); + listener.onFailure(expectedException); + return null; + }).when(modelRegistry).getModelWithSecrets(any(), any()); + + var listener = doExecute(taskType); + verify(listener).onFailure(same(expectedException)); + + verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { + assertThat(attributes.get("service"), nullValue()); + assertThat(attributes.get("task_type"), nullValue()); + assertThat(attributes.get("model_id"), nullValue()); + assertThat(attributes.get("status_code"), nullValue()); + assertThat(attributes.get("error.type"), is(expectedError)); + })); + } + + protected ActionListener doExecute(TaskType taskType) { + return doExecute(taskType, false); + } + + protected ActionListener doExecute(TaskType taskType, boolean stream) { + Request request = createRequest(); + when(request.getInferenceEntityId()).thenReturn(inferenceId); + when(request.getTaskType()).thenReturn(taskType); + when(request.isStreaming()).thenReturn(stream); + ActionListener listener = mock(); + action.doExecute(mock(), request, listener); + return listener; + } + + public void testMetricsAfterMissingService() { + mockModelRegistry(taskType); + + when(serviceRegistry.getService(any())).thenReturn(Optional.empty()); + + var listener = doExecute(taskType); + + verify(listener).onFailure(assertArg(e -> { + assertThat(e, isA(ElasticsearchStatusException.class)); + assertThat(e.getMessage(), is("Unknown service [" + serviceId + "] for model [" + inferenceId + "]. ")); + assertThat(((ElasticsearchStatusException) e).status(), is(RestStatus.BAD_REQUEST)); + })); + verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { + assertThat(attributes.get("service"), is(serviceId)); + assertThat(attributes.get("task_type"), is(taskType.toString())); + assertThat(attributes.get("model_id"), nullValue()); + assertThat(attributes.get("status_code"), is(RestStatus.BAD_REQUEST.getStatus())); + assertThat(attributes.get("error.type"), is(String.valueOf(RestStatus.BAD_REQUEST.getStatus()))); + })); + } + + protected void mockModelRegistry(TaskType expectedTaskType) { + var unparsedModel = new UnparsedModel(inferenceId, expectedTaskType, serviceId, Map.of(), Map.of()); + doAnswer(ans -> { + ActionListener listener = ans.getArgument(1); + listener.onResponse(unparsedModel); + return null; + }).when(modelRegistry).getModelWithSecrets(any(), any()); + } + + public void testMetricsAfterUnknownTaskType() { + var modelTaskType = TaskType.RERANK; + var requestTaskType = TaskType.SPARSE_EMBEDDING; + mockModelRegistry(modelTaskType); + when(serviceRegistry.getService(any())).thenReturn(Optional.of(mock())); + + var listener = doExecute(requestTaskType); + + verify(listener).onFailure(assertArg(e -> { + assertThat(e, isA(ElasticsearchStatusException.class)); + assertThat( + e.getMessage(), + is( + "Incompatible task_type, the requested type [" + + requestTaskType + + "] does not match the model type [" + + modelTaskType + + "]" + ) + ); + assertThat(((ElasticsearchStatusException) e).status(), is(RestStatus.BAD_REQUEST)); + })); + verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { + assertThat(attributes.get("service"), is(serviceId)); + assertThat(attributes.get("task_type"), is(modelTaskType.toString())); + assertThat(attributes.get("model_id"), nullValue()); + assertThat(attributes.get("status_code"), is(RestStatus.BAD_REQUEST.getStatus())); + assertThat(attributes.get("error.type"), is(String.valueOf(RestStatus.BAD_REQUEST.getStatus()))); + })); + } + + public void testMetricsAfterInferError() { + var expectedException = new IllegalStateException("hello"); + var expectedError = expectedException.getClass().getSimpleName(); + mockService(listener -> listener.onFailure(expectedException)); + + var listener = doExecute(taskType); + + verify(listener).onFailure(same(expectedException)); + verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { + assertThat(attributes.get("service"), is(serviceId)); + assertThat(attributes.get("task_type"), is(taskType.toString())); + assertThat(attributes.get("model_id"), nullValue()); + assertThat(attributes.get("status_code"), nullValue()); + assertThat(attributes.get("error.type"), is(expectedError)); + })); + } + + public void testMetricsAfterStreamUnsupported() { + var expectedStatus = RestStatus.METHOD_NOT_ALLOWED; + var expectedError = String.valueOf(expectedStatus.getStatus()); + mockService(l -> {}); + + var listener = doExecute(taskType, true); + + verify(listener).onFailure(assertArg(e -> { + assertThat(e, isA(ElasticsearchStatusException.class)); + var ese = (ElasticsearchStatusException) e; + assertThat(ese.getMessage(), is("Streaming is not allowed for service [" + serviceId + "].")); + assertThat(ese.status(), is(expectedStatus)); + })); + verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { + assertThat(attributes.get("service"), is(serviceId)); + assertThat(attributes.get("task_type"), is(taskType.toString())); + assertThat(attributes.get("model_id"), nullValue()); + assertThat(attributes.get("status_code"), is(expectedStatus.getStatus())); + assertThat(attributes.get("error.type"), is(expectedError)); + })); + } + + public void testMetricsAfterInferSuccess() { + mockService(listener -> listener.onResponse(mock())); + + var listener = doExecute(taskType); + + verify(listener).onResponse(any()); + verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { + assertThat(attributes.get("service"), is(serviceId)); + assertThat(attributes.get("task_type"), is(taskType.toString())); + assertThat(attributes.get("model_id"), nullValue()); + assertThat(attributes.get("status_code"), is(200)); + assertThat(attributes.get("error.type"), nullValue()); + })); + } + + public void testMetricsAfterStreamInferSuccess() { + mockStreamResponse(Flow.Subscriber::onComplete); + verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { + assertThat(attributes.get("service"), is(serviceId)); + assertThat(attributes.get("task_type"), is(taskType.toString())); + assertThat(attributes.get("model_id"), nullValue()); + assertThat(attributes.get("status_code"), is(200)); + assertThat(attributes.get("error.type"), nullValue()); + })); + } + + public void testMetricsAfterStreamInferFailure() { + var expectedException = new IllegalStateException("hello"); + var expectedError = expectedException.getClass().getSimpleName(); + mockStreamResponse(subscriber -> { + subscriber.subscribe(mock()); + subscriber.onError(expectedException); + }); + verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { + assertThat(attributes.get("service"), is(serviceId)); + assertThat(attributes.get("task_type"), is(taskType.toString())); + assertThat(attributes.get("model_id"), nullValue()); + assertThat(attributes.get("status_code"), nullValue()); + assertThat(attributes.get("error.type"), is(expectedError)); + })); + } + + public void testMetricsAfterStreamCancel() { + var response = mockStreamResponse(s -> s.onSubscribe(mock())); + response.subscribe(new Flow.Subscriber<>() { + @Override + public void onSubscribe(Flow.Subscription subscription) { + subscription.cancel(); + } + + @Override + public void onNext(ChunkedToXContent item) { + + } + + @Override + public void onError(Throwable throwable) { + + } + + @Override + public void onComplete() { + + } + }); + + verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { + assertThat(attributes.get("service"), is(serviceId)); + assertThat(attributes.get("task_type"), is(taskType.toString())); + assertThat(attributes.get("model_id"), nullValue()); + assertThat(attributes.get("status_code"), is(200)); + assertThat(attributes.get("error.type"), nullValue()); + })); + } + + protected Flow.Publisher mockStreamResponse(Consumer> action) { + mockService(true, Set.of(), listener -> { + Flow.Processor taskProcessor = mock(); + doAnswer(innerAns -> { + action.accept(innerAns.getArgument(0)); + return null; + }).when(taskProcessor).subscribe(any()); + when(streamingTaskManager.create(any(), any())).thenReturn(taskProcessor); + var inferenceServiceResults = mock(InferenceServiceResults.class); + when(inferenceServiceResults.publisher()).thenReturn(mock()); + listener.onResponse(inferenceServiceResults); + }); + + var listener = doExecute(taskType, true); + var captor = ArgumentCaptor.forClass(InferenceAction.Response.class); + verify(listener).onResponse(captor.capture()); + assertTrue(captor.getValue().isStreaming()); + assertNotNull(captor.getValue().publisher()); + return captor.getValue().publisher(); + } + + protected void mockService(Consumer> listenerAction) { + mockService(false, Set.of(), listenerAction); + } + + protected void mockService( + boolean stream, + Set supportedStreamingTasks, + Consumer> listenerAction + ) { + InferenceService service = mock(); + Model model = mockModel(); + when(service.parsePersistedConfigWithSecrets(any(), any(), any(), any())).thenReturn(model); + when(service.name()).thenReturn(serviceId); + + when(service.canStream(any())).thenReturn(stream); + when(service.supportedStreamingTasks()).thenReturn(supportedStreamingTasks); + doAnswer(ans -> { + listenerAction.accept(ans.getArgument(7)); + return null; + }).when(service).infer(any(), any(), any(), anyBoolean(), any(), any(), any(), any()); + doAnswer(ans -> { + listenerAction.accept(ans.getArgument(3)); + return null; + }).when(service).unifiedCompletionInfer(any(), any(), any(), any()); + mockModelAndServiceRegistry(service); + } + + protected Model mockModel() { + Model model = mock(); + ModelConfigurations modelConfigurations = mock(); + when(modelConfigurations.getService()).thenReturn(serviceId); + when(model.getConfigurations()).thenReturn(modelConfigurations); + when(model.getTaskType()).thenReturn(taskType); + when(model.getServiceSettings()).thenReturn(mock()); + return model; + } + + protected void mockModelAndServiceRegistry(InferenceService service) { + var unparsedModel = new UnparsedModel(inferenceId, taskType, serviceId, Map.of(), Map.of()); + doAnswer(ans -> { + ActionListener listener = ans.getArgument(1); + listener.onResponse(unparsedModel); + return null; + }).when(modelRegistry).getModelWithSecrets(any(), any()); + + when(serviceRegistry.getService(any())).thenReturn(Optional.of(service)); + } +} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/TransportInferenceActionTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/TransportInferenceActionTests.java index 0ed9cbf56b3f..e54175cb2700 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/TransportInferenceActionTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/TransportInferenceActionTests.java @@ -7,66 +7,28 @@ package org.elasticsearch.xpack.inference.action; -import org.elasticsearch.ElasticsearchStatusException; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; -import org.elasticsearch.common.xcontent.ChunkedToXContent; -import org.elasticsearch.inference.InferenceService; import org.elasticsearch.inference.InferenceServiceRegistry; -import org.elasticsearch.inference.InferenceServiceResults; -import org.elasticsearch.inference.Model; -import org.elasticsearch.inference.ModelConfigurations; -import org.elasticsearch.inference.TaskType; -import org.elasticsearch.inference.UnparsedModel; -import org.elasticsearch.rest.RestStatus; -import org.elasticsearch.test.ESTestCase; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xpack.core.inference.action.InferenceAction; import org.elasticsearch.xpack.inference.action.task.StreamingTaskManager; import org.elasticsearch.xpack.inference.registry.ModelRegistry; import org.elasticsearch.xpack.inference.telemetry.InferenceStats; -import org.junit.Before; -import org.mockito.ArgumentCaptor; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.Flow; -import java.util.function.Consumer; - -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.isA; -import static org.hamcrest.Matchers.nullValue; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyBoolean; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.assertArg; -import static org.mockito.ArgumentMatchers.same; -import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; -public class TransportInferenceActionTests extends ESTestCase { - private static final String serviceId = "serviceId"; - private static final TaskType taskType = TaskType.COMPLETION; - private static final String inferenceId = "inferenceEntityId"; - private ModelRegistry modelRegistry; - private InferenceServiceRegistry serviceRegistry; - private InferenceStats inferenceStats; - private StreamingTaskManager streamingTaskManager; - private TransportInferenceAction action; +public class TransportInferenceActionTests extends BaseTransportInferenceActionTestCase { - @Before - public void setUp() throws Exception { - super.setUp(); - TransportService transportService = mock(); - ActionFilters actionFilters = mock(); - modelRegistry = mock(); - serviceRegistry = mock(); - inferenceStats = new InferenceStats(mock(), mock()); - streamingTaskManager = mock(); - action = new TransportInferenceAction( + @Override + protected BaseTransportInferenceAction createAction( + TransportService transportService, + ActionFilters actionFilters, + ModelRegistry modelRegistry, + InferenceServiceRegistry serviceRegistry, + InferenceStats inferenceStats, + StreamingTaskManager streamingTaskManager + ) { + return new TransportInferenceAction( transportService, actionFilters, modelRegistry, @@ -76,279 +38,8 @@ public void setUp() throws Exception { ); } - public void testMetricsAfterModelRegistryError() { - var expectedException = new IllegalStateException("hello"); - var expectedError = expectedException.getClass().getSimpleName(); - - doAnswer(ans -> { - ActionListener listener = ans.getArgument(1); - listener.onFailure(expectedException); - return null; - }).when(modelRegistry).getModelWithSecrets(any(), any()); - - var listener = doExecute(taskType); - verify(listener).onFailure(same(expectedException)); - - verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { - assertThat(attributes.get("service"), nullValue()); - assertThat(attributes.get("task_type"), nullValue()); - assertThat(attributes.get("model_id"), nullValue()); - assertThat(attributes.get("status_code"), nullValue()); - assertThat(attributes.get("error.type"), is(expectedError)); - })); - } - - private ActionListener doExecute(TaskType taskType) { - return doExecute(taskType, false); - } - - private ActionListener doExecute(TaskType taskType, boolean stream) { - InferenceAction.Request request = mock(); - when(request.getInferenceEntityId()).thenReturn(inferenceId); - when(request.getTaskType()).thenReturn(taskType); - when(request.isStreaming()).thenReturn(stream); - ActionListener listener = mock(); - action.doExecute(mock(), request, listener); - return listener; - } - - public void testMetricsAfterMissingService() { - mockModelRegistry(taskType); - - when(serviceRegistry.getService(any())).thenReturn(Optional.empty()); - - var listener = doExecute(taskType); - - verify(listener).onFailure(assertArg(e -> { - assertThat(e, isA(ElasticsearchStatusException.class)); - assertThat(e.getMessage(), is("Unknown service [" + serviceId + "] for model [" + inferenceId + "]. ")); - assertThat(((ElasticsearchStatusException) e).status(), is(RestStatus.BAD_REQUEST)); - })); - verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { - assertThat(attributes.get("service"), is(serviceId)); - assertThat(attributes.get("task_type"), is(taskType.toString())); - assertThat(attributes.get("model_id"), nullValue()); - assertThat(attributes.get("status_code"), is(RestStatus.BAD_REQUEST.getStatus())); - assertThat(attributes.get("error.type"), is(String.valueOf(RestStatus.BAD_REQUEST.getStatus()))); - })); - } - - private void mockModelRegistry(TaskType expectedTaskType) { - var unparsedModel = new UnparsedModel(inferenceId, expectedTaskType, serviceId, Map.of(), Map.of()); - doAnswer(ans -> { - ActionListener listener = ans.getArgument(1); - listener.onResponse(unparsedModel); - return null; - }).when(modelRegistry).getModelWithSecrets(any(), any()); - } - - public void testMetricsAfterUnknownTaskType() { - var modelTaskType = TaskType.RERANK; - var requestTaskType = TaskType.SPARSE_EMBEDDING; - mockModelRegistry(modelTaskType); - when(serviceRegistry.getService(any())).thenReturn(Optional.of(mock())); - - var listener = doExecute(requestTaskType); - - verify(listener).onFailure(assertArg(e -> { - assertThat(e, isA(ElasticsearchStatusException.class)); - assertThat( - e.getMessage(), - is( - "Incompatible task_type, the requested type [" - + requestTaskType - + "] does not match the model type [" - + modelTaskType - + "]" - ) - ); - assertThat(((ElasticsearchStatusException) e).status(), is(RestStatus.BAD_REQUEST)); - })); - verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { - assertThat(attributes.get("service"), is(serviceId)); - assertThat(attributes.get("task_type"), is(modelTaskType.toString())); - assertThat(attributes.get("model_id"), nullValue()); - assertThat(attributes.get("status_code"), is(RestStatus.BAD_REQUEST.getStatus())); - assertThat(attributes.get("error.type"), is(String.valueOf(RestStatus.BAD_REQUEST.getStatus()))); - })); - } - - public void testMetricsAfterInferError() { - var expectedException = new IllegalStateException("hello"); - var expectedError = expectedException.getClass().getSimpleName(); - mockService(listener -> listener.onFailure(expectedException)); - - var listener = doExecute(taskType); - - verify(listener).onFailure(same(expectedException)); - verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { - assertThat(attributes.get("service"), is(serviceId)); - assertThat(attributes.get("task_type"), is(taskType.toString())); - assertThat(attributes.get("model_id"), nullValue()); - assertThat(attributes.get("status_code"), nullValue()); - assertThat(attributes.get("error.type"), is(expectedError)); - })); - } - - public void testMetricsAfterStreamUnsupported() { - var expectedStatus = RestStatus.METHOD_NOT_ALLOWED; - var expectedError = String.valueOf(expectedStatus.getStatus()); - mockService(l -> {}); - - var listener = doExecute(taskType, true); - - verify(listener).onFailure(assertArg(e -> { - assertThat(e, isA(ElasticsearchStatusException.class)); - var ese = (ElasticsearchStatusException) e; - assertThat(ese.getMessage(), is("Streaming is not allowed for service [" + serviceId + "].")); - assertThat(ese.status(), is(expectedStatus)); - })); - verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { - assertThat(attributes.get("service"), is(serviceId)); - assertThat(attributes.get("task_type"), is(taskType.toString())); - assertThat(attributes.get("model_id"), nullValue()); - assertThat(attributes.get("status_code"), is(expectedStatus.getStatus())); - assertThat(attributes.get("error.type"), is(expectedError)); - })); - } - - public void testMetricsAfterInferSuccess() { - mockService(listener -> listener.onResponse(mock())); - - var listener = doExecute(taskType); - - verify(listener).onResponse(any()); - verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { - assertThat(attributes.get("service"), is(serviceId)); - assertThat(attributes.get("task_type"), is(taskType.toString())); - assertThat(attributes.get("model_id"), nullValue()); - assertThat(attributes.get("status_code"), is(200)); - assertThat(attributes.get("error.type"), nullValue()); - })); - } - - public void testMetricsAfterStreamInferSuccess() { - mockStreamResponse(Flow.Subscriber::onComplete); - verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { - assertThat(attributes.get("service"), is(serviceId)); - assertThat(attributes.get("task_type"), is(taskType.toString())); - assertThat(attributes.get("model_id"), nullValue()); - assertThat(attributes.get("status_code"), is(200)); - assertThat(attributes.get("error.type"), nullValue()); - })); - } - - public void testMetricsAfterStreamInferFailure() { - var expectedException = new IllegalStateException("hello"); - var expectedError = expectedException.getClass().getSimpleName(); - mockStreamResponse(subscriber -> { - subscriber.subscribe(mock()); - subscriber.onError(expectedException); - }); - verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { - assertThat(attributes.get("service"), is(serviceId)); - assertThat(attributes.get("task_type"), is(taskType.toString())); - assertThat(attributes.get("model_id"), nullValue()); - assertThat(attributes.get("status_code"), nullValue()); - assertThat(attributes.get("error.type"), is(expectedError)); - })); - } - - public void testMetricsAfterStreamCancel() { - var response = mockStreamResponse(s -> s.onSubscribe(mock())); - response.subscribe(new Flow.Subscriber<>() { - @Override - public void onSubscribe(Flow.Subscription subscription) { - subscription.cancel(); - } - - @Override - public void onNext(ChunkedToXContent item) { - - } - - @Override - public void onError(Throwable throwable) { - - } - - @Override - public void onComplete() { - - } - }); - - verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { - assertThat(attributes.get("service"), is(serviceId)); - assertThat(attributes.get("task_type"), is(taskType.toString())); - assertThat(attributes.get("model_id"), nullValue()); - assertThat(attributes.get("status_code"), is(200)); - assertThat(attributes.get("error.type"), nullValue()); - })); - } - - private Flow.Publisher mockStreamResponse(Consumer> action) { - mockService(true, Set.of(), listener -> { - Flow.Processor taskProcessor = mock(); - doAnswer(innerAns -> { - action.accept(innerAns.getArgument(0)); - return null; - }).when(taskProcessor).subscribe(any()); - when(streamingTaskManager.create(any(), any())).thenReturn(taskProcessor); - var inferenceServiceResults = mock(InferenceServiceResults.class); - when(inferenceServiceResults.publisher()).thenReturn(mock()); - listener.onResponse(inferenceServiceResults); - }); - - var listener = doExecute(taskType, true); - var captor = ArgumentCaptor.forClass(InferenceAction.Response.class); - verify(listener).onResponse(captor.capture()); - assertTrue(captor.getValue().isStreaming()); - assertNotNull(captor.getValue().publisher()); - return captor.getValue().publisher(); - } - - private void mockService(Consumer> listenerAction) { - mockService(false, Set.of(), listenerAction); - } - - private void mockService( - boolean stream, - Set supportedStreamingTasks, - Consumer> listenerAction - ) { - InferenceService service = mock(); - Model model = mockModel(); - when(service.parsePersistedConfigWithSecrets(any(), any(), any(), any())).thenReturn(model); - when(service.name()).thenReturn(serviceId); - - when(service.canStream(any())).thenReturn(stream); - when(service.supportedStreamingTasks()).thenReturn(supportedStreamingTasks); - doAnswer(ans -> { - listenerAction.accept(ans.getArgument(7)); - return null; - }).when(service).infer(any(), any(), any(), anyBoolean(), any(), any(), any(), any()); - mockModelAndServiceRegistry(service); - } - - private Model mockModel() { - Model model = mock(); - ModelConfigurations modelConfigurations = mock(); - when(modelConfigurations.getService()).thenReturn(serviceId); - when(model.getConfigurations()).thenReturn(modelConfigurations); - when(model.getTaskType()).thenReturn(taskType); - when(model.getServiceSettings()).thenReturn(mock()); - return model; - } - - private void mockModelAndServiceRegistry(InferenceService service) { - var unparsedModel = new UnparsedModel(inferenceId, taskType, serviceId, Map.of(), Map.of()); - doAnswer(ans -> { - ActionListener listener = ans.getArgument(1); - listener.onResponse(unparsedModel); - return null; - }).when(modelRegistry).getModelWithSecrets(any(), any()); - - when(serviceRegistry.getService(any())).thenReturn(Optional.of(service)); + @Override + protected InferenceAction.Request createRequest() { + return mock(); } } diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/TransportUnifiedCompletionActionTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/TransportUnifiedCompletionActionTests.java new file mode 100644 index 000000000000..4c943599ce52 --- /dev/null +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/action/TransportUnifiedCompletionActionTests.java @@ -0,0 +1,124 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference.action; + +import org.elasticsearch.ElasticsearchStatusException; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.inference.InferenceServiceRegistry; +import org.elasticsearch.inference.TaskType; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.core.inference.action.UnifiedCompletionAction; +import org.elasticsearch.xpack.inference.action.task.StreamingTaskManager; +import org.elasticsearch.xpack.inference.registry.ModelRegistry; +import org.elasticsearch.xpack.inference.telemetry.InferenceStats; + +import java.util.Optional; + +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.isA; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.assertArg; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class TransportUnifiedCompletionActionTests extends BaseTransportInferenceActionTestCase { + + @Override + protected BaseTransportInferenceAction createAction( + TransportService transportService, + ActionFilters actionFilters, + ModelRegistry modelRegistry, + InferenceServiceRegistry serviceRegistry, + InferenceStats inferenceStats, + StreamingTaskManager streamingTaskManager + ) { + return new TransportUnifiedCompletionInferenceAction( + transportService, + actionFilters, + modelRegistry, + serviceRegistry, + inferenceStats, + streamingTaskManager + ); + } + + @Override + protected UnifiedCompletionAction.Request createRequest() { + return mock(); + } + + public void testThrows_IncompatibleTaskTypeException_WhenUsingATextEmbeddingInferenceEndpoint() { + var modelTaskType = TaskType.TEXT_EMBEDDING; + var requestTaskType = TaskType.TEXT_EMBEDDING; + mockModelRegistry(modelTaskType); + when(serviceRegistry.getService(any())).thenReturn(Optional.of(mock())); + + var listener = doExecute(requestTaskType); + + verify(listener).onFailure(assertArg(e -> { + assertThat(e, isA(ElasticsearchStatusException.class)); + assertThat( + e.getMessage(), + is("Incompatible task_type for unified API, the requested type [" + requestTaskType + "] must be one of [completion]") + ); + assertThat(((ElasticsearchStatusException) e).status(), is(RestStatus.BAD_REQUEST)); + })); + verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { + assertThat(attributes.get("service"), is(serviceId)); + assertThat(attributes.get("task_type"), is(modelTaskType.toString())); + assertThat(attributes.get("model_id"), nullValue()); + assertThat(attributes.get("status_code"), is(RestStatus.BAD_REQUEST.getStatus())); + assertThat(attributes.get("error.type"), is(String.valueOf(RestStatus.BAD_REQUEST.getStatus()))); + })); + } + + public void testThrows_IncompatibleTaskTypeException_WhenUsingRequestIsAny_ModelIsTextEmbedding() { + var modelTaskType = TaskType.ANY; + var requestTaskType = TaskType.TEXT_EMBEDDING; + mockModelRegistry(modelTaskType); + when(serviceRegistry.getService(any())).thenReturn(Optional.of(mock())); + + var listener = doExecute(requestTaskType); + + verify(listener).onFailure(assertArg(e -> { + assertThat(e, isA(ElasticsearchStatusException.class)); + assertThat( + e.getMessage(), + is("Incompatible task_type for unified API, the requested type [" + requestTaskType + "] must be one of [completion]") + ); + assertThat(((ElasticsearchStatusException) e).status(), is(RestStatus.BAD_REQUEST)); + })); + verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { + assertThat(attributes.get("service"), is(serviceId)); + assertThat(attributes.get("task_type"), is(modelTaskType.toString())); + assertThat(attributes.get("model_id"), nullValue()); + assertThat(attributes.get("status_code"), is(RestStatus.BAD_REQUEST.getStatus())); + assertThat(attributes.get("error.type"), is(String.valueOf(RestStatus.BAD_REQUEST.getStatus()))); + })); + } + + public void testMetricsAfterUnifiedInferSuccess_WithRequestTaskTypeAny() { + mockModelRegistry(TaskType.COMPLETION); + mockService(listener -> listener.onResponse(mock())); + + var listener = doExecute(TaskType.ANY); + + verify(listener).onResponse(any()); + verify(inferenceStats.inferenceDuration()).record(anyLong(), assertArg(attributes -> { + assertThat(attributes.get("service"), is(serviceId)); + assertThat(attributes.get("task_type"), is(taskType.toString())); + assertThat(attributes.get("model_id"), nullValue()); + assertThat(attributes.get("status_code"), is(200)); + assertThat(attributes.get("error.type"), nullValue()); + })); + } +} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/SingleInputSenderExecutableActionTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/SingleInputSenderExecutableActionTests.java index d4ab9b1f1e19..9e7c58b0ca79 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/SingleInputSenderExecutableActionTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/SingleInputSenderExecutableActionTests.java @@ -61,25 +61,11 @@ public void testOneInputIsValid() { assertTrue("Test failed to call listener.", testRan.get()); } - public void testInvalidInputType() { - var badInput = mock(InferenceInputs.class); - var actualException = new AtomicReference(); - - executableAction.execute( - badInput, - mock(TimeValue.class), - ActionListener.wrap(shouldNotSucceed -> fail("Test failed."), actualException::set) - ); - - assertThat(actualException.get(), notNullValue()); - assertThat(actualException.get().getMessage(), is("Invalid inference input type")); - assertThat(actualException.get(), instanceOf(ElasticsearchStatusException.class)); - assertThat(((ElasticsearchStatusException) actualException.get()).status(), is(RestStatus.INTERNAL_SERVER_ERROR)); - } - public void testMoreThanOneInput() { var badInput = mock(DocumentsOnlyInput.class); - when(badInput.getInputs()).thenReturn(List.of("one", "two")); + var input = List.of("one", "two"); + when(badInput.getInputs()).thenReturn(input); + when(badInput.inputSize()).thenReturn(input.size()); var actualException = new AtomicReference(); executableAction.execute( diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/amazonbedrock/AmazonBedrockActionCreatorTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/amazonbedrock/AmazonBedrockActionCreatorTests.java index 87d3a82b4aae..e7543aa6ba9e 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/amazonbedrock/AmazonBedrockActionCreatorTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/amazonbedrock/AmazonBedrockActionCreatorTests.java @@ -17,6 +17,7 @@ import org.elasticsearch.xpack.core.inference.results.ChatCompletionResults; import org.elasticsearch.xpack.core.inference.results.InferenceTextEmbeddingFloatResults; import org.elasticsearch.xpack.inference.external.amazonbedrock.AmazonBedrockMockRequestSender; +import org.elasticsearch.xpack.inference.external.http.sender.ChatCompletionInput; import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.services.ServiceComponentsTests; import org.elasticsearch.xpack.inference.services.amazonbedrock.AmazonBedrockProvider; @@ -130,7 +131,7 @@ public void testCompletionRequestAction() throws IOException { ); var action = creator.create(model, Map.of()); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var result = listener.actionGet(TIMEOUT); assertThat(result.asMap(), is(buildExpectationCompletion(List.of("test input string")))); @@ -163,7 +164,7 @@ public void testChatCompletionRequestAction_HandlesException() throws IOExceptio ); var action = creator.create(model, Map.of()); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); assertThat(sender.sendCount(), is(1)); diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/anthropic/AnthropicActionCreatorTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/anthropic/AnthropicActionCreatorTests.java index a3114300c5dd..f0de37ceaaf9 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/anthropic/AnthropicActionCreatorTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/anthropic/AnthropicActionCreatorTests.java @@ -20,7 +20,7 @@ import org.elasticsearch.xcontent.XContentType; import org.elasticsearch.xpack.core.inference.action.InferenceAction; import org.elasticsearch.xpack.inference.external.http.HttpClientManager; -import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; +import org.elasticsearch.xpack.inference.external.http.sender.ChatCompletionInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSenderTests; import org.elasticsearch.xpack.inference.external.request.anthropic.AnthropicRequestUtils; import org.elasticsearch.xpack.inference.logging.ThrottlerManager; @@ -49,6 +49,7 @@ import static org.mockito.Mockito.mock; public class AnthropicActionCreatorTests extends ESTestCase { + private static final TimeValue TIMEOUT = new TimeValue(30, TimeUnit.SECONDS); private final MockWebServer webServer = new MockWebServer(); private ThreadPool threadPool; @@ -103,7 +104,7 @@ public void testCreate_ChatCompletionModel() throws IOException { var action = actionCreator.create(model, overriddenTaskSettings); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var result = listener.actionGet(TIMEOUT); @@ -168,7 +169,7 @@ public void testCreate_ChatCompletionModel_FailsFromInvalidResponseFormat() thro var action = actionCreator.create(model, overriddenTaskSettings); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchStatusException.class, () -> listener.actionGet(TIMEOUT)); assertThat( diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/anthropic/AnthropicChatCompletionActionTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/anthropic/AnthropicChatCompletionActionTests.java index fca2e316af17..2065a726b758 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/anthropic/AnthropicChatCompletionActionTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/anthropic/AnthropicChatCompletionActionTests.java @@ -27,7 +27,7 @@ import org.elasticsearch.xpack.inference.external.action.SingleInputSenderExecutableAction; import org.elasticsearch.xpack.inference.external.http.HttpClientManager; import org.elasticsearch.xpack.inference.external.http.sender.AnthropicCompletionRequestManager; -import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; +import org.elasticsearch.xpack.inference.external.http.sender.ChatCompletionInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSenderTests; import org.elasticsearch.xpack.inference.external.http.sender.Sender; @@ -113,7 +113,7 @@ public void testExecute_ReturnsSuccessfulResponse() throws IOException { var action = createAction(getUrl(webServer), "secret", "model", 1, sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var result = listener.actionGet(TIMEOUT); @@ -149,7 +149,7 @@ public void testExecute_ThrowsElasticsearchException() { var action = createAction(getUrl(webServer), "secret", "model", 1, sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); @@ -170,7 +170,7 @@ public void testExecute_ThrowsElasticsearchException_WhenSenderOnFailureIsCalled var action = createAction(getUrl(webServer), "secret", "model", 1, sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); @@ -187,7 +187,7 @@ public void testExecute_ThrowsException() { var action = createAction(getUrl(webServer), "secret", "model", 1, sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); @@ -229,7 +229,7 @@ public void testExecute_ThrowsException_WhenInputIsGreaterThanOne() throws IOExc var action = createAction(getUrl(webServer), "secret", "model", 1, sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc", "def")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc", "def")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchStatusException.class, () -> listener.actionGet(TIMEOUT)); diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/azureaistudio/AzureAiStudioActionAndCreatorTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/azureaistudio/AzureAiStudioActionAndCreatorTests.java index 8792234102a9..210fab457de1 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/azureaistudio/AzureAiStudioActionAndCreatorTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/azureaistudio/AzureAiStudioActionAndCreatorTests.java @@ -21,6 +21,7 @@ import org.elasticsearch.xpack.core.inference.action.InferenceAction; import org.elasticsearch.xpack.inference.common.TruncatorTests; import org.elasticsearch.xpack.inference.external.http.HttpClientManager; +import org.elasticsearch.xpack.inference.external.http.sender.ChatCompletionInput; import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSenderTests; @@ -160,7 +161,7 @@ public void testChatCompletionRequestAction() throws IOException { var action = creator.create(model, Map.of()); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var result = listener.actionGet(TIMEOUT); diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/azureopenai/AzureOpenAiActionCreatorTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/azureopenai/AzureOpenAiActionCreatorTests.java index 45a2fb0954c7..7e1e3e55caed 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/azureopenai/AzureOpenAiActionCreatorTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/azureopenai/AzureOpenAiActionCreatorTests.java @@ -24,6 +24,7 @@ import org.elasticsearch.xpack.core.inference.action.InferenceAction; import org.elasticsearch.xpack.core.inference.results.ChatCompletionResults; import org.elasticsearch.xpack.inference.external.http.HttpClientManager; +import org.elasticsearch.xpack.inference.external.http.sender.ChatCompletionInput; import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSenderTests; import org.elasticsearch.xpack.inference.external.request.azureopenai.AzureOpenAiUtils; @@ -475,7 +476,7 @@ public void testInfer_AzureOpenAiCompletion_WithOverriddenUser() throws IOExcept var action = actionCreator.create(model, taskSettingsWithUserOverride); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of(completionInput)), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of(completionInput)), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var result = listener.actionGet(TIMEOUT); @@ -531,7 +532,7 @@ public void testInfer_AzureOpenAiCompletionModel_WithoutUser() throws IOExceptio var action = actionCreator.create(model, requestTaskSettingsWithoutUser); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of(completionInput)), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of(completionInput)), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var result = listener.actionGet(TIMEOUT); @@ -589,7 +590,7 @@ public void testInfer_AzureOpenAiCompletionModel_FailsFromInvalidResponseFormat( var action = actionCreator.create(model, requestTaskSettingsWithoutUser); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of(completionInput)), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of(completionInput)), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchStatusException.class, () -> listener.actionGet(TIMEOUT)); assertThat( diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/azureopenai/AzureOpenAiCompletionActionTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/azureopenai/AzureOpenAiCompletionActionTests.java index 4c7683c88281..dca12dfda9c9 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/azureopenai/AzureOpenAiCompletionActionTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/azureopenai/AzureOpenAiCompletionActionTests.java @@ -26,7 +26,7 @@ import org.elasticsearch.xpack.inference.external.action.SingleInputSenderExecutableAction; import org.elasticsearch.xpack.inference.external.http.HttpClientManager; import org.elasticsearch.xpack.inference.external.http.sender.AzureOpenAiCompletionRequestManager; -import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; +import org.elasticsearch.xpack.inference.external.http.sender.ChatCompletionInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSenderTests; import org.elasticsearch.xpack.inference.external.http.sender.Sender; import org.elasticsearch.xpack.inference.external.request.azureopenai.AzureOpenAiUtils; @@ -111,7 +111,7 @@ public void testExecute_ReturnsSuccessfulResponse() throws IOException { var action = createAction("resource", "deployment", "apiversion", user, apiKey, sender, "id"); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of(completionInput)), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of(completionInput)), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var result = listener.actionGet(TIMEOUT); @@ -142,7 +142,7 @@ public void testExecute_ThrowsElasticsearchException() { var action = createAction("resource", "deployment", "apiVersion", "user", "apikey", sender, "id"); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); @@ -163,7 +163,7 @@ public void testExecute_ThrowsElasticsearchException_WhenSenderOnFailureIsCalled var action = createAction("resource", "deployment", "apiVersion", "user", "apikey", sender, "id"); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); @@ -177,7 +177,7 @@ public void testExecute_ThrowsException() { var action = createAction("resource", "deployment", "apiVersion", "user", "apikey", sender, "id"); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/cohere/CohereActionCreatorTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/cohere/CohereActionCreatorTests.java index 9ec34e7d8e5c..3a512de25a39 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/cohere/CohereActionCreatorTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/cohere/CohereActionCreatorTests.java @@ -20,6 +20,7 @@ import org.elasticsearch.xcontent.XContentType; import org.elasticsearch.xpack.core.inference.action.InferenceAction; import org.elasticsearch.xpack.inference.external.http.HttpClientManager; +import org.elasticsearch.xpack.inference.external.http.sender.ChatCompletionInput; import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSenderTests; import org.elasticsearch.xpack.inference.logging.ThrottlerManager; @@ -197,7 +198,7 @@ public void testCreate_CohereCompletionModel_WithModelSpecified() throws IOExcep var action = actionCreator.create(model, Map.of()); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var result = listener.actionGet(TIMEOUT); @@ -257,7 +258,7 @@ public void testCreate_CohereCompletionModel_WithoutModelSpecified() throws IOEx var action = actionCreator.create(model, Map.of()); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var result = listener.actionGet(TIMEOUT); diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/cohere/CohereCompletionActionTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/cohere/CohereCompletionActionTests.java index ba839e0d7c5e..c5871adb3486 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/cohere/CohereCompletionActionTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/cohere/CohereCompletionActionTests.java @@ -26,8 +26,8 @@ import org.elasticsearch.xpack.inference.external.action.ExecutableAction; import org.elasticsearch.xpack.inference.external.action.SingleInputSenderExecutableAction; import org.elasticsearch.xpack.inference.external.http.HttpClientManager; +import org.elasticsearch.xpack.inference.external.http.sender.ChatCompletionInput; import org.elasticsearch.xpack.inference.external.http.sender.CohereCompletionRequestManager; -import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSenderTests; import org.elasticsearch.xpack.inference.external.http.sender.Sender; import org.elasticsearch.xpack.inference.external.request.cohere.CohereUtils; @@ -120,7 +120,7 @@ public void testExecute_ReturnsSuccessfulResponse_WithModelSpecified() throws IO var action = createAction(getUrl(webServer), "secret", "model", sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var result = listener.actionGet(TIMEOUT); @@ -181,7 +181,7 @@ public void testExecute_ReturnsSuccessfulResponse_WithoutModelSpecified() throws var action = createAction(getUrl(webServer), "secret", null, sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var result = listener.actionGet(TIMEOUT); @@ -214,7 +214,7 @@ public void testExecute_ThrowsElasticsearchException() { var action = createAction(getUrl(webServer), "secret", "model", sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); @@ -235,7 +235,7 @@ public void testExecute_ThrowsElasticsearchException_WhenSenderOnFailureIsCalled var action = createAction(getUrl(webServer), "secret", "model", sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); @@ -256,7 +256,7 @@ public void testExecute_ThrowsElasticsearchException_WhenSenderOnFailureIsCalled var action = createAction(null, "secret", "model", sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); @@ -270,7 +270,7 @@ public void testExecute_ThrowsException() { var action = createAction(getUrl(webServer), "secret", "model", sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); @@ -284,7 +284,7 @@ public void testExecute_ThrowsExceptionWithNullUrl() { var action = createAction(null, "secret", "model", sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); @@ -334,7 +334,7 @@ public void testExecute_ThrowsException_WhenInputIsGreaterThanOne() throws IOExc var action = createAction(getUrl(webServer), "secret", "model", sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc", "def")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc", "def")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchStatusException.class, () -> listener.actionGet(TIMEOUT)); diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/googleaistudio/GoogleAiStudioCompletionActionTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/googleaistudio/GoogleAiStudioCompletionActionTests.java index 72b5ffa45a0d..ff17bbf66e02 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/googleaistudio/GoogleAiStudioCompletionActionTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/googleaistudio/GoogleAiStudioCompletionActionTests.java @@ -25,7 +25,7 @@ import org.elasticsearch.xpack.inference.external.action.ExecutableAction; import org.elasticsearch.xpack.inference.external.action.SingleInputSenderExecutableAction; import org.elasticsearch.xpack.inference.external.http.HttpClientManager; -import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; +import org.elasticsearch.xpack.inference.external.http.sender.ChatCompletionInput; import org.elasticsearch.xpack.inference.external.http.sender.GoogleAiStudioCompletionRequestManager; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSenderTests; import org.elasticsearch.xpack.inference.external.http.sender.Sender; @@ -128,7 +128,7 @@ public void testExecute_ReturnsSuccessfulResponse() throws IOException { var action = createAction(getUrl(webServer), "secret", "model", sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("input")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("input")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var result = listener.actionGet(TIMEOUT); @@ -159,7 +159,7 @@ public void testExecute_ThrowsElasticsearchException() { var action = createAction(getUrl(webServer), "secret", "model", sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); @@ -180,7 +180,7 @@ public void testExecute_ThrowsElasticsearchException_WhenSenderOnFailureIsCalled var action = createAction(getUrl(webServer), "secret", "model", sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); @@ -197,7 +197,7 @@ public void testExecute_ThrowsException() { var action = createAction(getUrl(webServer), "secret", "model", sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); @@ -260,7 +260,7 @@ public void testExecute_ThrowsException_WhenInputIsGreaterThanOne() throws IOExc var action = createAction(getUrl(webServer), "secret", "model", sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc", "def")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc", "def")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchStatusException.class, () -> listener.actionGet(TIMEOUT)); diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/openai/OpenAiActionCreatorTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/openai/OpenAiActionCreatorTests.java index b6d7eb673b7f..fe076eb721ea 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/openai/OpenAiActionCreatorTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/openai/OpenAiActionCreatorTests.java @@ -21,6 +21,7 @@ import org.elasticsearch.xcontent.XContentType; import org.elasticsearch.xpack.core.inference.action.InferenceAction; import org.elasticsearch.xpack.inference.external.http.HttpClientManager; +import org.elasticsearch.xpack.inference.external.http.sender.ChatCompletionInput; import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSenderTests; import org.elasticsearch.xpack.inference.logging.ThrottlerManager; @@ -330,7 +331,7 @@ public void testCreate_OpenAiChatCompletionModel() throws IOException { var action = actionCreator.create(model, overriddenTaskSettings); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var result = listener.actionGet(TIMEOUT); @@ -345,11 +346,12 @@ public void testCreate_OpenAiChatCompletionModel() throws IOException { assertThat(request.getHeader(ORGANIZATION_HEADER), equalTo("org")); var requestMap = entityAsMap(webServer.requests().get(0).getBody()); - assertThat(requestMap.size(), is(4)); + assertThat(requestMap.size(), is(5)); assertThat(requestMap.get("messages"), is(List.of(Map.of("role", "user", "content", "abc")))); assertThat(requestMap.get("model"), is("model")); assertThat(requestMap.get("user"), is("overridden_user")); assertThat(requestMap.get("n"), is(1)); + assertThat(requestMap.get("stream"), is(false)); } } @@ -393,7 +395,7 @@ public void testCreate_OpenAiChatCompletionModel_WithoutUser() throws IOExceptio var action = actionCreator.create(model, overriddenTaskSettings); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var result = listener.actionGet(TIMEOUT); @@ -408,10 +410,11 @@ public void testCreate_OpenAiChatCompletionModel_WithoutUser() throws IOExceptio assertThat(request.getHeader(ORGANIZATION_HEADER), equalTo("org")); var requestMap = entityAsMap(webServer.requests().get(0).getBody()); - assertThat(requestMap.size(), is(3)); + assertThat(requestMap.size(), is(4)); assertThat(requestMap.get("messages"), is(List.of(Map.of("role", "user", "content", "abc")))); assertThat(requestMap.get("model"), is("model")); assertThat(requestMap.get("n"), is(1)); + assertThat(requestMap.get("stream"), is(false)); } } @@ -455,7 +458,7 @@ public void testCreate_OpenAiChatCompletionModel_WithoutOrganization() throws IO var action = actionCreator.create(model, overriddenTaskSettings); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var result = listener.actionGet(TIMEOUT); @@ -470,11 +473,12 @@ public void testCreate_OpenAiChatCompletionModel_WithoutOrganization() throws IO assertNull(request.getHeader(ORGANIZATION_HEADER)); var requestMap = entityAsMap(webServer.requests().get(0).getBody()); - assertThat(requestMap.size(), is(4)); + assertThat(requestMap.size(), is(5)); assertThat(requestMap.get("messages"), is(List.of(Map.of("role", "user", "content", "abc")))); assertThat(requestMap.get("model"), is("model")); assertThat(requestMap.get("user"), is("overridden_user")); assertThat(requestMap.get("n"), is(1)); + assertThat(requestMap.get("stream"), is(false)); } } @@ -523,7 +527,7 @@ public void testCreate_OpenAiChatCompletionModel_FailsFromInvalidResponseFormat( var action = actionCreator.create(model, overriddenTaskSettings); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchStatusException.class, () -> listener.actionGet(TIMEOUT)); assertThat( @@ -542,11 +546,12 @@ public void testCreate_OpenAiChatCompletionModel_FailsFromInvalidResponseFormat( assertNull(webServer.requests().get(0).getHeader(ORGANIZATION_HEADER)); var requestMap = entityAsMap(webServer.requests().get(0).getBody()); - assertThat(requestMap.size(), is(4)); + assertThat(requestMap.size(), is(5)); assertThat(requestMap.get("messages"), is(List.of(Map.of("role", "user", "content", "abc")))); assertThat(requestMap.get("model"), is("model")); assertThat(requestMap.get("user"), is("overridden_user")); assertThat(requestMap.get("n"), is(1)); + assertThat(requestMap.get("stream"), is(false)); } } diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/openai/OpenAiChatCompletionActionTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/openai/OpenAiChatCompletionActionTests.java index d84b2b5bb324..ba74d2ab42c2 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/openai/OpenAiChatCompletionActionTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/action/openai/OpenAiChatCompletionActionTests.java @@ -27,7 +27,7 @@ import org.elasticsearch.xpack.inference.external.action.ExecutableAction; import org.elasticsearch.xpack.inference.external.action.SingleInputSenderExecutableAction; import org.elasticsearch.xpack.inference.external.http.HttpClientManager; -import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; +import org.elasticsearch.xpack.inference.external.http.sender.ChatCompletionInput; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSenderTests; import org.elasticsearch.xpack.inference.external.http.sender.OpenAiCompletionRequestManager; @@ -119,7 +119,7 @@ public void testExecute_ReturnsSuccessfulResponse() throws IOException { var action = createAction(getUrl(webServer), "org", "secret", "model", "user", sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var result = listener.actionGet(TIMEOUT); @@ -134,11 +134,12 @@ public void testExecute_ReturnsSuccessfulResponse() throws IOException { assertThat(request.getHeader(ORGANIZATION_HEADER), equalTo("org")); var requestMap = entityAsMap(request.getBody()); - assertThat(requestMap.size(), is(4)); + assertThat(requestMap.size(), is(5)); assertThat(requestMap.get("messages"), is(List.of(Map.of("role", "user", "content", "abc")))); assertThat(requestMap.get("model"), is("model")); assertThat(requestMap.get("user"), is("user")); assertThat(requestMap.get("n"), is(1)); + assertThat(requestMap.get("stream"), is(false)); } } @@ -159,7 +160,7 @@ public void testExecute_ThrowsElasticsearchException() { var action = createAction(getUrl(webServer), "org", "secret", "model", "user", sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); @@ -180,7 +181,7 @@ public void testExecute_ThrowsElasticsearchException_WhenSenderOnFailureIsCalled var action = createAction(getUrl(webServer), "org", "secret", "model", "user", sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); @@ -201,7 +202,7 @@ public void testExecute_ThrowsElasticsearchException_WhenSenderOnFailureIsCalled var action = createAction(null, "org", "secret", "model", "user", sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); @@ -215,7 +216,7 @@ public void testExecute_ThrowsException() { var action = createAction(getUrl(webServer), "org", "secret", "model", "user", sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); @@ -229,7 +230,7 @@ public void testExecute_ThrowsExceptionWithNullUrl() { var action = createAction(null, "org", "secret", "model", "user", sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchException.class, () -> listener.actionGet(TIMEOUT)); @@ -273,7 +274,7 @@ public void testExecute_ThrowsException_WhenInputIsGreaterThanOne() throws IOExc var action = createAction(getUrl(webServer), "org", "secret", "model", "user", sender); PlainActionFuture listener = new PlainActionFuture<>(); - action.execute(new DocumentsOnlyInput(List.of("abc", "def")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); + action.execute(new ChatCompletionInput(List.of("abc", "def")), InferenceAction.Request.DEFAULT_TIMEOUT, listener); var thrownException = expectThrows(ElasticsearchStatusException.class, () -> listener.actionGet(TIMEOUT)); diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/amazonbedrock/AmazonBedrockMockRequestSender.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/amazonbedrock/AmazonBedrockMockRequestSender.java index e68beaf4c1eb..929aefeeef6b 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/amazonbedrock/AmazonBedrockMockRequestSender.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/amazonbedrock/AmazonBedrockMockRequestSender.java @@ -12,6 +12,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.core.TimeValue; import org.elasticsearch.inference.InferenceServiceResults; +import org.elasticsearch.xpack.inference.external.http.sender.ChatCompletionInput; import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.InferenceInputs; import org.elasticsearch.xpack.inference.external.http.sender.RequestManager; @@ -67,8 +68,15 @@ public void send( ActionListener listener ) { sendCounter++; - var docsInput = (DocumentsOnlyInput) inferenceInputs; - inputs.add(docsInput.getInputs()); + if (inferenceInputs instanceof DocumentsOnlyInput docsInput) { + inputs.add(docsInput.getInputs()); + } else if (inferenceInputs instanceof ChatCompletionInput chatCompletionInput) { + inputs.add(chatCompletionInput.getInputs()); + } else { + throw new IllegalArgumentException( + "Invalid inference inputs received in mock sender: " + inferenceInputs.getClass().getSimpleName() + ); + } if (results.isEmpty()) { listener.onFailure(new ElasticsearchException("No results found")); diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/amazonbedrock/AmazonBedrockRequestSenderTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/amazonbedrock/AmazonBedrockRequestSenderTests.java index 7fa8a09d5bf1..a8f37aedcece 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/amazonbedrock/AmazonBedrockRequestSenderTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/amazonbedrock/AmazonBedrockRequestSenderTests.java @@ -15,6 +15,7 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.inference.external.http.sender.AmazonBedrockChatCompletionRequestManager; import org.elasticsearch.xpack.inference.external.http.sender.AmazonBedrockEmbeddingsRequestManager; +import org.elasticsearch.xpack.inference.external.http.sender.ChatCompletionInput; import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; import org.elasticsearch.xpack.inference.external.http.sender.Sender; import org.elasticsearch.xpack.inference.logging.ThrottlerManager; @@ -107,7 +108,7 @@ public void testCreateSender_SendsCompletionRequestAndReceivesResponse() throws PlainActionFuture listener = new PlainActionFuture<>(); var requestManager = new AmazonBedrockChatCompletionRequestManager(model, threadPool, new TimeValue(30, TimeUnit.SECONDS)); - sender.send(requestManager, new DocumentsOnlyInput(List.of("abc")), null, listener); + sender.send(requestManager, new ChatCompletionInput(List.of("abc")), null, listener); var result = listener.actionGet(TIMEOUT); assertThat(result.asMap(), is(buildExpectationCompletion(List.of("test response text")))); diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/http/sender/InferenceInputsTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/http/sender/InferenceInputsTests.java new file mode 100644 index 000000000000..f0da67a98237 --- /dev/null +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/http/sender/InferenceInputsTests.java @@ -0,0 +1,40 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference.external.http.sender; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.inference.UnifiedCompletionRequest; +import org.elasticsearch.test.ESTestCase; +import org.hamcrest.Matchers; + +import java.util.List; + +public class InferenceInputsTests extends ESTestCase { + public void testCastToSucceeds() { + InferenceInputs inputs = new DocumentsOnlyInput(List.of(), false); + assertThat(inputs.castTo(DocumentsOnlyInput.class), Matchers.instanceOf(DocumentsOnlyInput.class)); + + var emptyRequest = new UnifiedCompletionRequest(List.of(), null, null, null, null, null, null, null); + assertThat(new UnifiedChatInput(emptyRequest, false).castTo(UnifiedChatInput.class), Matchers.instanceOf(UnifiedChatInput.class)); + assertThat( + new QueryAndDocsInputs("hello", List.of(), false).castTo(QueryAndDocsInputs.class), + Matchers.instanceOf(QueryAndDocsInputs.class) + ); + } + + public void testCastToFails() { + InferenceInputs inputs = new DocumentsOnlyInput(List.of(), false); + var exception = expectThrows(IllegalArgumentException.class, () -> inputs.castTo(QueryAndDocsInputs.class)); + assertThat( + exception.getMessage(), + Matchers.containsString( + Strings.format("Unable to convert inference inputs type: [%s] to [%s]", DocumentsOnlyInput.class, QueryAndDocsInputs.class) + ) + ); + } +} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/http/sender/UnifiedChatInputTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/http/sender/UnifiedChatInputTests.java new file mode 100644 index 000000000000..42e1b18168ae --- /dev/null +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/http/sender/UnifiedChatInputTests.java @@ -0,0 +1,46 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference.external.http.sender; + +import org.elasticsearch.inference.UnifiedCompletionRequest; +import org.elasticsearch.test.ESTestCase; +import org.hamcrest.Matchers; + +import java.util.List; + +public class UnifiedChatInputTests extends ESTestCase { + + public void testConvertsStringInputToMessages() { + var a = new UnifiedChatInput(List.of("hello", "awesome"), "a role", true); + + assertThat(a.inputSize(), Matchers.is(2)); + assertThat( + a.getRequest(), + Matchers.is( + UnifiedCompletionRequest.of( + List.of( + new UnifiedCompletionRequest.Message( + new UnifiedCompletionRequest.ContentString("hello"), + "a role", + null, + null, + null + ), + new UnifiedCompletionRequest.Message( + new UnifiedCompletionRequest.ContentString("awesome"), + "a role", + null, + null, + null + ) + ) + ) + ) + ); + } +} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedStreamingProcessorTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedStreamingProcessorTests.java new file mode 100644 index 000000000000..0f127998f9c5 --- /dev/null +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/openai/OpenAiUnifiedStreamingProcessorTests.java @@ -0,0 +1,383 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference.external.openai; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xcontent.XContentFactory; +import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xcontent.XContentParserConfiguration; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.core.inference.results.StreamingUnifiedChatCompletionResults; + +import java.io.IOException; +import java.util.List; + +public class OpenAiUnifiedStreamingProcessorTests extends ESTestCase { + + public void testJsonLiteral() { + String json = """ + { + "id": "example_id", + "choices": [ + { + "delta": { + "content": "example_content", + "refusal": null, + "role": "assistant", + "tool_calls": [ + { + "index": 1, + "id": "tool_call_id", + "function": { + "arguments": "example_arguments", + "name": "example_function_name" + }, + "type": "function" + } + ] + }, + "finish_reason": "stop", + "index": 0 + } + ], + "model": "example_model", + "object": "chat.completion.chunk", + "usage": { + "completion_tokens": 50, + "prompt_tokens": 20, + "total_tokens": 70 + } + } + """; + // Parse the JSON + XContentParserConfiguration parserConfig = XContentParserConfiguration.EMPTY.withDeprecationHandler( + LoggingDeprecationHandler.INSTANCE + ); + try (XContentParser parser = XContentFactory.xContent(XContentType.JSON).createParser(parserConfig, json)) { + StreamingUnifiedChatCompletionResults.ChatCompletionChunk chunk = OpenAiUnifiedStreamingProcessor.ChatCompletionChunkParser + .parse(parser); + + // Assertions to verify the parsed object + assertEquals("example_id", chunk.getId()); + assertEquals("example_model", chunk.getModel()); + assertEquals("chat.completion.chunk", chunk.getObject()); + assertNotNull(chunk.getUsage()); + assertEquals(50, chunk.getUsage().completionTokens()); + assertEquals(20, chunk.getUsage().promptTokens()); + assertEquals(70, chunk.getUsage().totalTokens()); + + List choices = chunk.getChoices(); + assertEquals(1, choices.size()); + StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice choice = choices.get(0); + assertEquals("example_content", choice.delta().getContent()); + assertNull(choice.delta().getRefusal()); + assertEquals("assistant", choice.delta().getRole()); + assertEquals("stop", choice.finishReason()); + assertEquals(0, choice.index()); + + List toolCalls = choice.delta().getToolCalls(); + assertEquals(1, toolCalls.size()); + StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta.ToolCall toolCall = toolCalls.get(0); + assertEquals(1, toolCall.getIndex()); + assertEquals("tool_call_id", toolCall.getId()); + assertEquals("example_function_name", toolCall.getFunction().getName()); + assertEquals("example_arguments", toolCall.getFunction().getArguments()); + assertEquals("function", toolCall.getType()); + } catch (IOException e) { + fail(); + } + } + + public void testJsonLiteralCornerCases() { + String json = """ + { + "id": "example_id", + "choices": [ + { + "delta": { + "content": null, + "refusal": null, + "role": "assistant", + "tool_calls": [] + }, + "finish_reason": null, + "index": 0 + }, + { + "delta": { + "content": "example_content", + "refusal": "example_refusal", + "role": "user", + "tool_calls": [ + { + "index": 1, + "function": { + "name": "example_function_name" + }, + "type": "function" + } + ] + }, + "finish_reason": "stop", + "index": 1 + } + ], + "model": "example_model", + "object": "chat.completion.chunk", + "usage": null + } + """; + // Parse the JSON + XContentParserConfiguration parserConfig = XContentParserConfiguration.EMPTY.withDeprecationHandler( + LoggingDeprecationHandler.INSTANCE + ); + try (XContentParser parser = XContentFactory.xContent(XContentType.JSON).createParser(parserConfig, json)) { + StreamingUnifiedChatCompletionResults.ChatCompletionChunk chunk = OpenAiUnifiedStreamingProcessor.ChatCompletionChunkParser + .parse(parser); + + // Assertions to verify the parsed object + assertEquals("example_id", chunk.getId()); + assertEquals("example_model", chunk.getModel()); + assertEquals("chat.completion.chunk", chunk.getObject()); + assertNull(chunk.getUsage()); + + List choices = chunk.getChoices(); + assertEquals(2, choices.size()); + + // First choice assertions + StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice firstChoice = choices.get(0); + assertNull(firstChoice.delta().getContent()); + assertNull(firstChoice.delta().getRefusal()); + assertEquals("assistant", firstChoice.delta().getRole()); + assertTrue(firstChoice.delta().getToolCalls().isEmpty()); + assertNull(firstChoice.finishReason()); + assertEquals(0, firstChoice.index()); + + // Second choice assertions + StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice secondChoice = choices.get(1); + assertEquals("example_content", secondChoice.delta().getContent()); + assertEquals("example_refusal", secondChoice.delta().getRefusal()); + assertEquals("user", secondChoice.delta().getRole()); + assertEquals("stop", secondChoice.finishReason()); + assertEquals(1, secondChoice.index()); + + List toolCalls = secondChoice.delta() + .getToolCalls(); + assertEquals(1, toolCalls.size()); + StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta.ToolCall toolCall = toolCalls.get(0); + assertEquals(1, toolCall.getIndex()); + assertNull(toolCall.getId()); + assertEquals("example_function_name", toolCall.getFunction().getName()); + assertNull(toolCall.getFunction().getArguments()); + assertEquals("function", toolCall.getType()); + } catch (IOException e) { + fail(); + } + } + + public void testOpenAiUnifiedStreamingProcessorParsing() throws IOException { + // Generate random values for the JSON fields + int toolCallIndex = randomIntBetween(0, 10); + String toolCallId = randomAlphaOfLength(5); + String toolCallFunctionName = randomAlphaOfLength(8); + String toolCallFunctionArguments = randomAlphaOfLength(10); + String toolCallType = "function"; + String toolCallJson = createToolCallJson(toolCallIndex, toolCallId, toolCallFunctionName, toolCallFunctionArguments, toolCallType); + + String choiceContent = randomAlphaOfLength(10); + String choiceRole = randomFrom("system", "user", "assistant", "tool"); + String choiceFinishReason = randomFrom("stop", "length", "tool_calls", "content_filter", "function_call", null); + int choiceIndex = randomIntBetween(0, 10); + String choiceJson = createChoiceJson(choiceContent, null, choiceRole, toolCallJson, choiceFinishReason, choiceIndex); + + int usageCompletionTokens = randomIntBetween(1, 100); + int usagePromptTokens = randomIntBetween(1, 100); + int usageTotalTokens = randomIntBetween(1, 200); + String usageJson = createUsageJson(usageCompletionTokens, usagePromptTokens, usageTotalTokens); + + String chatCompletionChunkId = randomAlphaOfLength(10); + String chatCompletionChunkModel = randomAlphaOfLength(5); + String chatCompletionChunkJson = createChatCompletionChunkJson( + chatCompletionChunkId, + choiceJson, + chatCompletionChunkModel, + "chat.completion.chunk", + usageJson + ); + + // Parse the JSON + XContentParserConfiguration parserConfig = XContentParserConfiguration.EMPTY.withDeprecationHandler( + LoggingDeprecationHandler.INSTANCE + ); + try (XContentParser parser = XContentFactory.xContent(XContentType.JSON).createParser(parserConfig, chatCompletionChunkJson)) { + StreamingUnifiedChatCompletionResults.ChatCompletionChunk chunk = OpenAiUnifiedStreamingProcessor.ChatCompletionChunkParser + .parse(parser); + + // Assertions to verify the parsed object + assertEquals(chatCompletionChunkId, chunk.getId()); + assertEquals(chatCompletionChunkModel, chunk.getModel()); + assertEquals("chat.completion.chunk", chunk.getObject()); + assertNotNull(chunk.getUsage()); + assertEquals(usageCompletionTokens, chunk.getUsage().completionTokens()); + assertEquals(usagePromptTokens, chunk.getUsage().promptTokens()); + assertEquals(usageTotalTokens, chunk.getUsage().totalTokens()); + + List choices = chunk.getChoices(); + assertEquals(1, choices.size()); + StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice choice = choices.get(0); + assertEquals(choiceContent, choice.delta().getContent()); + assertNull(choice.delta().getRefusal()); + assertEquals(choiceRole, choice.delta().getRole()); + assertEquals(choiceFinishReason, choice.finishReason()); + assertEquals(choiceIndex, choice.index()); + + List toolCalls = choice.delta().getToolCalls(); + assertEquals(1, toolCalls.size()); + StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice.Delta.ToolCall toolCall = toolCalls.get(0); + assertEquals(toolCallIndex, toolCall.getIndex()); + assertEquals(toolCallId, toolCall.getId()); + assertEquals(toolCallFunctionName, toolCall.getFunction().getName()); + assertEquals(toolCallFunctionArguments, toolCall.getFunction().getArguments()); + assertEquals(toolCallType, toolCall.getType()); + } + } + + public void testOpenAiUnifiedStreamingProcessorParsingWithNullFields() throws IOException { + // JSON with null fields + int choiceIndex = randomIntBetween(0, 10); + String choiceJson = createChoiceJson(null, null, null, "", null, choiceIndex); + + String chatCompletionChunkId = randomAlphaOfLength(10); + String chatCompletionChunkModel = randomAlphaOfLength(5); + String chatCompletionChunkJson = createChatCompletionChunkJson( + chatCompletionChunkId, + choiceJson, + chatCompletionChunkModel, + "chat.completion.chunk", + null + ); + + // Parse the JSON + XContentParserConfiguration parserConfig = XContentParserConfiguration.EMPTY.withDeprecationHandler( + LoggingDeprecationHandler.INSTANCE + ); + try (XContentParser parser = XContentFactory.xContent(XContentType.JSON).createParser(parserConfig, chatCompletionChunkJson)) { + StreamingUnifiedChatCompletionResults.ChatCompletionChunk chunk = OpenAiUnifiedStreamingProcessor.ChatCompletionChunkParser + .parse(parser); + + // Assertions to verify the parsed object + assertEquals(chatCompletionChunkId, chunk.getId()); + assertEquals(chatCompletionChunkModel, chunk.getModel()); + assertEquals("chat.completion.chunk", chunk.getObject()); + assertNull(chunk.getUsage()); + + List choices = chunk.getChoices(); + assertEquals(1, choices.size()); + StreamingUnifiedChatCompletionResults.ChatCompletionChunk.Choice choice = choices.get(0); + assertNull(choice.delta().getContent()); + assertNull(choice.delta().getRefusal()); + assertNull(choice.delta().getRole()); + assertNull(choice.finishReason()); + assertEquals(choiceIndex, choice.index()); + assertTrue(choice.delta().getToolCalls().isEmpty()); + } + } + + private String createToolCallJson(int index, String id, String functionName, String functionArguments, String type) { + return Strings.format(""" + { + "index": %d, + "id": "%s", + "function": { + "name": "%s", + "arguments": "%s" + }, + "type": "%s" + } + """, index, id, functionName, functionArguments, type); + } + + private String createChoiceJson(String content, String refusal, String role, String toolCallsJson, String finishReason, int index) { + if (role == null) { + return Strings.format( + """ + { + "delta": { + "content": %s, + "refusal": %s, + "tool_calls": [%s] + }, + "finish_reason": %s, + "index": %d + } + """, + content != null ? "\"" + content + "\"" : "null", + refusal != null ? "\"" + refusal + "\"" : "null", + toolCallsJson, + finishReason != null ? "\"" + finishReason + "\"" : "null", + index + ); + } else { + return Strings.format( + """ + { + "delta": { + "content": %s, + "refusal": %s, + "role": %s, + "tool_calls": [%s] + }, + "finish_reason": %s, + "index": %d + } + """, + content != null ? "\"" + content + "\"" : "null", + refusal != null ? "\"" + refusal + "\"" : "null", + role != null ? "\"" + role + "\"" : "null", + toolCallsJson, + finishReason != null ? "\"" + finishReason + "\"" : "null", + index + ); + } + } + + private String createChatCompletionChunkJson(String id, String choicesJson, String model, String object, String usageJson) { + if (usageJson != null) { + return Strings.format(""" + { + "id": "%s", + "choices": [%s], + "model": "%s", + "object": "%s", + "usage": %s + } + """, id, choicesJson, model, object, usageJson); + } else { + return Strings.format(""" + { + "id": "%s", + "choices": [%s], + "model": "%s", + "object": "%s" + } + """, id, choicesJson, model, object); + } + } + + private String createUsageJson(int completionTokens, int promptTokens, int totalTokens) { + return Strings.format(""" + { + "completion_tokens": %d, + "prompt_tokens": %d, + "total_tokens": %d + } + """, completionTokens, promptTokens, totalTokens); + } +} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/googleaistudio/completion/GoogleAiStudioCompletionRequestTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/googleaistudio/completion/GoogleAiStudioCompletionRequestTests.java index 7ffa8940ad6b..065dfee577a8 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/googleaistudio/completion/GoogleAiStudioCompletionRequestTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/googleaistudio/completion/GoogleAiStudioCompletionRequestTests.java @@ -10,7 +10,7 @@ import org.apache.http.client.methods.HttpPost; import org.elasticsearch.common.Strings; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xpack.inference.external.http.sender.DocumentsOnlyInput; +import org.elasticsearch.xpack.inference.external.http.sender.ChatCompletionInput; import org.elasticsearch.xpack.inference.external.request.googleaistudio.GoogleAiStudioCompletionRequest; import org.elasticsearch.xpack.inference.services.googleaistudio.completion.GoogleAiStudioCompletionModelTests; @@ -72,7 +72,7 @@ public void testTruncationInfo_ReturnsNull() { assertNull(request.getTruncationInfo()); } - private static DocumentsOnlyInput listOf(String... input) { - return new DocumentsOnlyInput(List.of(input)); + private static ChatCompletionInput listOf(String... input) { + return new ChatCompletionInput(List.of(input)); } } diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiChatCompletionRequestEntityTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiChatCompletionRequestEntityTests.java deleted file mode 100644 index 9d5492f9e951..000000000000 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiChatCompletionRequestEntityTests.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License - * 2.0; you may not use this file except in compliance with the Elastic License - * 2.0. - */ - -package org.elasticsearch.xpack.inference.external.request.openai; - -import org.elasticsearch.common.Strings; -import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xcontent.XContentBuilder; -import org.elasticsearch.xcontent.XContentFactory; -import org.elasticsearch.xcontent.XContentType; - -import java.io.IOException; -import java.util.List; - -import static org.hamcrest.CoreMatchers.is; - -public class OpenAiChatCompletionRequestEntityTests extends ESTestCase { - - public void testXContent_WritesUserWhenDefined() throws IOException { - var entity = new OpenAiChatCompletionRequestEntity(List.of("abc"), "model", "user", false); - - XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON); - entity.toXContent(builder, null); - String xContentResult = Strings.toString(builder); - - assertThat(xContentResult, is(""" - {"messages":[{"role":"user","content":"abc"}],"model":"model","n":1,"user":"user"}""")); - - } - - public void testXContent_DoesNotWriteUserWhenItIsNull() throws IOException { - var entity = new OpenAiChatCompletionRequestEntity(List.of("abc"), "model", null, false); - - XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON); - entity.toXContent(builder, null); - String xContentResult = Strings.toString(builder); - - assertThat(xContentResult, is(""" - {"messages":[{"role":"user","content":"abc"}],"model":"model","n":1}""")); - } - - public void testXContent_ThrowsIfModelIsNull() { - assertThrows(NullPointerException.class, () -> new OpenAiChatCompletionRequestEntity(List.of("abc"), null, "user", false)); - } - - public void testXContent_ThrowsIfMessagesAreNull() { - assertThrows(NullPointerException.class, () -> new OpenAiChatCompletionRequestEntity(null, "model", "user", false)); - } -} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiUnifiedChatCompletionRequestEntityTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiUnifiedChatCompletionRequestEntityTests.java new file mode 100644 index 000000000000..f945c154ea23 --- /dev/null +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiUnifiedChatCompletionRequestEntityTests.java @@ -0,0 +1,856 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference.external.request.openai; + +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.Strings; +import org.elasticsearch.inference.UnifiedCompletionRequest; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xcontent.ToXContent; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.json.JsonXContent; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; +import org.elasticsearch.xpack.inference.services.openai.completion.OpenAiChatCompletionModel; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.Locale; +import java.util.Map; +import java.util.Random; + +import static org.elasticsearch.xpack.inference.services.openai.completion.OpenAiChatCompletionModelTests.createChatCompletionModel; +import static org.hamcrest.Matchers.equalTo; + +public class OpenAiUnifiedChatCompletionRequestEntityTests extends ESTestCase { + + // 1. Basic Serialization + // Test with minimal required fields to ensure basic serialization works. + public void testBasicSerialization() throws IOException { + UnifiedCompletionRequest.Message message = new UnifiedCompletionRequest.Message( + new UnifiedCompletionRequest.ContentString("Hello, world!"), + OpenAiUnifiedChatCompletionRequestEntity.USER_FIELD, + null, + null, + null + ); + var messageList = new ArrayList(); + messageList.add(message); + UnifiedCompletionRequest unifiedRequest = new UnifiedCompletionRequest(messageList, null, null, null, null, null, null, null); + + UnifiedChatInput unifiedChatInput = new UnifiedChatInput(unifiedRequest, true); + OpenAiChatCompletionModel model = createChatCompletionModel("test-url", "organizationId", "api-key", "test-endpoint", null); + + OpenAiUnifiedChatCompletionRequestEntity entity = new OpenAiUnifiedChatCompletionRequestEntity(unifiedChatInput, model); + + XContentBuilder builder = JsonXContent.contentBuilder(); + entity.toXContent(builder, ToXContent.EMPTY_PARAMS); + + String jsonString = Strings.toString(builder); + String expectedJson = """ + { + "messages": [ + { + "content": "Hello, world!", + "role": "user" + } + ], + "model": "test-endpoint", + "n": 1, + "stream": true, + "stream_options": { + "include_usage": true + } + } + """; + assertJsonEquals(jsonString, expectedJson); + } + + // 2. Serialization with All Fields + // Test with all possible fields populated to ensure complete serialization. + public void testSerializationWithAllFields() throws IOException { + // Create a message with all fields populated + UnifiedCompletionRequest.Message message = new UnifiedCompletionRequest.Message( + new UnifiedCompletionRequest.ContentString("Hello, world!"), + OpenAiUnifiedChatCompletionRequestEntity.USER_FIELD, + "name", + "tool_call_id", + Collections.singletonList( + new UnifiedCompletionRequest.ToolCall( + "id", + new UnifiedCompletionRequest.ToolCall.FunctionField("arguments", "function_name"), + "type" + ) + ) + ); + + // Create a tool with all fields populated + UnifiedCompletionRequest.Tool tool = new UnifiedCompletionRequest.Tool( + "type", + new UnifiedCompletionRequest.Tool.FunctionField( + "Fetches the weather in the given location", + "get_weather", + createParameters(), + true + ) + ); + var messageList = new ArrayList(); + messageList.add(message); + // Create the unified request with all fields populated + UnifiedCompletionRequest unifiedRequest = new UnifiedCompletionRequest( + messageList, + "model", + 100L, // maxCompletionTokens + Collections.singletonList("stop"), + 0.9f, // temperature + new UnifiedCompletionRequest.ToolChoiceString("tool_choice"), + Collections.singletonList(tool), + 0.8f // topP + ); + + // Create the unified chat input + UnifiedChatInput unifiedChatInput = new UnifiedChatInput(unifiedRequest, true); + + OpenAiChatCompletionModel model = createChatCompletionModel("test-endpoint", "organizationId", "api-key", "model-name", null); + + // Create the entity + OpenAiUnifiedChatCompletionRequestEntity entity = new OpenAiUnifiedChatCompletionRequestEntity(unifiedChatInput, model); + + // Serialize to XContent + XContentBuilder builder = JsonXContent.contentBuilder(); + entity.toXContent(builder, ToXContent.EMPTY_PARAMS); + + // Convert to string and verify + String jsonString = Strings.toString(builder); + String expectedJson = """ + { + "messages": [ + { + "content": "Hello, world!", + "role": "user", + "name": "name", + "tool_call_id": "tool_call_id", + "tool_calls": [ + { + "id": "id", + "function": { + "arguments": "arguments", + "name": "function_name" + }, + "type": "type" + } + ] + } + ], + "model": "model-name", + "max_completion_tokens": 100, + "n": 1, + "stop": ["stop"], + "temperature": 0.9, + "tool_choice": "tool_choice", + "tools": [ + { + "type": "type", + "function": { + "description": "Fetches the weather in the given location", + "name": "get_weather", + "parameters": { + "type": "object", + "properties": { + "location": { + "description": "The location to get the weather for", + "type": "string" + }, + "unit": { + "description": "The unit to return the temperature in", + "type": "string", + "enum": ["F", "C"] + } + }, + "additionalProperties": false, + "required": ["location", "unit"] + }, + "strict": true + } + } + ], + "top_p": 0.8, + "stream": true, + "stream_options": { + "include_usage": true + } + } + """; + assertJsonEquals(jsonString, expectedJson); + + } + + // 3. Serialization with Null Optional Fields + // Test with optional fields set to null to ensure they are correctly omitted from the output. + public void testSerializationWithNullOptionalFields() throws IOException { + // Create a message with minimal required fields + UnifiedCompletionRequest.Message message = new UnifiedCompletionRequest.Message( + new UnifiedCompletionRequest.ContentString("Hello, world!"), + OpenAiUnifiedChatCompletionRequestEntity.USER_FIELD, + null, + null, + null + ); + var messageList = new ArrayList(); + messageList.add(message); + + // Create the unified request with optional fields set to null + UnifiedCompletionRequest unifiedRequest = new UnifiedCompletionRequest( + messageList, + null, // model + null, // maxCompletionTokens + null, // stop + null, // temperature + null, // toolChoice + null, // tools + null // topP + ); + + // Create the unified chat input + UnifiedChatInput unifiedChatInput = new UnifiedChatInput(unifiedRequest, true); + + OpenAiChatCompletionModel model = createChatCompletionModel("test-endpoint", "organizationId", "api-key", "model-name", null); + + // Create the entity + OpenAiUnifiedChatCompletionRequestEntity entity = new OpenAiUnifiedChatCompletionRequestEntity(unifiedChatInput, model); + + // Serialize to XContent + XContentBuilder builder = JsonXContent.contentBuilder(); + entity.toXContent(builder, ToXContent.EMPTY_PARAMS); + + // Convert to string and verify + String jsonString = Strings.toString(builder); + String expectedJson = """ + { + "messages": [ + { + "content": "Hello, world!", + "role": "user" + } + ], + "model": "model-name", + "n": 1, + "stream": true, + "stream_options": { + "include_usage": true + } + } + """; + assertJsonEquals(jsonString, expectedJson); + } + + // 4. Serialization with Empty Lists + // Test with fields that are lists set to empty lists to ensure they are correctly serialized. + public void testSerializationWithEmptyLists() throws IOException { + // Create a message with minimal required fields + UnifiedCompletionRequest.Message message = new UnifiedCompletionRequest.Message( + new UnifiedCompletionRequest.ContentString("Hello, world!"), + OpenAiUnifiedChatCompletionRequestEntity.USER_FIELD, + null, + null, + Collections.emptyList() // empty toolCalls list + ); + var messageList = new ArrayList(); + messageList.add(message); + // Create the unified request with empty lists + UnifiedCompletionRequest unifiedRequest = new UnifiedCompletionRequest( + messageList, + null, // model + null, // maxCompletionTokens + Collections.emptyList(), // empty stop list + null, // temperature + null, // toolChoice + Collections.emptyList(), // empty tools list + null // topP + ); + + // Create the unified chat input + UnifiedChatInput unifiedChatInput = new UnifiedChatInput(unifiedRequest, true); + + OpenAiChatCompletionModel model = createChatCompletionModel("test-endpoint", "organizationId", "api-key", "model-name", null); + + // Create the entity + OpenAiUnifiedChatCompletionRequestEntity entity = new OpenAiUnifiedChatCompletionRequestEntity(unifiedChatInput, model); + + // Serialize to XContent + XContentBuilder builder = JsonXContent.contentBuilder(); + entity.toXContent(builder, ToXContent.EMPTY_PARAMS); + + // Convert to string and verify + String jsonString = Strings.toString(builder); + String expectedJson = """ + { + "messages": [ + { + "content": "Hello, world!", + "role": "user", + "tool_calls": [] + } + ], + "model": "model-name", + "n": 1, + "stream": true, + "stream_options": { + "include_usage": true + } + } + """; + assertJsonEquals(jsonString, expectedJson); + } + + // 5. Serialization with Nested Objects + // Test with nested objects (e.g., toolCalls, toolChoice, tool) to ensure they are correctly serialized. + public void testSerializationWithNestedObjects() throws IOException { + Random random = Randomness.get(); + + // Generate random values + String randomContent = "Hello, world! " + random.nextInt(1000); + String randomName = "name" + random.nextInt(1000); + String randomToolCallId = "tool_call_id" + random.nextInt(1000); + String randomArguments = "arguments" + random.nextInt(1000); + String randomFunctionName = "function_name" + random.nextInt(1000); + String randomType = "type" + random.nextInt(1000); + String randomModel = "model" + random.nextInt(1000); + String randomStop = "stop" + random.nextInt(1000); + float randomTemperature = (float) ((float) Math.round(0.5d + (double) random.nextFloat() * 0.5d * 100000d) / 100000d); + float randomTopP = (float) ((float) Math.round(0.5d + (double) random.nextFloat() * 0.5d * 100000d) / 100000d); + + // Create a message with nested toolCalls + UnifiedCompletionRequest.Message message = new UnifiedCompletionRequest.Message( + new UnifiedCompletionRequest.ContentString(randomContent), + OpenAiUnifiedChatCompletionRequestEntity.USER_FIELD, + randomName, + randomToolCallId, + Collections.singletonList( + new UnifiedCompletionRequest.ToolCall( + "id", + new UnifiedCompletionRequest.ToolCall.FunctionField(randomArguments, randomFunctionName), + randomType + ) + ) + ); + + // Create a tool with nested function fields + UnifiedCompletionRequest.Tool tool = new UnifiedCompletionRequest.Tool( + randomType, + new UnifiedCompletionRequest.Tool.FunctionField( + "Fetches the weather in the given location", + "get_weather", + createParameters(), + true + ) + ); + var messageList = new ArrayList(); + messageList.add(message); + // Create the unified request with nested objects + UnifiedCompletionRequest unifiedRequest = new UnifiedCompletionRequest( + messageList, + randomModel, + 100L, // maxCompletionTokens + Collections.singletonList(randomStop), + randomTemperature, // temperature + new UnifiedCompletionRequest.ToolChoiceObject( + randomType, + new UnifiedCompletionRequest.ToolChoiceObject.FunctionField(randomFunctionName) + ), + Collections.singletonList(tool), + randomTopP // topP + ); + + // Create the unified chat input + UnifiedChatInput unifiedChatInput = new UnifiedChatInput(unifiedRequest, true); + + OpenAiChatCompletionModel model = createChatCompletionModel("test-endpoint", "organizationId", "api-key", randomModel, null); + + // Create the entity + OpenAiUnifiedChatCompletionRequestEntity entity = new OpenAiUnifiedChatCompletionRequestEntity(unifiedChatInput, model); + + // Serialize to XContent + XContentBuilder builder = JsonXContent.contentBuilder(); + entity.toXContent(builder, ToXContent.EMPTY_PARAMS); + + // Convert to string and verify + String jsonString = Strings.toString(builder); + // Expected JSON should be dynamically generated based on random values + String expectedJson = String.format( + Locale.US, + """ + { + "messages": [ + { + "content": "%s", + "role": "user", + "name": "%s", + "tool_call_id": "%s", + "tool_calls": [ + { + "id": "id", + "function": { + "arguments": "%s", + "name": "%s" + }, + "type": "%s" + } + ] + } + ], + "model": "%s", + "max_completion_tokens": 100, + "n": 1, + "stop": ["%s"], + "temperature": %.5f, + "tool_choice": { + "type": "%s", + "function": { + "name": "%s" + } + }, + "tools": [ + { + "type": "%s", + "function": { + "description": "Fetches the weather in the given location", + "name": "get_weather", + "parameters": { + "type": "object", + "properties": { + "unit": { + "description": "The unit to return the temperature in", + "type": "string", + "enum": ["F", "C"] + }, + "location": { + "description": "The location to get the weather for", + "type": "string" + } + }, + "additionalProperties": false, + "required": ["location", "unit"] + }, + "strict": true + } + } + ], + "top_p": %.5f, + "stream": true, + "stream_options": { + "include_usage": true + } + } + """, + randomContent, + randomName, + randomToolCallId, + randomArguments, + randomFunctionName, + randomType, + randomModel, + randomStop, + randomTemperature, + randomType, + randomFunctionName, + randomType, + randomTopP + ); + assertJsonEquals(jsonString, expectedJson); + } + + // 6. Serialization with Different Content Types + // Test with different content types in messages (e.g., ContentString, ContentObjects) to ensure they are correctly serialized. + public void testSerializationWithDifferentContentTypes() throws IOException { + Random random = Randomness.get(); + + // Generate random values for ContentString + String randomContentString = "Hello, world! " + random.nextInt(1000); + + // Generate random values for ContentObjects + String randomText = "Random text " + random.nextInt(1000); + String randomType = "type" + random.nextInt(1000); + UnifiedCompletionRequest.ContentObject contentObject = new UnifiedCompletionRequest.ContentObject(randomText, randomType); + + var contentObjectsList = new ArrayList(); + contentObjectsList.add(contentObject); + UnifiedCompletionRequest.ContentObjects contentObjects = new UnifiedCompletionRequest.ContentObjects(contentObjectsList); + + // Create messages with different content types + UnifiedCompletionRequest.Message messageWithString = new UnifiedCompletionRequest.Message( + new UnifiedCompletionRequest.ContentString(randomContentString), + OpenAiUnifiedChatCompletionRequestEntity.USER_FIELD, + null, + null, + null + ); + + UnifiedCompletionRequest.Message messageWithObjects = new UnifiedCompletionRequest.Message( + contentObjects, + OpenAiUnifiedChatCompletionRequestEntity.USER_FIELD, + null, + null, + null + ); + var messageList = new ArrayList(); + messageList.add(messageWithString); + messageList.add(messageWithObjects); + + // Create the unified request with both types of messages + UnifiedCompletionRequest unifiedRequest = UnifiedCompletionRequest.of(messageList); + + // Create the unified chat input + UnifiedChatInput unifiedChatInput = new UnifiedChatInput(unifiedRequest, true); + + OpenAiChatCompletionModel model = createChatCompletionModel("test-endpoint", "organizationId", "api-key", "model-name", null); + + // Create the entity + OpenAiUnifiedChatCompletionRequestEntity entity = new OpenAiUnifiedChatCompletionRequestEntity(unifiedChatInput, model); + + // Serialize to XContent + XContentBuilder builder = JsonXContent.contentBuilder(); + entity.toXContent(builder, ToXContent.EMPTY_PARAMS); + + // Convert to string and verify + String jsonString = Strings.toString(builder); + String expectedJson = String.format(Locale.US, """ + { + "messages": [ + { + "content": "%s", + "role": "user" + }, + { + "content": [ + { + "text": "%s", + "type": "%s" + } + ], + "role": "user" + } + ], + "model": "model-name", + "n": 1, + "stream": true, + "stream_options": { + "include_usage": true + } + } + """, randomContentString, randomText, randomType); + assertJsonEquals(jsonString, expectedJson); + } + + // 7. Serialization with Special Characters + // Test with special characters in string fields to ensure they are correctly escaped and serialized. + public void testSerializationWithSpecialCharacters() throws IOException { + // Create a message with special characters + UnifiedCompletionRequest.Message message = new UnifiedCompletionRequest.Message( + new UnifiedCompletionRequest.ContentString("Hello, world! \n \"Special\" characters: \t \\ /"), + OpenAiUnifiedChatCompletionRequestEntity.USER_FIELD, + "name\nwith\nnewlines", + "tool_call_id\twith\ttabs", + Collections.singletonList( + new UnifiedCompletionRequest.ToolCall( + "id\\with\\backslashes", + new UnifiedCompletionRequest.ToolCall.FunctionField("arguments\"with\"quotes", "function_name/with/slashes"), + "type" + ) + ) + ); + var messageList = new ArrayList(); + messageList.add(message); + // Create the unified request + UnifiedCompletionRequest unifiedRequest = new UnifiedCompletionRequest( + messageList, + null, // model + null, // maxCompletionTokens + null, // stop + null, // temperature + null, // toolChoice + null, // tools + null // topP + ); + + // Create the unified chat input + UnifiedChatInput unifiedChatInput = new UnifiedChatInput(unifiedRequest, true); + + OpenAiChatCompletionModel model = createChatCompletionModel("test-endpoint", "organizationId", "api-key", "model-name", null); + + // Create the entity + OpenAiUnifiedChatCompletionRequestEntity entity = new OpenAiUnifiedChatCompletionRequestEntity(unifiedChatInput, model); + + // Serialize to XContent + XContentBuilder builder = JsonXContent.contentBuilder(); + entity.toXContent(builder, ToXContent.EMPTY_PARAMS); + + // Convert to string and verify + String jsonString = Strings.toString(builder); + String expectedJson = """ + { + "messages": [ + { + "content": "Hello, world! \\n \\"Special\\" characters: \\t \\\\ /", + "role": "user", + "name": "name\\nwith\\nnewlines", + "tool_call_id": "tool_call_id\\twith\\ttabs", + "tool_calls": [ + { + "id": "id\\\\with\\\\backslashes", + "function": { + "arguments": "arguments\\"with\\"quotes", + "name": "function_name/with/slashes" + }, + "type": "type" + } + ] + } + ], + "model": "model-name", + "n": 1, + "stream": true, + "stream_options": { + "include_usage": true + } + } + """; + assertJsonEquals(jsonString, expectedJson); + } + + // 8. Serialization with Boolean Fields + // Test with boolean fields (stream) set to both true and false to ensure they are correctly serialized. + public void testSerializationWithBooleanFields() throws IOException { + // Create a message with minimal required fields + UnifiedCompletionRequest.Message message = new UnifiedCompletionRequest.Message( + new UnifiedCompletionRequest.ContentString("Hello, world!"), + OpenAiUnifiedChatCompletionRequestEntity.USER_FIELD, + null, + null, + null + ); + var messageList = new ArrayList(); + messageList.add(message); + // Create the unified request + UnifiedCompletionRequest unifiedRequest = new UnifiedCompletionRequest( + messageList, + null, // model + null, // maxCompletionTokens + null, // stop + null, // temperature + null, // toolChoice + null, // tools + null // topP + ); + + OpenAiChatCompletionModel model = createChatCompletionModel("test-endpoint", "organizationId", "api-key", "model-name", null); + + // Test with stream set to true + UnifiedChatInput unifiedChatInputTrue = new UnifiedChatInput(unifiedRequest, true); + OpenAiUnifiedChatCompletionRequestEntity entityTrue = new OpenAiUnifiedChatCompletionRequestEntity(unifiedChatInputTrue, model); + + XContentBuilder builderTrue = JsonXContent.contentBuilder(); + entityTrue.toXContent(builderTrue, ToXContent.EMPTY_PARAMS); + + String jsonStringTrue = Strings.toString(builderTrue); + String expectedJsonTrue = """ + { + "messages": [ + { + "content": "Hello, world!", + "role": "user" + } + ], + "model": "model-name", + "n": 1, + "stream": true, + "stream_options": { + "include_usage": true + } + } + """; + assertJsonEquals(expectedJsonTrue, jsonStringTrue); + + // Test with stream set to false + UnifiedChatInput unifiedChatInputFalse = new UnifiedChatInput(unifiedRequest, false); + OpenAiUnifiedChatCompletionRequestEntity entityFalse = new OpenAiUnifiedChatCompletionRequestEntity(unifiedChatInputFalse, model); + + XContentBuilder builderFalse = JsonXContent.contentBuilder(); + entityFalse.toXContent(builderFalse, ToXContent.EMPTY_PARAMS); + + String jsonStringFalse = Strings.toString(builderFalse); + String expectedJsonFalse = """ + { + "messages": [ + { + "content": "Hello, world!", + "role": "user" + } + ], + "model": "model-name", + "n": 1, + "stream": false + } + """; + assertJsonEquals(expectedJsonFalse, jsonStringFalse); + } + + // 9. Serialization with Missing Required Fields + // Test with missing required fields to ensure appropriate exceptions are thrown. + public void testSerializationWithMissingRequiredFields() { + // Create a message with missing content (required field) + UnifiedCompletionRequest.Message message = new UnifiedCompletionRequest.Message( + null, // missing content + OpenAiUnifiedChatCompletionRequestEntity.USER_FIELD, + null, + null, + null + ); + var messageList = new ArrayList(); + messageList.add(message); + // Create the unified request + UnifiedCompletionRequest unifiedRequest = new UnifiedCompletionRequest( + messageList, + null, // model + null, // maxCompletionTokens + null, // stop + null, // temperature + null, // toolChoice + null, // tools + null // topP + ); + + // Create the unified chat input + UnifiedChatInput unifiedChatInput = new UnifiedChatInput(unifiedRequest, true); + + OpenAiChatCompletionModel model = createChatCompletionModel("test-endpoint", "organizationId", "api-key", "model-name", null); + + // Create the entity + OpenAiUnifiedChatCompletionRequestEntity entity = new OpenAiUnifiedChatCompletionRequestEntity(unifiedChatInput, model); + + // Attempt to serialize to XContent and expect an exception + try { + XContentBuilder builder = JsonXContent.contentBuilder(); + entity.toXContent(builder, ToXContent.EMPTY_PARAMS); + fail("Expected an exception due to missing required fields"); + } catch (NullPointerException | IOException e) { + // Expected exception + } + } + + // 10. Serialization with Mixed Valid and Invalid Data + // Test with a mix of valid and invalid data to ensure the serializer handles it gracefully. + public void testSerializationWithMixedValidAndInvalidData() throws IOException { + // Create a valid message + UnifiedCompletionRequest.Message validMessage = new UnifiedCompletionRequest.Message( + new UnifiedCompletionRequest.ContentString("Valid content"), + OpenAiUnifiedChatCompletionRequestEntity.USER_FIELD, + "validName", + "validToolCallId", + Collections.singletonList( + new UnifiedCompletionRequest.ToolCall( + "validId", + new UnifiedCompletionRequest.ToolCall.FunctionField("validArguments", "validFunctionName"), + "validType" + ) + ) + ); + + // Create an invalid message with null content + UnifiedCompletionRequest.Message invalidMessage = new UnifiedCompletionRequest.Message( + null, // invalid content + OpenAiUnifiedChatCompletionRequestEntity.USER_FIELD, + "invalidName", + "invalidToolCallId", + Collections.singletonList( + new UnifiedCompletionRequest.ToolCall( + "invalidId", + new UnifiedCompletionRequest.ToolCall.FunctionField("invalidArguments", "invalidFunctionName"), + "invalidType" + ) + ) + ); + var messageList = new ArrayList(); + messageList.add(validMessage); + messageList.add(invalidMessage); + // Create the unified request with both valid and invalid messages + UnifiedCompletionRequest unifiedRequest = new UnifiedCompletionRequest( + messageList, + "model-name", + 100L, // maxCompletionTokens + Collections.singletonList("stop"), + 0.9f, // temperature + new UnifiedCompletionRequest.ToolChoiceString("tool_choice"), + Collections.singletonList( + new UnifiedCompletionRequest.Tool( + "type", + new UnifiedCompletionRequest.Tool.FunctionField( + "Fetches the weather in the given location", + "get_weather", + createParameters(), + true + ) + ) + ), + 0.8f // topP + ); + + // Create the unified chat input + UnifiedChatInput unifiedChatInput = new UnifiedChatInput(unifiedRequest, true); + + OpenAiChatCompletionModel model = createChatCompletionModel("test-endpoint", "organizationId", "api-key", "model-name", null); + + // Create the entity + OpenAiUnifiedChatCompletionRequestEntity entity = new OpenAiUnifiedChatCompletionRequestEntity(unifiedChatInput, model); + + // Serialize to XContent and verify + try { + XContentBuilder builder = JsonXContent.contentBuilder(); + entity.toXContent(builder, ToXContent.EMPTY_PARAMS); + fail("Expected an exception due to invalid data"); + } catch (NullPointerException | IOException e) { + // Expected exception + } + } + + public static Map createParameters() { + Map parameters = new LinkedHashMap<>(); + parameters.put("type", "object"); + + Map properties = new HashMap<>(); + + Map location = new HashMap<>(); + location.put("type", "string"); + location.put("description", "The location to get the weather for"); + properties.put("location", location); + + Map unit = new HashMap<>(); + unit.put("type", "string"); + unit.put("description", "The unit to return the temperature in"); + unit.put("enum", new String[] { "F", "C" }); + properties.put("unit", unit); + + parameters.put("properties", properties); + parameters.put("additionalProperties", false); + parameters.put("required", new String[] { "location", "unit" }); + + return parameters; + } + + private void assertJsonEquals(String actual, String expected) throws IOException { + try ( + var actualParser = createParser(JsonXContent.jsonXContent, actual); + var expectedParser = createParser(JsonXContent.jsonXContent, expected) + ) { + assertThat(actualParser.mapOrdered(), equalTo(expectedParser.mapOrdered())); + } + } + +} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiChatCompletionRequestTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiUnifiedChatCompletionRequestTests.java similarity index 75% rename from x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiChatCompletionRequestTests.java rename to x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiUnifiedChatCompletionRequestTests.java index b6ebfd02941f..2be12c9b12e0 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiChatCompletionRequestTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/external/request/openai/OpenAiUnifiedChatCompletionRequestTests.java @@ -12,6 +12,7 @@ import org.elasticsearch.core.Nullable; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; import org.elasticsearch.xpack.inference.services.openai.completion.OpenAiChatCompletionModelTests; import java.io.IOException; @@ -20,16 +21,16 @@ import java.util.Map; import static org.elasticsearch.xpack.inference.external.http.Utils.entityAsMap; -import static org.elasticsearch.xpack.inference.external.request.openai.OpenAiChatCompletionRequest.buildDefaultUri; +import static org.elasticsearch.xpack.inference.external.request.openai.OpenAiUnifiedChatCompletionRequest.buildDefaultUri; import static org.elasticsearch.xpack.inference.external.request.openai.OpenAiUtils.ORGANIZATION_HEADER; import static org.hamcrest.Matchers.aMapWithSize; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; -public class OpenAiChatCompletionRequestTests extends ESTestCase { +public class OpenAiUnifiedChatCompletionRequestTests extends ESTestCase { public void testCreateRequest_WithUrlOrganizationUserDefined() throws IOException { - var request = createRequest("www.google.com", "org", "secret", "abc", "model", "user"); + var request = createRequest("www.google.com", "org", "secret", "abc", "model", "user", true); var httpRequest = request.createHttpRequest(); assertThat(httpRequest.httpRequestBase(), instanceOf(HttpPost.class)); @@ -41,15 +42,27 @@ public void testCreateRequest_WithUrlOrganizationUserDefined() throws IOExceptio assertThat(httpPost.getLastHeader(ORGANIZATION_HEADER).getValue(), is("org")); var requestMap = entityAsMap(httpPost.getEntity().getContent()); - assertThat(requestMap, aMapWithSize(4)); + assertRequestMapWithUser(requestMap, "user"); + } + + private void assertRequestMapWithoutUser(Map requestMap) { + assertRequestMapWithUser(requestMap, null); + } + + private void assertRequestMapWithUser(Map requestMap, @Nullable String user) { + assertThat(requestMap, aMapWithSize(user != null ? 6 : 5)); assertThat(requestMap.get("messages"), is(List.of(Map.of("role", "user", "content", "abc")))); assertThat(requestMap.get("model"), is("model")); - assertThat(requestMap.get("user"), is("user")); + if (user != null) { + assertThat(requestMap.get("user"), is(user)); + } assertThat(requestMap.get("n"), is(1)); + assertTrue((Boolean) requestMap.get("stream")); + assertThat(requestMap.get("stream_options"), is(Map.of("include_usage", true))); } public void testCreateRequest_WithDefaultUrl() throws URISyntaxException, IOException { - var request = createRequest(null, "org", "secret", "abc", "model", "user"); + var request = createRequest(null, "org", "secret", "abc", "model", "user", true); var httpRequest = request.createHttpRequest(); assertThat(httpRequest.httpRequestBase(), instanceOf(HttpPost.class)); @@ -61,33 +74,27 @@ public void testCreateRequest_WithDefaultUrl() throws URISyntaxException, IOExce assertThat(httpPost.getLastHeader(ORGANIZATION_HEADER).getValue(), is("org")); var requestMap = entityAsMap(httpPost.getEntity().getContent()); - assertThat(requestMap, aMapWithSize(4)); - assertThat(requestMap.get("messages"), is(List.of(Map.of("role", "user", "content", "abc")))); - assertThat(requestMap.get("model"), is("model")); - assertThat(requestMap.get("user"), is("user")); - assertThat(requestMap.get("n"), is(1)); + assertRequestMapWithUser(requestMap, "user"); + } public void testCreateRequest_WithDefaultUrlAndWithoutUserOrganization() throws URISyntaxException, IOException { - var request = createRequest(null, null, "secret", "abc", "model", null); + var request = createRequest(null, null, "secret", "abc", "model", null, true); var httpRequest = request.createHttpRequest(); assertThat(httpRequest.httpRequestBase(), instanceOf(HttpPost.class)); var httpPost = (HttpPost) httpRequest.httpRequestBase(); - assertThat(httpPost.getURI().toString(), is(OpenAiChatCompletionRequest.buildDefaultUri().toString())); + assertThat(httpPost.getURI().toString(), is(OpenAiUnifiedChatCompletionRequest.buildDefaultUri().toString())); assertThat(httpPost.getLastHeader(HttpHeaders.CONTENT_TYPE).getValue(), is(XContentType.JSON.mediaType())); assertThat(httpPost.getLastHeader(HttpHeaders.AUTHORIZATION).getValue(), is("Bearer secret")); assertNull(httpPost.getLastHeader(ORGANIZATION_HEADER)); var requestMap = entityAsMap(httpPost.getEntity().getContent()); - assertThat(requestMap, aMapWithSize(3)); - assertThat(requestMap.get("messages"), is(List.of(Map.of("role", "user", "content", "abc")))); - assertThat(requestMap.get("model"), is("model")); - assertThat(requestMap.get("n"), is(1)); + assertRequestMapWithoutUser(requestMap); } - public void testCreateRequest_WithStreaming() throws URISyntaxException, IOException { + public void testCreateRequest_WithStreaming() throws IOException { var request = createRequest(null, null, "secret", "abc", "model", null, true); var httpRequest = request.createHttpRequest(); @@ -99,29 +106,31 @@ public void testCreateRequest_WithStreaming() throws URISyntaxException, IOExcep } public void testTruncate_DoesNotReduceInputTextSize() throws URISyntaxException, IOException { - var request = createRequest(null, null, "secret", "abcd", "model", null); + var request = createRequest(null, null, "secret", "abcd", "model", null, true); var truncatedRequest = request.truncate(); - assertThat(request.getURI().toString(), is(OpenAiChatCompletionRequest.buildDefaultUri().toString())); + assertThat(request.getURI().toString(), is(OpenAiUnifiedChatCompletionRequest.buildDefaultUri().toString())); var httpRequest = truncatedRequest.createHttpRequest(); assertThat(httpRequest.httpRequestBase(), instanceOf(HttpPost.class)); var httpPost = (HttpPost) httpRequest.httpRequestBase(); var requestMap = entityAsMap(httpPost.getEntity().getContent()); - assertThat(requestMap, aMapWithSize(3)); + assertThat(requestMap, aMapWithSize(5)); // We do not truncate for OpenAi chat completions assertThat(requestMap.get("messages"), is(List.of(Map.of("role", "user", "content", "abcd")))); assertThat(requestMap.get("model"), is("model")); assertThat(requestMap.get("n"), is(1)); + assertTrue((Boolean) requestMap.get("stream")); + assertThat(requestMap.get("stream_options"), is(Map.of("include_usage", true))); } public void testTruncationInfo_ReturnsNull() { - var request = createRequest(null, null, "secret", "abcd", "model", null); + var request = createRequest(null, null, "secret", "abcd", "model", null, true); assertNull(request.getTruncationInfo()); } - public static OpenAiChatCompletionRequest createRequest( + public static OpenAiUnifiedChatCompletionRequest createRequest( @Nullable String url, @Nullable String org, String apiKey, @@ -132,7 +141,7 @@ public static OpenAiChatCompletionRequest createRequest( return createRequest(url, org, apiKey, input, model, user, false); } - public static OpenAiChatCompletionRequest createRequest( + public static OpenAiUnifiedChatCompletionRequest createRequest( @Nullable String url, @Nullable String org, String apiKey, @@ -142,7 +151,7 @@ public static OpenAiChatCompletionRequest createRequest( boolean stream ) { var chatCompletionModel = OpenAiChatCompletionModelTests.createChatCompletionModel(url, org, apiKey, model, user); - return new OpenAiChatCompletionRequest(List.of(input), chatCompletionModel, stream); + return new OpenAiUnifiedChatCompletionRequest(new UnifiedChatInput(List.of(input), "user", stream), chatCompletionModel); } } diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/rest/BaseInferenceActionTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/rest/BaseInferenceActionTests.java index 05a8d52be5df..5528c80066b0 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/rest/BaseInferenceActionTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/rest/BaseInferenceActionTests.java @@ -8,11 +8,14 @@ package org.elasticsearch.xpack.inference.rest; import org.apache.lucene.util.SetOnce; +import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.core.TimeValue; +import org.elasticsearch.inference.TaskType; import org.elasticsearch.rest.RestChannel; import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.RestRequestTests; import org.elasticsearch.rest.action.RestChunkedToXContentListener; import org.elasticsearch.test.rest.FakeRestRequest; import org.elasticsearch.test.rest.RestActionTestCase; @@ -26,6 +29,10 @@ import java.util.Map; import static org.elasticsearch.rest.RestRequest.Method.POST; +import static org.elasticsearch.xpack.inference.rest.BaseInferenceAction.parseParams; +import static org.elasticsearch.xpack.inference.rest.BaseInferenceAction.parseTimeout; +import static org.elasticsearch.xpack.inference.rest.Paths.INFERENCE_ID; +import static org.elasticsearch.xpack.inference.rest.Paths.TASK_TYPE_OR_INFERENCE_ID; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; @@ -56,6 +63,42 @@ private static String route(String param) { return "_route/" + param; } + public void testParseParams_ExtractsInferenceIdAndTaskType() { + var params = parseParams( + RestRequestTests.contentRestRequest("{}", Map.of(INFERENCE_ID, "id", TASK_TYPE_OR_INFERENCE_ID, TaskType.COMPLETION.toString())) + ); + assertThat(params, is(new BaseInferenceAction.Params("id", TaskType.COMPLETION))); + } + + public void testParseParams_DefaultsToTaskTypeAny_WhenInferenceId_IsMissing() { + var params = parseParams( + RestRequestTests.contentRestRequest("{}", Map.of(TASK_TYPE_OR_INFERENCE_ID, TaskType.COMPLETION.toString())) + ); + assertThat(params, is(new BaseInferenceAction.Params("completion", TaskType.ANY))); + } + + public void testParseParams_ThrowsStatusException_WhenTaskTypeIsMissing() { + var e = expectThrows( + ElasticsearchStatusException.class, + () -> parseParams(RestRequestTests.contentRestRequest("{}", Map.of(INFERENCE_ID, "id"))) + ); + assertThat(e.getMessage(), is("Task type must not be null")); + } + + public void testParseTimeout_ReturnsTimeout() { + var timeout = parseTimeout( + RestRequestTests.contentRestRequest("{}", Map.of(InferenceAction.Request.TIMEOUT.getPreferredName(), "4s")) + ); + + assertThat(timeout, is(TimeValue.timeValueSeconds(4))); + } + + public void testParseTimeout_ReturnsDefaultTimeout() { + var timeout = parseTimeout(RestRequestTests.contentRestRequest("{}", Map.of())); + + assertThat(timeout, is(TimeValue.timeValueSeconds(30))); + } + public void testUsesDefaultTimeout() { SetOnce executeCalled = new SetOnce<>(); verifyingClient.setExecuteVerifier(((actionType, actionRequest) -> { diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/rest/RestUnifiedCompletionInferenceActionTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/rest/RestUnifiedCompletionInferenceActionTests.java new file mode 100644 index 000000000000..5acfe67b175d --- /dev/null +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/rest/RestUnifiedCompletionInferenceActionTests.java @@ -0,0 +1,81 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.inference.rest; + +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.rest.AbstractRestChannel; +import org.elasticsearch.rest.RestChannel; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.RestResponse; +import org.elasticsearch.test.rest.FakeRestRequest; +import org.elasticsearch.test.rest.RestActionTestCase; +import org.elasticsearch.xcontent.XContentType; +import org.elasticsearch.xpack.core.inference.action.UnifiedCompletionAction; +import org.junit.Before; + +import static org.elasticsearch.xpack.inference.rest.BaseInferenceActionTests.createResponse; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; + +public class RestUnifiedCompletionInferenceActionTests extends RestActionTestCase { + + @Before + public void setUpAction() { + controller().registerHandler(new RestUnifiedCompletionInferenceAction()); + } + + public void testStreamIsTrue() { + SetOnce executeCalled = new SetOnce<>(); + verifyingClient.setExecuteVerifier(((actionType, actionRequest) -> { + assertThat(actionRequest, instanceOf(UnifiedCompletionAction.Request.class)); + + var request = (UnifiedCompletionAction.Request) actionRequest; + assertThat(request.isStreaming(), is(true)); + + executeCalled.set(true); + return createResponse(); + })); + + var requestBody = """ + { + "messages": [ + { + "content": "abc", + "role": "user" + } + ] + } + """; + + RestRequest inferenceRequest = new FakeRestRequest.Builder(xContentRegistry()).withMethod(RestRequest.Method.POST) + .withPath("_inference/completion/test/_unified") + .withContent(new BytesArray(requestBody), XContentType.JSON) + .build(); + + final SetOnce responseSetOnce = new SetOnce<>(); + dispatchRequest(inferenceRequest, new AbstractRestChannel(inferenceRequest, true) { + @Override + public void sendResponse(RestResponse response) { + responseSetOnce.set(response); + } + }); + + // the response content will be null when there is no error + assertNull(responseSetOnce.get().content()); + assertThat(executeCalled.get(), equalTo(true)); + } + + private void dispatchRequest(final RestRequest request, final RestChannel channel) { + ThreadContext threadContext = new ThreadContext(Settings.EMPTY); + controller().dispatchRequest(request, channel, threadContext); + } +} diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/SenderServiceTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/SenderServiceTests.java index 47a96bf78dda..6768583598b2 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/SenderServiceTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/SenderServiceTests.java @@ -26,6 +26,7 @@ import org.elasticsearch.xpack.inference.external.http.sender.HttpRequestSender; import org.elasticsearch.xpack.inference.external.http.sender.InferenceInputs; import org.elasticsearch.xpack.inference.external.http.sender.Sender; +import org.elasticsearch.xpack.inference.external.http.sender.UnifiedChatInput; import org.junit.After; import org.junit.Before; @@ -119,6 +120,14 @@ protected void doInfer( } + @Override + protected void doUnifiedCompletionInfer( + Model model, + UnifiedChatInput inputs, + TimeValue timeout, + ActionListener listener + ) {} + @Override protected void doChunkedInfer( Model model, diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/openai/OpenAiServiceTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/openai/OpenAiServiceTests.java index 76b5d6fee2c5..159b77789482 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/openai/OpenAiServiceTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/openai/OpenAiServiceTests.java @@ -27,6 +27,7 @@ import org.elasticsearch.inference.Model; import org.elasticsearch.inference.SimilarityMeasure; import org.elasticsearch.inference.TaskType; +import org.elasticsearch.inference.UnifiedCompletionRequest; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.http.MockResponse; import org.elasticsearch.test.http.MockWebServer; @@ -920,6 +921,68 @@ public void testInfer_SendsRequest() throws IOException { } } + public void testUnifiedCompletionInfer() throws Exception { + // The escapes are because the streaming response must be on a single line + String responseJson = """ + data: {\ + "id":"12345",\ + "object":"chat.completion.chunk",\ + "created":123456789,\ + "model":"gpt-4o-mini",\ + "system_fingerprint": "123456789",\ + "choices":[\ + {\ + "index":0,\ + "delta":{\ + "content":"hello, world"\ + },\ + "logprobs":null,\ + "finish_reason":"stop"\ + }\ + ],\ + "usage":{\ + "prompt_tokens": 16,\ + "completion_tokens": 28,\ + "total_tokens": 44,\ + "prompt_tokens_details": {\ + "cached_tokens": 0,\ + "audio_tokens": 0\ + },\ + "completion_tokens_details": {\ + "reasoning_tokens": 0,\ + "audio_tokens": 0,\ + "accepted_prediction_tokens": 0,\ + "rejected_prediction_tokens": 0\ + }\ + }\ + } + + """; + webServer.enqueue(new MockResponse().setResponseCode(200).setBody(responseJson)); + + var senderFactory = HttpRequestSenderTests.createSenderFactory(threadPool, clientManager); + try (var service = new OpenAiService(senderFactory, createWithEmptySettings(threadPool))) { + var model = OpenAiChatCompletionModelTests.createChatCompletionModel(getUrl(webServer), "org", "secret", "model", "user"); + PlainActionFuture listener = new PlainActionFuture<>(); + service.unifiedCompletionInfer( + model, + UnifiedCompletionRequest.of( + List.of( + new UnifiedCompletionRequest.Message(new UnifiedCompletionRequest.ContentString("hello"), "user", null, null, null) + ) + ), + InferenceAction.Request.DEFAULT_TIMEOUT, + listener + ); + + var result = listener.actionGet(TIMEOUT); + InferenceEventsAssertion.assertThat(result).hasFinishedStream().hasNoErrors().hasEvent(""" + {"id":"12345","choices":[{"delta":{"content":"hello, world"},"finish_reason":"stop","index":0}],""" + """ + "model":"gpt-4o-mini","object":"chat.completion.chunk",""" + """ + "usage":{"completion_tokens":28,"prompt_tokens":16,"total_tokens":44}}"""); + } + } + public void testInfer_StreamRequest() throws Exception { String responseJson = """ data: {\ diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/openai/completion/OpenAiChatCompletionModelTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/openai/completion/OpenAiChatCompletionModelTests.java index ab1786f0a584..e7ac4cf879e9 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/openai/completion/OpenAiChatCompletionModelTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/services/openai/completion/OpenAiChatCompletionModelTests.java @@ -10,9 +10,11 @@ import org.elasticsearch.common.settings.SecureString; import org.elasticsearch.core.Nullable; import org.elasticsearch.inference.TaskType; +import org.elasticsearch.inference.UnifiedCompletionRequest; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.inference.services.settings.DefaultSecretSettings; +import java.util.List; import java.util.Map; import static org.elasticsearch.xpack.inference.services.openai.completion.OpenAiChatCompletionRequestTaskSettingsTests.getChatCompletionRequestTaskSettingsMap; @@ -42,10 +44,48 @@ public void testOverrideWith_EmptyMap() { public void testOverrideWith_NullMap() { var model = createChatCompletionModel("url", "org", "api_key", "model_name", null); - var overriddenModel = OpenAiChatCompletionModel.of(model, null); + var overriddenModel = OpenAiChatCompletionModel.of(model, (Map) null); assertThat(overriddenModel, sameInstance(model)); } + public void testOverrideWith_UnifiedCompletionRequest_OverridesModelId() { + var model = createChatCompletionModel("url", "org", "api_key", "model_name", "user"); + var request = new UnifiedCompletionRequest( + List.of(new UnifiedCompletionRequest.Message(new UnifiedCompletionRequest.ContentString("hello"), "role", null, null, null)), + "different_model", + null, + null, + null, + null, + null, + null + ); + + assertThat( + OpenAiChatCompletionModel.of(model, request), + is(createChatCompletionModel("url", "org", "api_key", "different_model", "user")) + ); + } + + public void testOverrideWith_UnifiedCompletionRequest_UsesModelFields_WhenRequestDoesNotOverride() { + var model = createChatCompletionModel("url", "org", "api_key", "model_name", "user"); + var request = new UnifiedCompletionRequest( + List.of(new UnifiedCompletionRequest.Message(new UnifiedCompletionRequest.ContentString("hello"), "role", null, null, null)), + null, // not overriding model + null, + null, + null, + null, + null, + null + ); + + assertThat( + OpenAiChatCompletionModel.of(model, request), + is(createChatCompletionModel("url", "org", "api_key", "model_name", "user")) + ); + } + public static OpenAiChatCompletionModel createChatCompletionModel( String url, @Nullable String org, diff --git a/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java b/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java index 8df10037affd..c91314716cf9 100644 --- a/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java +++ b/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java @@ -386,6 +386,7 @@ public class Constants { "cluster:monitor/xpack/esql/stats/dist", "cluster:monitor/xpack/inference", "cluster:monitor/xpack/inference/get", + "cluster:monitor/xpack/inference/unified", "cluster:monitor/xpack/inference/diagnostics/get", "cluster:monitor/xpack/inference/services/get", "cluster:monitor/xpack/info", From 4df28327efddc8864590d33eb68105146b8b7754 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Sat, 7 Dec 2024 07:55:27 +1100 Subject: [PATCH 12/27] Mute org.elasticsearch.xpack.esql.action.EsqlActionTaskIT testCancelRequestWhenFailingFetchingPages #118193 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index a39265756599..4431d3661eb0 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -263,6 +263,9 @@ tests: - class: org.elasticsearch.xpack.inference.DefaultEndPointsIT method: testInferDeploysDefaultRerank issue: https://github.com/elastic/elasticsearch/issues/118184 +- class: org.elasticsearch.xpack.esql.action.EsqlActionTaskIT + method: testCancelRequestWhenFailingFetchingPages + issue: https://github.com/elastic/elasticsearch/issues/118193 # Examples: # From c9feb7690bfe880784406d4ad32ca7d1547d8cfe Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Sat, 7 Dec 2024 08:17:11 +1100 Subject: [PATCH 13/27] Mute org.elasticsearch.packaging.test.MemoryLockingTests test20MemoryLockingEnabled #118195 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index 4431d3661eb0..22a53673dcb3 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -266,6 +266,9 @@ tests: - class: org.elasticsearch.xpack.esql.action.EsqlActionTaskIT method: testCancelRequestWhenFailingFetchingPages issue: https://github.com/elastic/elasticsearch/issues/118193 +- class: org.elasticsearch.packaging.test.MemoryLockingTests + method: test20MemoryLockingEnabled + issue: https://github.com/elastic/elasticsearch/issues/118195 # Examples: # From 287ed8a1c177a98e7397463e9a02794fed8b2761 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Sat, 7 Dec 2024 08:17:26 +1100 Subject: [PATCH 14/27] Mute org.elasticsearch.packaging.test.ArchiveTests test42AutoconfigurationNotTriggeredWhenNodeCannotBecomeMaster #118196 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index 22a53673dcb3..7c5df966e6bd 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -269,6 +269,9 @@ tests: - class: org.elasticsearch.packaging.test.MemoryLockingTests method: test20MemoryLockingEnabled issue: https://github.com/elastic/elasticsearch/issues/118195 +- class: org.elasticsearch.packaging.test.ArchiveTests + method: test42AutoconfigurationNotTriggeredWhenNodeCannotBecomeMaster + issue: https://github.com/elastic/elasticsearch/issues/118196 # Examples: # From 05eee61038693a11ed8fc8ea5c5a66d460d5c0cf Mon Sep 17 00:00:00 2001 From: Kazuma Arimura Date: Sat, 7 Dec 2024 02:00:03 +0000 Subject: [PATCH 15/27] add documentation for kuromoji_completion (#117808) --- docs/plugins/analysis-kuromoji.asciidoc | 36 +++++++++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/docs/plugins/analysis-kuromoji.asciidoc b/docs/plugins/analysis-kuromoji.asciidoc index 0a167bf3f024..217d88f36122 100644 --- a/docs/plugins/analysis-kuromoji.asciidoc +++ b/docs/plugins/analysis-kuromoji.asciidoc @@ -750,3 +750,39 @@ Which results in: ] } -------------------------------------------------- + +[[analysis-kuromoji-completion]] +==== `kuromoji_completion` token filter + +The `kuromoji_completion` token filter adds Japanese romanized tokens to the term attributes along with the original tokens (surface forms). + +[source,console] +-------------------------------------------------- +GET _analyze +{ + "analyzer": "kuromoji_completion", + "text": "寿司" <1> +} +-------------------------------------------------- + +<1> Returns `寿司`, `susi` (Kunrei-shiki) and `sushi` (Hepburn-shiki). + +The `kuromoji_completion` token filter accepts the following settings: + +`mode`:: ++ +-- + +The tokenization mode determines how the tokenizer handles compound and +unknown words. It can be set to: + +`index`:: + + Simple romanization. Expected to be used when indexing. + +`query`:: + + Input Method aware romanization. Expected to be used when querying. + +Defaults to `index`. +-- From 39c7e0bc2fa6345cde605257c80182a83c2cc951 Mon Sep 17 00:00:00 2001 From: Ryan Ernst Date: Fri, 6 Dec 2024 18:24:08 -0800 Subject: [PATCH 16/27] Always write unicast hosts file in tests (#118121) This commit adds the ability to restart a node within a test cluster. The newly started node uses the same configuration and directories. Since we use ephemeral ports, the unicast hosts list is rewritten for all nodes once the restarted node comes back up. --- .../test/cluster/local/DefaultLocalClusterHandle.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/DefaultLocalClusterHandle.java b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/DefaultLocalClusterHandle.java index eb45aacda68d..13adde1da8a6 100644 --- a/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/DefaultLocalClusterHandle.java +++ b/test/test-clusters/src/main/java/org/elasticsearch/test/cluster/local/DefaultLocalClusterHandle.java @@ -176,8 +176,9 @@ public long getPid(int index) { return nodes.get(index).getPid(); } + @Override public void stopNode(int index, boolean forcibly) { - nodes.get(index).stop(false); + nodes.get(index).stop(forcibly); } @Override @@ -252,9 +253,8 @@ private void writeUnicastHostsFile() { execute(() -> nodes.parallelStream().forEach(node -> { try { Path hostsFile = node.getWorkingDir().resolve("config").resolve("unicast_hosts.txt"); - if (Files.notExists(hostsFile)) { - Files.writeString(hostsFile, transportUris); - } + LOGGER.info("Writing unicast hosts file {} for node {}", hostsFile, node.getName()); + Files.writeString(hostsFile, transportUris); } catch (IOException e) { throw new UncheckedIOException("Failed to write unicast_hosts for: " + node, e); } From bc0b77e147e2baa84f9d920c1f9604211f6e3ab8 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Sat, 7 Dec 2024 16:17:38 +1100 Subject: [PATCH 17/27] Mute org.elasticsearch.packaging.test.ArchiveTests test43AutoconfigurationNotTriggeredWhenTlsAlreadyConfigured #118202 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index 7c5df966e6bd..05f6e0b57415 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -272,6 +272,9 @@ tests: - class: org.elasticsearch.packaging.test.ArchiveTests method: test42AutoconfigurationNotTriggeredWhenNodeCannotBecomeMaster issue: https://github.com/elastic/elasticsearch/issues/118196 +- class: org.elasticsearch.packaging.test.ArchiveTests + method: test43AutoconfigurationNotTriggeredWhenTlsAlreadyConfigured + issue: https://github.com/elastic/elasticsearch/issues/118202 # Examples: # From 2005e2eec1f53c26b53fcafb58e312fe10b5aae4 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Sun, 8 Dec 2024 00:16:15 +1100 Subject: [PATCH 18/27] Mute org.elasticsearch.packaging.test.ArchiveTests test44AutoConfigurationNotTriggeredOnNotWriteableConfDir #118208 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index 05f6e0b57415..5914978cba07 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -275,6 +275,9 @@ tests: - class: org.elasticsearch.packaging.test.ArchiveTests method: test43AutoconfigurationNotTriggeredWhenTlsAlreadyConfigured issue: https://github.com/elastic/elasticsearch/issues/118202 +- class: org.elasticsearch.packaging.test.ArchiveTests + method: test44AutoConfigurationNotTriggeredOnNotWriteableConfDir + issue: https://github.com/elastic/elasticsearch/issues/118208 # Examples: # From eb0a21efd8946df04a3f6f0457bb0a1b73b50bf0 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sat, 7 Dec 2024 15:13:56 +0100 Subject: [PATCH 19/27] Speedup OsStats initialization (#118141) Similar to other OS/FS type stats we can optimize here. Found this as a slowdown when profiling tests in a loop during test fixing. This helps node startup and maybe more importantly test performance. No need to initialize the stats eagerly when we can just get them as we load them the first time. --- .../elasticsearch/monitor/os/OsService.java | 28 +++++++++++++------ 1 file changed, 20 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/monitor/os/OsService.java b/server/src/main/java/org/elasticsearch/monitor/os/OsService.java index 7609cc14c6b3..ceed2b0e41fc 100644 --- a/server/src/main/java/org/elasticsearch/monitor/os/OsService.java +++ b/server/src/main/java/org/elasticsearch/monitor/os/OsService.java @@ -25,7 +25,6 @@ public class OsService implements ReportingService { private static final Logger logger = LogManager.getLogger(OsService.class); - private final OsProbe probe; private final OsInfo info; private final SingleObjectCache osStatsCache; @@ -37,10 +36,9 @@ public class OsService implements ReportingService { ); public OsService(Settings settings) throws IOException { - this.probe = OsProbe.getInstance(); TimeValue refreshInterval = REFRESH_INTERVAL_SETTING.get(settings); - this.info = probe.osInfo(refreshInterval.millis(), EsExecutors.nodeProcessors(settings)); - this.osStatsCache = new OsStatsCache(refreshInterval, probe.osStats()); + this.info = OsProbe.getInstance().osInfo(refreshInterval.millis(), EsExecutors.nodeProcessors(settings)); + this.osStatsCache = new OsStatsCache(refreshInterval); logger.debug("using refresh_interval [{}]", refreshInterval); } @@ -53,14 +51,28 @@ public OsStats stats() { return osStatsCache.getOrRefresh(); } - private class OsStatsCache extends SingleObjectCache { - OsStatsCache(TimeValue interval, OsStats initValue) { - super(interval, initValue); + private static class OsStatsCache extends SingleObjectCache { + + private static final OsStats MISSING = new OsStats( + 0L, + new OsStats.Cpu((short) 0, new double[0]), + new OsStats.Mem(0, 0, 0), + new OsStats.Swap(0, 0), + null + ); + + OsStatsCache(TimeValue interval) { + super(interval, MISSING); } @Override protected OsStats refresh() { - return probe.osStats(); + return OsProbe.getInstance().osStats(); + } + + @Override + protected boolean needsRefresh() { + return getNoRefresh() == MISSING || super.needsRefresh(); } } } From 230b283beaff21c331c3da9c31d29211e6392d75 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Sun, 8 Dec 2024 08:15:30 +1100 Subject: [PATCH 20/27] Mute org.elasticsearch.packaging.test.ArchiveTests test51AutoConfigurationWithPasswordProtectedKeystore #118212 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index 5914978cba07..273094134fa2 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -278,6 +278,9 @@ tests: - class: org.elasticsearch.packaging.test.ArchiveTests method: test44AutoConfigurationNotTriggeredOnNotWriteableConfDir issue: https://github.com/elastic/elasticsearch/issues/118208 +- class: org.elasticsearch.packaging.test.ArchiveTests + method: test51AutoConfigurationWithPasswordProtectedKeystore + issue: https://github.com/elastic/elasticsearch/issues/118212 # Examples: # From 9071e80a923690dcb37369849450da6deb38f95f Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Sun, 8 Dec 2024 08:55:27 +1100 Subject: [PATCH 21/27] Mute org.elasticsearch.xpack.inference.InferenceCrudIT testUnifiedCompletionInference #118210 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index 273094134fa2..29621676fa83 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -281,6 +281,9 @@ tests: - class: org.elasticsearch.packaging.test.ArchiveTests method: test51AutoConfigurationWithPasswordProtectedKeystore issue: https://github.com/elastic/elasticsearch/issues/118212 +- class: org.elasticsearch.xpack.inference.InferenceCrudIT + method: testUnifiedCompletionInference + issue: https://github.com/elastic/elasticsearch/issues/118210 # Examples: # From 92e0f972fb1795fb8c7e96c0b60aa6f2f2405b83 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Sun, 8 Dec 2024 16:25:11 +1100 Subject: [PATCH 22/27] Mute org.elasticsearch.ingest.common.IngestCommonClientYamlTestSuiteIT org.elasticsearch.ingest.common.IngestCommonClientYamlTestSuiteIT #118215 --- muted-tests.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index 29621676fa83..e00bcb068ac2 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -284,6 +284,8 @@ tests: - class: org.elasticsearch.xpack.inference.InferenceCrudIT method: testUnifiedCompletionInference issue: https://github.com/elastic/elasticsearch/issues/118210 +- class: org.elasticsearch.ingest.common.IngestCommonClientYamlTestSuiteIT + issue: https://github.com/elastic/elasticsearch/issues/118215 # Examples: # From 2b0bf196be87bc2b10580f2d89b971b82927cdd6 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Sun, 8 Dec 2024 16:59:40 +1100 Subject: [PATCH 23/27] Mute org.elasticsearch.datastreams.DataStreamsClientYamlTestSuiteIT test {p0=data_stream/120_data_streams_stats/Multiple data stream} #118217 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index e00bcb068ac2..3977b63224f2 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -286,6 +286,9 @@ tests: issue: https://github.com/elastic/elasticsearch/issues/118210 - class: org.elasticsearch.ingest.common.IngestCommonClientYamlTestSuiteIT issue: https://github.com/elastic/elasticsearch/issues/118215 +- class: org.elasticsearch.datastreams.DataStreamsClientYamlTestSuiteIT + method: test {p0=data_stream/120_data_streams_stats/Multiple data stream} + issue: https://github.com/elastic/elasticsearch/issues/118217 # Examples: # From 4a8a8a0bfe337aff50774c2c398060399a4d93e5 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Sun, 8 Dec 2024 18:18:14 +1100 Subject: [PATCH 24/27] Mute org.elasticsearch.xpack.security.operator.OperatorPrivilegesIT testEveryActionIsEitherOperatorOnlyOrNonOperator #118220 --- muted-tests.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index 3977b63224f2..bfcac3a02b16 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -289,6 +289,9 @@ tests: - class: org.elasticsearch.datastreams.DataStreamsClientYamlTestSuiteIT method: test {p0=data_stream/120_data_streams_stats/Multiple data stream} issue: https://github.com/elastic/elasticsearch/issues/118217 +- class: org.elasticsearch.xpack.security.operator.OperatorPrivilegesIT + method: testEveryActionIsEitherOperatorOnlyOrNonOperator + issue: https://github.com/elastic/elasticsearch/issues/118220 # Examples: # From c09c4f427709f2a4373372f42f74abb556273c57 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Mon, 9 Dec 2024 08:27:23 +1100 Subject: [PATCH 25/27] Mute org.elasticsearch.validation.DotPrefixClientYamlTestSuiteIT org.elasticsearch.validation.DotPrefixClientYamlTestSuiteIT #118224 --- muted-tests.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/muted-tests.yml b/muted-tests.yml index bfcac3a02b16..a36723469189 100644 --- a/muted-tests.yml +++ b/muted-tests.yml @@ -292,6 +292,8 @@ tests: - class: org.elasticsearch.xpack.security.operator.OperatorPrivilegesIT method: testEveryActionIsEitherOperatorOnlyOrNonOperator issue: https://github.com/elastic/elasticsearch/issues/118220 +- class: org.elasticsearch.validation.DotPrefixClientYamlTestSuiteIT + issue: https://github.com/elastic/elasticsearch/issues/118224 # Examples: # From 8107cc9e5e234447db9132068973c135fec2ff36 Mon Sep 17 00:00:00 2001 From: Keith Massey Date: Sun, 8 Dec 2024 17:56:09 -0600 Subject: [PATCH 26/27] Adding reindex data stream rest action (#118109) * Adding a _migration/reindex endpoint * Adding rest api spec and test * Adding a feature flag for reindex data streams * updating json spec * fixing a typo * Changing mode to an enum * Moving ParseFields into public static finals * Commenting out test that leaves task running, until we add a cancel API * Removing persistent task id from output * replacing a string with a variable --- .../rest-api-spec/api/migrate.reindex.json | 29 +++++ .../ReindexDataStreamTransportActionIT.java | 10 +- .../xpack/migrate/MigratePlugin.java | 102 +++++++++++++----- .../action/ReindexDataStreamAction.java | 89 +++++++++++++-- .../rest/RestMigrationReindexAction.java | 64 +++++++++++ .../action/ReindexDataStreamRequestTests.java | 39 +++++++ .../ReindexDataStreamResponseTests.java | 2 +- .../rest-api-spec/test/migrate/10_reindex.yml | 89 +++++++++++++++ 8 files changed, 387 insertions(+), 37 deletions(-) create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/api/migrate.reindex.json create mode 100644 x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/rest/RestMigrationReindexAction.java create mode 100644 x-pack/plugin/migrate/src/test/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamRequestTests.java create mode 100644 x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/migrate/10_reindex.yml diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/migrate.reindex.json b/rest-api-spec/src/main/resources/rest-api-spec/api/migrate.reindex.json new file mode 100644 index 000000000000..149a90bc198b --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/migrate.reindex.json @@ -0,0 +1,29 @@ +{ + "migrate.reindex":{ + "documentation":{ + "url":"https://www.elastic.co/guide/en/elasticsearch/reference/master/data-stream-reindex.html", + "description":"This API reindexes all legacy backing indices for a data stream. It does this in a persistent task. The persistent task id is returned immediately, and the reindexing work is completed in that task" + }, + "stability":"experimental", + "visibility":"private", + "headers":{ + "accept": [ "application/json"], + "content_type": ["application/json"] + }, + "url":{ + "paths":[ + { + "path":"/_migration/reindex", + "methods":[ + "POST" + ] + } + ] + }, + "body":{ + "description":"The body contains the fields `mode` and `source.index, where the only mode currently supported is `upgrade`, and the `source.index` must be a data stream name", + "required":true + } + } +} + diff --git a/x-pack/plugin/migrate/src/internalClusterTest/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamTransportActionIT.java b/x-pack/plugin/migrate/src/internalClusterTest/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamTransportActionIT.java index 3b68fc9995b5..62716e11f172 100644 --- a/x-pack/plugin/migrate/src/internalClusterTest/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamTransportActionIT.java +++ b/x-pack/plugin/migrate/src/internalClusterTest/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamTransportActionIT.java @@ -51,7 +51,10 @@ protected Collection> nodePlugins() { public void testNonExistentDataStream() { String nonExistentDataStreamName = randomAlphaOfLength(50); - ReindexDataStreamRequest reindexDataStreamRequest = new ReindexDataStreamRequest(nonExistentDataStreamName); + ReindexDataStreamRequest reindexDataStreamRequest = new ReindexDataStreamRequest( + ReindexDataStreamAction.Mode.UPGRADE, + nonExistentDataStreamName + ); assertThrows( ResourceNotFoundException.class, () -> client().execute(new ActionType(ReindexDataStreamAction.NAME), reindexDataStreamRequest) @@ -61,7 +64,10 @@ public void testNonExistentDataStream() { public void testAlreadyUpToDateDataStream() throws Exception { String dataStreamName = randomAlphaOfLength(50).toLowerCase(Locale.ROOT); - ReindexDataStreamRequest reindexDataStreamRequest = new ReindexDataStreamRequest(dataStreamName); + ReindexDataStreamRequest reindexDataStreamRequest = new ReindexDataStreamRequest( + ReindexDataStreamAction.Mode.UPGRADE, + dataStreamName + ); createDataStream(dataStreamName); ReindexDataStreamResponse response = client().execute( new ActionType(ReindexDataStreamAction.NAME), diff --git a/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/MigratePlugin.java b/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/MigratePlugin.java index 118cd69ece4d..ac9e38da0742 100644 --- a/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/MigratePlugin.java +++ b/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/MigratePlugin.java @@ -11,21 +11,30 @@ import org.elasticsearch.action.ActionResponse; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.IndexScopedSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.settings.SettingsFilter; import org.elasticsearch.common.settings.SettingsModule; +import org.elasticsearch.features.NodeFeature; import org.elasticsearch.persistent.PersistentTaskParams; import org.elasticsearch.persistent.PersistentTaskState; import org.elasticsearch.persistent.PersistentTasksExecutor; import org.elasticsearch.plugins.ActionPlugin; import org.elasticsearch.plugins.PersistentTaskPlugin; import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.rest.RestController; +import org.elasticsearch.rest.RestHandler; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xcontent.NamedXContentRegistry; import org.elasticsearch.xcontent.ParseField; import org.elasticsearch.xpack.migrate.action.ReindexDataStreamAction; import org.elasticsearch.xpack.migrate.action.ReindexDataStreamTransportAction; +import org.elasticsearch.xpack.migrate.rest.RestMigrationReindexAction; import org.elasticsearch.xpack.migrate.task.ReindexDataStreamPersistentTaskExecutor; import org.elasticsearch.xpack.migrate.task.ReindexDataStreamPersistentTaskState; import org.elasticsearch.xpack.migrate.task.ReindexDataStreamStatus; @@ -34,47 +43,80 @@ import java.util.ArrayList; import java.util.List; +import java.util.function.Predicate; +import java.util.function.Supplier; + +import static org.elasticsearch.xpack.migrate.action.ReindexDataStreamAction.REINDEX_DATA_STREAM_FEATURE_FLAG; public class MigratePlugin extends Plugin implements ActionPlugin, PersistentTaskPlugin { + @Override + public List getRestHandlers( + Settings unused, + NamedWriteableRegistry namedWriteableRegistry, + RestController restController, + ClusterSettings clusterSettings, + IndexScopedSettings indexScopedSettings, + SettingsFilter settingsFilter, + IndexNameExpressionResolver indexNameExpressionResolver, + Supplier nodesInCluster, + Predicate clusterSupportsFeature + ) { + List handlers = new ArrayList<>(); + if (REINDEX_DATA_STREAM_FEATURE_FLAG.isEnabled()) { + handlers.add(new RestMigrationReindexAction()); + } + return handlers; + } + @Override public List> getActions() { List> actions = new ArrayList<>(); - actions.add(new ActionHandler<>(ReindexDataStreamAction.INSTANCE, ReindexDataStreamTransportAction.class)); + if (REINDEX_DATA_STREAM_FEATURE_FLAG.isEnabled()) { + actions.add(new ActionHandler<>(ReindexDataStreamAction.INSTANCE, ReindexDataStreamTransportAction.class)); + } return actions; } @Override public List getNamedXContent() { - return List.of( - new NamedXContentRegistry.Entry( - PersistentTaskState.class, - new ParseField(ReindexDataStreamPersistentTaskState.NAME), - ReindexDataStreamPersistentTaskState::fromXContent - ), - new NamedXContentRegistry.Entry( - PersistentTaskParams.class, - new ParseField(ReindexDataStreamTaskParams.NAME), - ReindexDataStreamTaskParams::fromXContent - ) - ); + if (REINDEX_DATA_STREAM_FEATURE_FLAG.isEnabled()) { + return List.of( + new NamedXContentRegistry.Entry( + PersistentTaskState.class, + new ParseField(ReindexDataStreamPersistentTaskState.NAME), + ReindexDataStreamPersistentTaskState::fromXContent + ), + new NamedXContentRegistry.Entry( + PersistentTaskParams.class, + new ParseField(ReindexDataStreamTaskParams.NAME), + ReindexDataStreamTaskParams::fromXContent + ) + ); + } else { + return List.of(); + } } @Override public List getNamedWriteables() { - return List.of( - new NamedWriteableRegistry.Entry( - PersistentTaskState.class, - ReindexDataStreamPersistentTaskState.NAME, - ReindexDataStreamPersistentTaskState::new - ), - new NamedWriteableRegistry.Entry( - PersistentTaskParams.class, - ReindexDataStreamTaskParams.NAME, - ReindexDataStreamTaskParams::new - ), - new NamedWriteableRegistry.Entry(Task.Status.class, ReindexDataStreamStatus.NAME, ReindexDataStreamStatus::new) - ); + if (REINDEX_DATA_STREAM_FEATURE_FLAG.isEnabled()) { + return List.of( + new NamedWriteableRegistry.Entry( + PersistentTaskState.class, + ReindexDataStreamPersistentTaskState.NAME, + ReindexDataStreamPersistentTaskState::new + ), + new NamedWriteableRegistry.Entry( + PersistentTaskParams.class, + ReindexDataStreamTaskParams.NAME, + ReindexDataStreamTaskParams::new + ), + new NamedWriteableRegistry.Entry(Task.Status.class, ReindexDataStreamStatus.NAME, ReindexDataStreamStatus::new) + ); + } else { + return List.of(); + } } @Override @@ -85,6 +127,12 @@ public List> getPersistentTasksExecutor( SettingsModule settingsModule, IndexNameExpressionResolver expressionResolver ) { - return List.of(new ReindexDataStreamPersistentTaskExecutor(client, clusterService, ReindexDataStreamTask.TASK_NAME, threadPool)); + if (REINDEX_DATA_STREAM_FEATURE_FLAG.isEnabled()) { + return List.of( + new ReindexDataStreamPersistentTaskExecutor(client, clusterService, ReindexDataStreamTask.TASK_NAME, threadPool) + ); + } else { + return List.of(); + } } } diff --git a/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamAction.java b/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamAction.java index 1785e6971f82..eb7a910df8c0 100644 --- a/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamAction.java +++ b/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamAction.java @@ -11,23 +11,41 @@ import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.util.FeatureFlag; +import org.elasticsearch.features.NodeFeature; +import org.elasticsearch.xcontent.ConstructingObjectParser; +import org.elasticsearch.xcontent.ParseField; +import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.ToXContentObject; import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentParser; import java.io.IOException; +import java.util.Locale; import java.util.Objects; +import java.util.function.Predicate; public class ReindexDataStreamAction extends ActionType { + public static final FeatureFlag REINDEX_DATA_STREAM_FEATURE_FLAG = new FeatureFlag("reindex_data_stream"); public static final ReindexDataStreamAction INSTANCE = new ReindexDataStreamAction(); public static final String NAME = "indices:admin/data_stream/reindex"; + public static final ParseField MODE_FIELD = new ParseField("mode"); + public static final ParseField SOURCE_FIELD = new ParseField("source"); + public static final ParseField INDEX_FIELD = new ParseField("index"); public ReindexDataStreamAction() { super(NAME); } + public enum Mode { + UPGRADE + } + public static class ReindexDataStreamResponse extends ActionResponse implements ToXContentObject { private final String taskId; @@ -49,7 +67,7 @@ public void writeTo(StreamOutput out) throws IOException { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(); - builder.field("task", getTaskId()); + builder.field("acknowledged", true); builder.endObject(); return builder; } @@ -70,22 +88,52 @@ public boolean equals(Object other) { } - public static class ReindexDataStreamRequest extends ActionRequest { + public static class ReindexDataStreamRequest extends ActionRequest implements IndicesRequest, ToXContent { + private final Mode mode; private final String sourceDataStream; - public ReindexDataStreamRequest(String sourceDataStream) { - super(); + public ReindexDataStreamRequest(Mode mode, String sourceDataStream) { + this.mode = mode; this.sourceDataStream = sourceDataStream; } public ReindexDataStreamRequest(StreamInput in) throws IOException { super(in); + this.mode = Mode.valueOf(in.readString()); this.sourceDataStream = in.readString(); } + private static final ConstructingObjectParser> PARSER = + new ConstructingObjectParser<>("migration_reindex", objects -> { + Mode mode = Mode.valueOf(((String) objects[0]).toUpperCase(Locale.ROOT)); + String source = (String) objects[1]; + return new ReindexDataStreamRequest(mode, source); + }); + + private static final ConstructingObjectParser SOURCE_PARSER = new ConstructingObjectParser<>( + SOURCE_FIELD.getPreferredName(), + false, + (a, id) -> (String) a[0] + ); + + static { + SOURCE_PARSER.declareString(ConstructingObjectParser.constructorArg(), INDEX_FIELD); + PARSER.declareString(ConstructingObjectParser.constructorArg(), MODE_FIELD); + PARSER.declareObject( + ConstructingObjectParser.constructorArg(), + (parser, id) -> SOURCE_PARSER.apply(parser, null), + SOURCE_FIELD + ); + } + + public static ReindexDataStreamRequest fromXContent(XContentParser parser) { + return PARSER.apply(parser, null); + } + @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); + out.writeString(mode.name()); out.writeString(sourceDataStream); } @@ -103,15 +151,42 @@ public String getSourceDataStream() { return sourceDataStream; } + public Mode getMode() { + return mode; + } + @Override public int hashCode() { - return Objects.hashCode(sourceDataStream); + return Objects.hash(mode, sourceDataStream); } @Override public boolean equals(Object other) { - return other instanceof ReindexDataStreamRequest - && sourceDataStream.equals(((ReindexDataStreamRequest) other).sourceDataStream); + return other instanceof ReindexDataStreamRequest otherRequest + && mode.equals(otherRequest.mode) + && sourceDataStream.equals(otherRequest.sourceDataStream); + } + + @Override + public String[] indices() { + return new String[] { sourceDataStream }; + } + + @Override + public IndicesOptions indicesOptions() { + return IndicesOptions.strictSingleIndexNoExpandForbidClosed(); + } + + /* + * This only exists for the sake of testing the xcontent parser + */ + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.field(MODE_FIELD.getPreferredName(), mode); + builder.startObject(SOURCE_FIELD.getPreferredName()); + builder.field(INDEX_FIELD.getPreferredName(), sourceDataStream); + builder.endObject(); + return builder; } } } diff --git a/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/rest/RestMigrationReindexAction.java b/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/rest/RestMigrationReindexAction.java new file mode 100644 index 000000000000..a7f630d68234 --- /dev/null +++ b/x-pack/plugin/migrate/src/main/java/org/elasticsearch/xpack/migrate/rest/RestMigrationReindexAction.java @@ -0,0 +1,64 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.migrate.rest; + +import org.elasticsearch.client.internal.node.NodeClient; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestChannel; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.RestResponse; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.rest.action.RestBuilderListener; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xpack.migrate.action.ReindexDataStreamAction; +import org.elasticsearch.xpack.migrate.action.ReindexDataStreamAction.ReindexDataStreamResponse; + +import java.io.IOException; +import java.util.List; + +import static org.elasticsearch.rest.RestRequest.Method.POST; + +public class RestMigrationReindexAction extends BaseRestHandler { + + @Override + public String getName() { + return "migration_reindex"; + } + + @Override + public List routes() { + return List.of(new Route(POST, "/_migration/reindex")); + } + + @Override + protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException { + ReindexDataStreamAction.ReindexDataStreamRequest reindexRequest; + try (XContentParser parser = request.contentParser()) { + reindexRequest = ReindexDataStreamAction.ReindexDataStreamRequest.fromXContent(parser); + } + return channel -> client.execute( + ReindexDataStreamAction.INSTANCE, + reindexRequest, + new ReindexDataStreamRestToXContentListener(channel) + ); + } + + static class ReindexDataStreamRestToXContentListener extends RestBuilderListener { + + ReindexDataStreamRestToXContentListener(RestChannel channel) { + super(channel); + } + + @Override + public RestResponse buildResponse(ReindexDataStreamResponse response, XContentBuilder builder) throws Exception { + response.toXContent(builder, channel.request()); + return new RestResponse(RestStatus.OK, builder); + } + } +} diff --git a/x-pack/plugin/migrate/src/test/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamRequestTests.java b/x-pack/plugin/migrate/src/test/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamRequestTests.java new file mode 100644 index 000000000000..9c7bf87b6cff --- /dev/null +++ b/x-pack/plugin/migrate/src/test/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamRequestTests.java @@ -0,0 +1,39 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.migrate.action; + +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.test.AbstractXContentSerializingTestCase; +import org.elasticsearch.xcontent.XContentParser; +import org.elasticsearch.xpack.migrate.action.ReindexDataStreamAction.ReindexDataStreamRequest; + +import java.io.IOException; + +public class ReindexDataStreamRequestTests extends AbstractXContentSerializingTestCase { + + @Override + protected ReindexDataStreamRequest createTestInstance() { + return new ReindexDataStreamRequest(ReindexDataStreamAction.Mode.UPGRADE, randomAlphaOfLength(40)); + } + + @Override + protected ReindexDataStreamRequest mutateInstance(ReindexDataStreamRequest instance) { + // There is currently only one possible value for mode, so we can't change it + return new ReindexDataStreamRequest(instance.getMode(), randomAlphaOfLength(50)); + } + + @Override + protected ReindexDataStreamRequest doParseInstance(XContentParser parser) throws IOException { + return ReindexDataStreamRequest.fromXContent(parser); + } + + @Override + protected Writeable.Reader instanceReader() { + return ReindexDataStreamRequest::new; + } +} diff --git a/x-pack/plugin/migrate/src/test/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamResponseTests.java b/x-pack/plugin/migrate/src/test/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamResponseTests.java index 06844577c4e3..d886fc660d7a 100644 --- a/x-pack/plugin/migrate/src/test/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamResponseTests.java +++ b/x-pack/plugin/migrate/src/test/java/org/elasticsearch/xpack/migrate/action/ReindexDataStreamResponseTests.java @@ -43,7 +43,7 @@ public void testToXContent() throws IOException { builder.humanReadable(true); response.toXContent(builder, EMPTY_PARAMS); try (XContentParser parser = createParser(JsonXContent.jsonXContent, BytesReference.bytes(builder))) { - assertThat(parser.map(), equalTo(Map.of("task", response.getTaskId()))); + assertThat(parser.map(), equalTo(Map.of("acknowledged", true))); } } } diff --git a/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/migrate/10_reindex.yml b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/migrate/10_reindex.yml new file mode 100644 index 000000000000..01a41b3aa8c9 --- /dev/null +++ b/x-pack/plugin/src/yamlRestTest/resources/rest-api-spec/test/migrate/10_reindex.yml @@ -0,0 +1,89 @@ +--- +setup: + - do: + cluster.health: + wait_for_status: yellow + +--- +"Test Reindex With Unsupported Mode": + - do: + catch: /illegal_argument_exception/ + migrate.reindex: + body: | + { + "mode": "unsupported_mode", + "source": { + "index": "my-data-stream" + } + } + +--- +"Test Reindex With Nonexistent Data Stream": + - do: + catch: /resource_not_found_exception/ + migrate.reindex: + body: | + { + "mode": "upgrade", + "source": { + "index": "my-data-stream" + } + } + + - do: + catch: /resource_not_found_exception/ + migrate.reindex: + body: | + { + "mode": "upgrade", + "source": { + "index": "my-data-stream1,my-data-stream2" + } + } + + +--- +"Test Reindex With Bad Data Stream Name": + - do: + catch: /illegal_argument_exception/ + migrate.reindex: + body: | + { + "mode": "upgrade", + "source": { + "index": "my-data-stream*" + } + } + +--- +"Test Reindex With Existing Data Stream": + - do: + indices.put_index_template: + name: my-template1 + body: + index_patterns: [my-data-stream*] + template: + mappings: + properties: + '@timestamp': + type: date + 'foo': + type: keyword + data_stream: {} + + - do: + indices.create_data_stream: + name: my-data-stream + - is_true: acknowledged + +# Uncomment once the cancel API is in place +# - do: +# migrate.reindex: +# body: | +# { +# "mode": "upgrade", +# "source": { +# "index": "my-data-stream" +# } +# } +# - match: { task: "reindex-data-stream-my-data-stream" } From d411ad82daee10d298c2eb47d3fd900a83091baf Mon Sep 17 00:00:00 2001 From: Nick Tindall Date: Mon, 9 Dec 2024 13:44:10 +1100 Subject: [PATCH 27/27] AbstractRepositoryS3RestTestCase: Parse response for assertion (#118230) --- .../s3/AbstractRepositoryS3RestTestCase.java | 32 +++++++++---------- 1 file changed, 15 insertions(+), 17 deletions(-) diff --git a/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/AbstractRepositoryS3RestTestCase.java b/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/AbstractRepositoryS3RestTestCase.java index 2199a6452175..67ada622efee 100644 --- a/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/AbstractRepositoryS3RestTestCase.java +++ b/modules/repository-s3/src/javaRestTest/java/org/elasticsearch/repositories/s3/AbstractRepositoryS3RestTestCase.java @@ -19,6 +19,7 @@ import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.rest.ESRestTestCase; +import org.elasticsearch.test.rest.ObjectPath; import java.io.Closeable; import java.io.IOException; @@ -27,7 +28,6 @@ import java.util.function.UnaryOperator; import java.util.stream.Collectors; -import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -152,10 +152,9 @@ private void testNonexistentBucket(Boolean readonly) throws Exception { final var responseException = expectThrows(ResponseException.class, () -> client().performRequest(registerRequest)); assertEquals(RestStatus.INTERNAL_SERVER_ERROR.getStatus(), responseException.getResponse().getStatusLine().getStatusCode()); - assertThat( - responseException.getMessage(), - allOf(containsString("repository_verification_exception"), containsString("is not accessible on master node")) - ); + final var responseObjectPath = ObjectPath.createFromResponse(responseException.getResponse()); + assertThat(responseObjectPath.evaluate("error.type"), equalTo("repository_verification_exception")); + assertThat(responseObjectPath.evaluate("error.reason"), containsString("is not accessible on master node")); } public void testNonexistentClient() throws Exception { @@ -181,15 +180,11 @@ private void testNonexistentClient(Boolean readonly) throws Exception { final var responseException = expectThrows(ResponseException.class, () -> client().performRequest(registerRequest)); assertEquals(RestStatus.INTERNAL_SERVER_ERROR.getStatus(), responseException.getResponse().getStatusLine().getStatusCode()); - assertThat( - responseException.getMessage(), - allOf( - containsString("repository_verification_exception"), - containsString("is not accessible on master node"), - containsString("illegal_argument_exception"), - containsString("Unknown s3 client name") - ) - ); + final var responseObjectPath = ObjectPath.createFromResponse(responseException.getResponse()); + assertThat(responseObjectPath.evaluate("error.type"), equalTo("repository_verification_exception")); + assertThat(responseObjectPath.evaluate("error.reason"), containsString("is not accessible on master node")); + assertThat(responseObjectPath.evaluate("error.caused_by.type"), equalTo("illegal_argument_exception")); + assertThat(responseObjectPath.evaluate("error.caused_by.reason"), containsString("Unknown s3 client name")); } public void testNonexistentSnapshot() throws Exception { @@ -212,7 +207,8 @@ private void testNonexistentSnapshot(Boolean readonly) throws Exception { final var getSnapshotRequest = new Request("GET", "/_snapshot/" + repositoryName + "/" + randomIdentifier()); final var getSnapshotException = expectThrows(ResponseException.class, () -> client().performRequest(getSnapshotRequest)); assertEquals(RestStatus.NOT_FOUND.getStatus(), getSnapshotException.getResponse().getStatusLine().getStatusCode()); - assertThat(getSnapshotException.getMessage(), containsString("snapshot_missing_exception")); + final var getResponseObjectPath = ObjectPath.createFromResponse(getSnapshotException.getResponse()); + assertThat(getResponseObjectPath.evaluate("error.type"), equalTo("snapshot_missing_exception")); final var restoreRequest = new Request("POST", "/_snapshot/" + repositoryName + "/" + randomIdentifier() + "/_restore"); if (randomBoolean()) { @@ -220,13 +216,15 @@ private void testNonexistentSnapshot(Boolean readonly) throws Exception { } final var restoreException = expectThrows(ResponseException.class, () -> client().performRequest(restoreRequest)); assertEquals(RestStatus.INTERNAL_SERVER_ERROR.getStatus(), restoreException.getResponse().getStatusLine().getStatusCode()); - assertThat(restoreException.getMessage(), containsString("snapshot_restore_exception")); + final var restoreResponseObjectPath = ObjectPath.createFromResponse(restoreException.getResponse()); + assertThat(restoreResponseObjectPath.evaluate("error.type"), equalTo("snapshot_restore_exception")); if (readonly != Boolean.TRUE) { final var deleteRequest = new Request("DELETE", "/_snapshot/" + repositoryName + "/" + randomIdentifier()); final var deleteException = expectThrows(ResponseException.class, () -> client().performRequest(deleteRequest)); assertEquals(RestStatus.NOT_FOUND.getStatus(), deleteException.getResponse().getStatusLine().getStatusCode()); - assertThat(deleteException.getMessage(), containsString("snapshot_missing_exception")); + final var deleteResponseObjectPath = ObjectPath.createFromResponse(deleteException.getResponse()); + assertThat(deleteResponseObjectPath.evaluate("error.type"), equalTo("snapshot_missing_exception")); } } }